From 7679af313a119ca096f4ab474833c67defb6e7ed Mon Sep 17 00:00:00 2001 From: alextheimer Date: Tue, 11 Jul 2023 15:38:02 -0700 Subject: [PATCH 01/39] Revert "maint(core): upgrade lucene to 9.7.0 (#1617)" (#1622) This reverts commit fa731f467f03c63948a9bef18920ba6a3196edf0. Co-authored-by: Amol Nayak --- .../scala/filodb.core/memstore/PartKeyLuceneIndex.scala | 7 +++---- project/Dependencies.scala | 4 ++-- 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala b/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala index 795a0b6446..89b5210ed2 100644 --- a/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala +++ b/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala @@ -24,7 +24,6 @@ import org.apache.lucene.analysis.standard.StandardAnalyzer import org.apache.lucene.document._ import org.apache.lucene.document.Field.Store import org.apache.lucene.facet.{FacetsCollector, FacetsConfig} -import org.apache.lucene.facet.FacetsConfig.DrillDownTermsIndexing import org.apache.lucene.facet.sortedset.{SortedSetDocValuesFacetCounts, SortedSetDocValuesFacetField} import org.apache.lucene.facet.sortedset.DefaultSortedSetDocValuesReaderState import org.apache.lucene.index._ @@ -304,7 +303,7 @@ class PartKeyLuceneIndex(ref: DatasetRef, if (name.nonEmpty && value.nonEmpty && (always || facetEnabledForLabel(name)) && value.length < FACET_FIELD_MAX_LEN) { - facetsConfig.setDrillDownTermsIndexing(name, DrillDownTermsIndexing.NONE) + facetsConfig.setRequireDimensionDrillDown(name, false) facetsConfig.setIndexFieldName(name, FACET_FIELD_PREFIX + name) document.add(new SortedSetDocValuesFacetField(name, value)) } @@ -499,14 +498,14 @@ class PartKeyLuceneIndex(ref: DatasetRef, .maximumSize(100) .recordStats() .build((key: (IndexReader, String)) => { - new DefaultSortedSetDocValuesReaderState(key._1, FACET_FIELD_PREFIX + key._2, new FacetsConfig()) + new DefaultSortedSetDocValuesReaderState(key._1, FACET_FIELD_PREFIX + key._2) }) private val readerStateCacheNonShardKeys: LoadingCache[(IndexReader, String), DefaultSortedSetDocValuesReaderState] = Caffeine.newBuilder() .maximumSize(200) .recordStats() .build((key: (IndexReader, String)) => { - new DefaultSortedSetDocValuesReaderState(key._1, FACET_FIELD_PREFIX + key._2, new FacetsConfig()) + new DefaultSortedSetDocValuesReaderState(key._1, FACET_FIELD_PREFIX + key._2) }) def labelNamesEfficient(colFilters: Seq[ColumnFilter], startTime: Long, endTime: Long): Seq[String] = { diff --git a/project/Dependencies.scala b/project/Dependencies.scala index 8c370fae55..81def65677 100644 --- a/project/Dependencies.scala +++ b/project/Dependencies.scala @@ -73,8 +73,8 @@ object Dependencies { "com.googlecode.javaewah" % "JavaEWAH" % "1.1.6" withJavadoc(), "com.github.rholder.fauxflake" % "fauxflake-core" % "1.1.0", "org.scalactic" %% "scalactic" % "3.2.0" withJavadoc(), - "org.apache.lucene" % "lucene-core" % "9.7.0" withJavadoc(), - "org.apache.lucene" % "lucene-facet" % "9.7.0" withJavadoc(), + "org.apache.lucene" % "lucene-core" % "8.8.2" withJavadoc(), + "org.apache.lucene" % "lucene-facet" % "8.8.2" withJavadoc(), "com.github.alexandrnikitin" %% "bloom-filter" % "0.11.0", "org.rocksdb" % "rocksdbjni" % "6.29.5", "com.esotericsoftware" % "kryo" % "4.0.0" excludeAll(excludeMinlog), From 80245ce17c655b7656eac52cdc73676010c4247f Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Wed, 12 Jul 2023 11:19:49 -0700 Subject: [PATCH 02/39] feat(core): Simplify configuration to scale Filodb horizontally (#1610) Today scaling filoDB horizontally involves re-calculation of memory settings in a manual way involving tribal knowledge. This PR aims to automate it and make it simple math. It is backward compatible and behind feature flag. * Min-num-nodes moved from dataset into server config. * New server config will drive automatic memory calculation * Each dataset requires another configuration that determines what fraction of resources each dataset gets. --- .../MemstoreCassandraSinkSpec.scala | 2 +- .../columnstore/OdpSpec.scala | 8 +- conf/timeseries-dev-source.conf | 3 + .../filodb.coordinator/FilodbSettings.scala | 1 + .../filodb.coordinator/IngestionActor.scala | 6 +- .../NodeCoordinatorActor.scala | 8 +- .../filodb.coordinator/ShardManager.scala | 3 +- .../v2/NewNodeCoordinatorActor.scala | 5 +- core/src/main/resources/filodb-defaults.conf | 39 ++++++++ core/src/main/scala/filodb.core/Utils.scala | 25 +++++ .../DownsampledTimeSeriesStore.scala | 2 +- .../memstore/OnDemandPagingShard.scala | 3 +- .../memstore/TimeSeriesMemStore.scala | 22 +++-- .../memstore/TimeSeriesShard.scala | 21 ++++- .../memstore/TimeSeriesStore.scala | 2 +- .../filodb.core/store/IngestionConfig.scala | 4 + core/src/test/resources/application_test.conf | 10 ++ .../memstore/DemandPagedChunkStoreSpec.scala | 2 +- .../TimeSeriesMemStoreForMetadataSpec.scala | 2 +- .../memstore/TimeSeriesMemStoreSpec.scala | 94 ++++++++++++++----- .../filodb.core/store/ColumnStoreSpec.scala | 4 +- .../filodb.jmh/HistogramIngestBenchmark.scala | 4 +- .../filodb.jmh/HistogramQueryBenchmark.scala | 4 +- .../scala/filodb.jmh/IngestionBenchmark.scala | 2 +- .../exec/InProcessPlanDispatcherSpec.scala | 8 +- .../filodb/query/exec/MetadataExecSpec.scala | 2 +- .../exec/MultiSchemaPartitionsExecSpec.scala | 8 +- .../query/exec/RemoteMetadataExecSpec.scala | 2 +- ...plitLocalPartitionDistConcatExecSpec.scala | 4 +- .../downsampler/DownsamplerMainSpec.scala | 14 +-- 30 files changed, 241 insertions(+), 73 deletions(-) diff --git a/cassandra/src/test/scala/filodb.cassandra/MemstoreCassandraSinkSpec.scala b/cassandra/src/test/scala/filodb.cassandra/MemstoreCassandraSinkSpec.scala index b23bb1623a..f11dffa5d9 100644 --- a/cassandra/src/test/scala/filodb.cassandra/MemstoreCassandraSinkSpec.scala +++ b/cassandra/src/test/scala/filodb.cassandra/MemstoreCassandraSinkSpec.scala @@ -34,7 +34,7 @@ class MemstoreCassandraSinkSpec extends AllTablesTest { } it("should flush MemStore data to C*, and be able to read back data from C* directly") { - memStore.setup(dataset1.ref, Schemas(dataset1.schema), 0, TestData.storeConf) + memStore.setup(dataset1.ref, Schemas(dataset1.schema), 0, TestData.storeConf, 1) memStore.store.sinkStats.chunksetsWritten.get shouldEqual 0 // Flush every ~50 records diff --git a/cassandra/src/test/scala/filodb.cassandra/columnstore/OdpSpec.scala b/cassandra/src/test/scala/filodb.cassandra/columnstore/OdpSpec.scala index 4216d83486..9891688269 100644 --- a/cassandra/src/test/scala/filodb.cassandra/columnstore/OdpSpec.scala +++ b/cassandra/src/test/scala/filodb.cassandra/columnstore/OdpSpec.scala @@ -96,7 +96,7 @@ class OdpSpec extends AnyFunSpec with Matchers with BeforeAndAfterAll with Scala val policy = new FixedMaxPartitionsEvictionPolicy(20) val memStore = new TimeSeriesMemStore(config, colStore, new InMemoryMetaStore(), Some(policy)) try { - memStore.setup(dataset.ref, schemas, 0, TestData.storeConf) + memStore.setup(dataset.ref, schemas, 0, TestData.storeConf, 1) memStore.recoverIndex(dataset.ref, 0).futureValue memStore.refreshIndexForTesting(dataset.ref) @@ -112,7 +112,7 @@ class OdpSpec extends AnyFunSpec with Matchers with BeforeAndAfterAll with Scala val policy = new FixedMaxPartitionsEvictionPolicy(20) val memStore = new TimeSeriesMemStore(config, colStore, new InMemoryMetaStore(), Some(policy)) try { - memStore.setup(dataset.ref, schemas, 0, TestData.storeConf) + memStore.setup(dataset.ref, schemas, 0, TestData.storeConf, 1) memStore.recoverIndex(dataset.ref, 0).futureValue memStore.refreshIndexForTesting(dataset.ref) @@ -134,7 +134,7 @@ class OdpSpec extends AnyFunSpec with Matchers with BeforeAndAfterAll with Scala val policy = new FixedMaxPartitionsEvictionPolicy(20) val memStore = new TimeSeriesMemStore(config, colStore, new InMemoryMetaStore(), Some(policy)) try { - memStore.setup(dataset.ref, schemas, 0, TestData.storeConf) + memStore.setup(dataset.ref, schemas, 0, TestData.storeConf, 1) memStore.recoverIndex(dataset.ref, 0).futureValue memStore.refreshIndexForTesting(dataset.ref) @@ -157,7 +157,7 @@ class OdpSpec extends AnyFunSpec with Matchers with BeforeAndAfterAll with Scala val policy = new FixedMaxPartitionsEvictionPolicy(20) val memStore = new TimeSeriesMemStore(config, colStore, new InMemoryMetaStore(), Some(policy)) try { - memStore.setup(dataset.ref, schemas, 0, TestData.storeConf) + memStore.setup(dataset.ref, schemas, 0, TestData.storeConf, 1) memStore.recoverIndex(dataset.ref, 0).futureValue memStore.refreshIndexForTesting(dataset.ref) diff --git a/conf/timeseries-dev-source.conf b/conf/timeseries-dev-source.conf index ea1a09d11d..da732b47ed 100644 --- a/conf/timeseries-dev-source.conf +++ b/conf/timeseries-dev-source.conf @@ -6,7 +6,10 @@ # Should not change once dataset has been set up on the server and data has been persisted to cassandra num-shards = 4 + # deprecated in favor of min-num-nodes-in-cluster config in filodb server config + # To be removed eventually. There is no reason to set a value for this for each dataset min-num-nodes = 2 + # Length of chunks to be written, roughly sourcefactory = "filodb.kafka.KafkaIngestionStreamFactory" diff --git a/coordinator/src/main/scala/filodb.coordinator/FilodbSettings.scala b/coordinator/src/main/scala/filodb.coordinator/FilodbSettings.scala index e543ccf8b8..e762dc94ce 100755 --- a/coordinator/src/main/scala/filodb.coordinator/FilodbSettings.scala +++ b/coordinator/src/main/scala/filodb.coordinator/FilodbSettings.scala @@ -55,6 +55,7 @@ final class FilodbSettings(val conf: Config) { lazy val hostList = config.as[Option[Seq[String]]]("cluster-discovery.host-list") lazy val localhostOrdinal = config.as[Option[Int]]("cluster-discovery.localhost-ordinal") + lazy val minNumNodes = config.as[Option[Int]]("min-num-nodes-in-cluster") /** * Returns IngestionConfig/dataset configuration from parsing dataset-configs file paths. diff --git a/coordinator/src/main/scala/filodb.coordinator/IngestionActor.scala b/coordinator/src/main/scala/filodb.coordinator/IngestionActor.scala index 0c67cedab5..183ca08ec7 100644 --- a/coordinator/src/main/scala/filodb.coordinator/IngestionActor.scala +++ b/coordinator/src/main/scala/filodb.coordinator/IngestionActor.scala @@ -37,8 +37,9 @@ object IngestionActor { source: NodeClusterActor.IngestionSource, downsample: DownsampleConfig, storeConfig: StoreConfig, + numShards: Int, statusActor: ActorRef): Props = - Props(new IngestionActor(ref, schemas, memStore, source, downsample, storeConfig, statusActor)) + Props(new IngestionActor(ref, schemas, memStore, source, downsample, storeConfig, numShards, statusActor)) } /** @@ -62,6 +63,7 @@ private[filodb] final class IngestionActor(ref: DatasetRef, source: NodeClusterActor.IngestionSource, downsample: DownsampleConfig, storeConfig: StoreConfig, + numShards: Int, statusActor: ActorRef) extends BaseActor { import IngestionActor._ @@ -170,7 +172,7 @@ private[filodb] final class IngestionActor(ref: DatasetRef, // scalastyle:off method.length private def startIngestion(shard: Int): Unit = { - try tsStore.setup(ref, schemas, shard, storeConfig, downsample) catch { + try tsStore.setup(ref, schemas, shard, storeConfig, numShards, downsample) catch { case ShardAlreadySetup(ds, s) => logger.warn(s"dataset=$ds shard=$s already setup, skipping....") return diff --git a/coordinator/src/main/scala/filodb.coordinator/NodeCoordinatorActor.scala b/coordinator/src/main/scala/filodb.coordinator/NodeCoordinatorActor.scala index f3abf68bbc..6896d73ae3 100644 --- a/coordinator/src/main/scala/filodb.coordinator/NodeCoordinatorActor.scala +++ b/coordinator/src/main/scala/filodb.coordinator/NodeCoordinatorActor.scala @@ -115,6 +115,7 @@ private[filodb] final class NodeCoordinatorActor(metaStore: MetaStore, setupDataset( dataset, ingestConfig.storeConfig, + ingestConfig.numShards, IngestionSource(ingestConfig.streamFactoryClass, ingestConfig.sourceConfig), ingestConfig.downsampleConfig) } @@ -141,6 +142,7 @@ private[filodb] final class NodeCoordinatorActor(metaStore: MetaStore, */ private def setupDataset(dataset: Dataset, storeConf: StoreConfig, + numShards: Int, source: IngestionSource, downsample: DownsampleConfig, schemaOverride: Boolean = false): Unit = { @@ -154,7 +156,7 @@ private[filodb] final class NodeCoordinatorActor(metaStore: MetaStore, val schemas = if (schemaOverride) Schemas(dataset.schema) else settings.schemas if (schemaOverride) logger.info(s"Overriding schemas from settings: this better be a test!") val props = IngestionActor.props(dataset.ref, schemas, memStore, - source, downsample, storeConf, statusActor.get) + source, downsample, storeConf, numShards, statusActor.get) val ingester = context.actorOf(props, s"$Ingestion-${dataset.name}") context.watch(ingester) ingesters(ref) = ingester @@ -187,7 +189,9 @@ private[filodb] final class NodeCoordinatorActor(metaStore: MetaStore, def ingestHandlers: Receive = LoggingReceive { case SetupDataset(dataset, resources, source, storeConf, downsample) => // used only in unit tests - if (!(ingesters contains dataset.ref)) { setupDataset(dataset, storeConf, source, downsample, true) } + if (!(ingesters contains dataset.ref)) { + setupDataset(dataset, storeConf, resources.numShards, source, downsample, true) + } case IngestRows(dataset, shard, rows) => withIngester(sender(), dataset) { _ ! IngestionActor.IngestRows(sender(), shard, rows) } diff --git a/coordinator/src/main/scala/filodb.coordinator/ShardManager.scala b/coordinator/src/main/scala/filodb.coordinator/ShardManager.scala index 12a68e24b7..dd63e13ab8 100644 --- a/coordinator/src/main/scala/filodb.coordinator/ShardManager.scala +++ b/coordinator/src/main/scala/filodb.coordinator/ShardManager.scala @@ -402,7 +402,8 @@ private[coordinator] final class ShardManager(settings: FilodbSettings, ackTo.foreach(_ ! DatasetExists(dataset.ref)) Map.empty case None => - val resources = DatasetResourceSpec(ingestConfig.numShards, ingestConfig.minNumNodes) + val minNumNodes = settings.minNumNodes.getOrElse(ingestConfig.minNumNodes) + val resources = DatasetResourceSpec(ingestConfig.numShards, minNumNodes) val mapper = new ShardMapper(resources.numShards) _shardMappers(dataset.ref) = mapper // Access the shardmapper through the HashMap so even if it gets replaced it will update the shard stats diff --git a/coordinator/src/main/scala/filodb/coordinator/v2/NewNodeCoordinatorActor.scala b/coordinator/src/main/scala/filodb/coordinator/v2/NewNodeCoordinatorActor.scala index 6462b356e4..e96b054160 100644 --- a/coordinator/src/main/scala/filodb/coordinator/v2/NewNodeCoordinatorActor.scala +++ b/coordinator/src/main/scala/filodb/coordinator/v2/NewNodeCoordinatorActor.scala @@ -84,7 +84,7 @@ private[filodb] final class NewNodeCoordinatorActor(memStore: TimeSeriesStore, .foreach { downsampleDataset => memStore.store.initialize(downsampleDataset, ingestConfig.numShards) } setupDataset( dataset, - ingestConfig.storeConfig, + ingestConfig.storeConfig, ingestConfig.numShards, IngestionSource(ingestConfig.streamFactoryClass, ingestConfig.sourceConfig), ingestConfig.downsampleConfig) initShards(dataset, ingestConfig) @@ -121,6 +121,7 @@ private[filodb] final class NewNodeCoordinatorActor(memStore: TimeSeriesStore, */ private def setupDataset(dataset: Dataset, storeConf: StoreConfig, + numShards: Int, source: IngestionSource, downsample: DownsampleConfig, schemaOverride: Boolean = false): Unit = { @@ -132,7 +133,7 @@ private[filodb] final class NewNodeCoordinatorActor(memStore: TimeSeriesStore, val schemas = if (schemaOverride) Schemas(dataset.schema) else settings.schemas if (schemaOverride) logger.info(s"Overriding schemas from settings: this better be a test!") val props = IngestionActor.props(dataset.ref, schemas, memStore, - source, downsample, storeConf, self) + source, downsample, storeConf, numShards, self) val ingester = context.actorOf(props, s"$Ingestion-${dataset.name}") context.watch(ingester) ingestionActors(ref) = ingester diff --git a/core/src/main/resources/filodb-defaults.conf b/core/src/main/resources/filodb-defaults.conf index f55e234594..a951a14769 100644 --- a/core/src/main/resources/filodb-defaults.conf +++ b/core/src/main/resources/filodb-defaults.conf @@ -1,5 +1,9 @@ filodb { v2-cluster-enabled = false + + # Number of nodes in cluster; used to calculate per-dhard resources based on how many shards assigned to node + # min-num-nodes-in-cluster = 2 + cluster-discovery { // set this to a smaller value (like 30s) at thee query entry points // if FiloDB HTTP API is indeed the query entry point, this should be overridden to small value @@ -685,6 +689,41 @@ filodb { # Note: this memory is shared across all configued datasets on a node. ingestion-buffer-mem-size = 200MB + memory-alloc { + # automatic memory allocation is enabled if true + automatic-alloc-enabled = false + + # if not provided this is calculated as ContainerOrNodeMemory - os-memory-needs - CurrentJVMHeapMemory + # available-memory-bytes = 5GB + + # memory dedicated for proper functioning of OS + os-memory-needs = 500MB + + # NOTE: In the three configs below, + # lucene-memory-percent + native-memory-manager-percent + block-memory-manager-percent + # should equal 100 + ############################################################## + # # # # + # LuceneMemPercent # NativeMemPercent # BlockMemPercent # + # # # # + ############################################################## + + # Memory percent of available-memory reserved for Lucene memory maps. + # Note we do not use this config to explicitly allocate space for lucene. + # But reserving this space ensures that more of the lucene memory maps are stored in memory + lucene-memory-percent = 5 + + # memory percent of available-memory reserved for native memory manager + # (used for partKeys, chunkMaps, chunkInfos, writeBuffers) + native-memory-manager-percent = 24 + + # Memory percent of available-memory reserved for block memory manager + # (used for storing chunks) + # This is divvied amongst datasets on the node per configuration for dataset + # The shards of the dataset on the node get even amount of memory from this fraction + block-memory-manager-percent = 73 + } + # At the cost of some extra heap memory, we can track queries holding shared lock for a long time # and starving the exclusive access of lock for eviction track-queries-holding-eviction-lock = true diff --git a/core/src/main/scala/filodb.core/Utils.scala b/core/src/main/scala/filodb.core/Utils.scala index 724c4556fb..7968fcd257 100644 --- a/core/src/main/scala/filodb.core/Utils.scala +++ b/core/src/main/scala/filodb.core/Utils.scala @@ -2,6 +2,7 @@ package filodb.core import java.lang.management.ManagementFactory +import com.typesafe.config.{Config, ConfigRenderOptions} import com.typesafe.scalalogging.StrictLogging object Utils extends StrictLogging { @@ -13,4 +14,28 @@ object Utils extends StrictLogging { if (cpuTimeEnabled) threadMbean.getCurrentThreadCpuTime else System.nanoTime() } + + def calculateAvailableOffHeapMemory(filodbConfig: Config): Long = { + val containerMemory = ManagementFactory.getOperatingSystemMXBean() + .asInstanceOf[com.sun.management.OperatingSystemMXBean].getTotalPhysicalMemorySize() + val currentJavaHeapMemory = Runtime.getRuntime().maxMemory() + val osMemoryNeeds = filodbConfig.getMemorySize("memstore.memory-alloc.os-memory-needs").toBytes + logger.info(s"Detected available memory containerMemory=$containerMemory" + + s" currentJavaHeapMemory=$currentJavaHeapMemory osMemoryNeeds=$osMemoryNeeds") + + logger.info(s"Memory Alloc Options: " + + s"${filodbConfig.getConfig("memstore.memory-alloc").root().render(ConfigRenderOptions.concise())}") + + val availableMem = if (filodbConfig.hasPath("memstore.memory-alloc.available-memory-bytes")) { + val avail = filodbConfig.getMemorySize("memstore.memory-alloc.available-memory-bytes").toBytes + logger.info(s"Using automatic-memory-config using overridden memory-alloc.available-memory $avail") + avail + } else { + logger.info(s"Using automatic-memory-config using without available memory override") + containerMemory - currentJavaHeapMemory - osMemoryNeeds + } + logger.info(s"Available memory calculated or configured as $availableMem") + availableMem + } + } diff --git a/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesStore.scala b/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesStore.scala index 9702442bec..5f83c06c9c 100644 --- a/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesStore.scala +++ b/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesStore.scala @@ -59,7 +59,7 @@ extends TimeSeriesStore with StrictLogging { override def metastore: MetaStore = ??? // Not needed // TODO: Change the API to return Unit Or ShardAlreadySetup, instead of throwing. Make idempotent. - def setup(ref: DatasetRef, schemas: Schemas, shard: Int, storeConf: StoreConfig, + def setup(ref: DatasetRef, schemas: Schemas, shard: Int, storeConf: StoreConfig, numShards: Int, downsampleConfig: DownsampleConfig = DownsampleConfig.disabled): Unit = synchronized { val shards = datasets.getOrElseUpdate(ref, new NonBlockingHashMapLong[DownsampledTimeSeriesShard](32, false)) val quotaSource = quotaSources.getOrElseUpdate(ref, diff --git a/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala b/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala index 91ee5c0e70..760fd9e823 100644 --- a/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala +++ b/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala @@ -26,6 +26,7 @@ import filodb.memory.NativeMemoryManager class OnDemandPagingShard(ref: DatasetRef, schemas: Schemas, storeConfig: StoreConfig, + numShards: Int, quotaSource: QuotaSource, shardNum: Int, bufferMemoryManager: NativeMemoryManager, @@ -34,7 +35,7 @@ class OnDemandPagingShard(ref: DatasetRef, evictionPolicy: PartitionEvictionPolicy, filodbConfig: Config) (implicit ec: ExecutionContext) extends -TimeSeriesShard(ref, schemas, storeConfig, quotaSource, shardNum, bufferMemoryManager, rawStore, +TimeSeriesShard(ref, schemas, storeConfig, numShards, quotaSource, shardNum, bufferMemoryManager, rawStore, metastore, evictionPolicy, filodbConfig)(ec) { import TimeSeriesShard._ import FiloSchedulers._ diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala index fd3f018b91..496a026d24 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala @@ -11,7 +11,7 @@ import monix.execution.{CancelableFuture, Scheduler} import monix.reactive.Observable import org.jctools.maps.NonBlockingHashMapLong -import filodb.core.{DatasetRef, QueryTimeoutException, Response, Types} +import filodb.core.{DatasetRef, QueryTimeoutException, Response, Types, Utils} import filodb.core.downsample.DownsampleConfig import filodb.core.memstore.ratelimit.{CardinalityRecord, ConfigQuotaSource} import filodb.core.metadata.Schemas @@ -33,7 +33,6 @@ extends TimeSeriesStore with StrictLogging { type Shards = NonBlockingHashMapLong[TimeSeriesShard] private val datasets = new HashMap[DatasetRef, Shards] - private val datasetMemFactories = new HashMap[DatasetRef, NativeMemoryManager] private val quotaSources = new HashMap[DatasetRef, ConfigQuotaSource] val stats = new ChunkSourceStats @@ -44,7 +43,18 @@ extends TimeSeriesStore with StrictLogging { private val partEvictionPolicy = evictionPolicy.getOrElse( new CompositeEvictionPolicy(ensureTspHeadroomPercent, ensureNmmHeadroomPercent)) - private lazy val ingestionMemory = filodbConfig.getMemorySize("memstore.ingestion-buffer-mem-size").toBytes + private[memstore] lazy val ingestionMemory = { + if (filodbConfig.getBoolean("memstore.memory-alloc.automatic-alloc-enabled")) { + val availableMemoryBytes: Long = Utils.calculateAvailableOffHeapMemory(filodbConfig) + val nativeMemoryManagerPercent = filodbConfig.getDouble("memstore.memory-alloc.native-memory-manager-percent") + val blockMemoryManagerPercent = filodbConfig.getDouble("memstore.memory-alloc.block-memory-manager-percent") + val lucenePercent = filodbConfig.getDouble("memstore.memory-alloc.lucene-memory-percent") + require(nativeMemoryManagerPercent + blockMemoryManagerPercent + lucenePercent == 100.0, + s"Configured Block($nativeMemoryManagerPercent), Native($nativeMemoryManagerPercent) " + + s"and Lucene($lucenePercent) memory percents don't sum to 100.0") + (availableMemoryBytes * nativeMemoryManagerPercent / 100).toLong + } else filodbConfig.getMemorySize("memstore.ingestion-buffer-mem-size").toBytes + } private[this] lazy val ingestionMemFactory: NativeMemoryManager = { logger.info(s"Allocating $ingestionMemory bytes for WriteBufferPool/PartitionKeys") @@ -67,7 +77,7 @@ extends TimeSeriesStore with StrictLogging { } // TODO: Change the API to return Unit Or ShardAlreadySetup, instead of throwing. Make idempotent. - def setup(ref: DatasetRef, schemas: Schemas, shard: Int, storeConf: StoreConfig, + def setup(ref: DatasetRef, schemas: Schemas, shard: Int, storeConf: StoreConfig, numShards: Int, downsample: DownsampleConfig = DownsampleConfig.disabled): Unit = synchronized { val shards = datasets.getOrElseUpdate(ref, new NonBlockingHashMapLong[TimeSeriesShard](32, false)) val quotaSource = quotaSources.getOrElseUpdate(ref, @@ -75,8 +85,8 @@ extends TimeSeriesStore with StrictLogging { if (shards.containsKey(shard)) { throw ShardAlreadySetup(ref, shard) } else { - val tsdb = new OnDemandPagingShard(ref, schemas, storeConf, quotaSource, shard, ingestionMemFactory, store, - metastore, partEvictionPolicy, filodbConfig) + val tsdb = new OnDemandPagingShard(ref, schemas, storeConf, numShards, quotaSource, shard, + ingestionMemFactory, store, metastore, partEvictionPolicy, filodbConfig) shards.put(shard, tsdb) } } diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala index 2c5dc84bad..f11c957d33 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala @@ -258,6 +258,7 @@ case class TimeSeriesShardInfo(shardNum: Int, class TimeSeriesShard(val ref: DatasetRef, val schemas: Schemas, val storeConfig: StoreConfig, + numShards: Int, quotaSource: QuotaSource, val shardNum: Int, val bufferMemoryManager: NativeMemoryManager, @@ -365,7 +366,25 @@ class TimeSeriesShard(val ref: DatasetRef, val ingestSched = Scheduler.singleThread(s"$IngestSchedName-$ref-$shardNum", reporter = UncaughtExceptionReporter(logger.error("Uncaught Exception in TimeSeriesShard.ingestSched", _))) - private val blockMemorySize = storeConfig.shardMemSize + private[memstore] val blockMemorySize = { + val size = if (filodbConfig.getBoolean("memstore.memory-alloc.automatic-alloc-enabled")) { + val numNodes = filodbConfig.getInt("min-num-nodes-in-cluster") + val availableMemoryBytes: Long = Utils.calculateAvailableOffHeapMemory(filodbConfig) + val blockMemoryManagerPercent = filodbConfig.getDouble("memstore.memory-alloc.block-memory-manager-percent") + val blockMemForDatasetPercent = storeConfig.shardMemPercent // fraction of block memory for this dataset + val numShardsPerNode = Math.ceil(numShards / numNodes.toDouble) + logger.info(s"Calculating Block memory size with automatic allocation strategy. " + + s"Dataset dataset=$ref has blockMemForDatasetPercent=$blockMemForDatasetPercent " + + s"numShardsPerNode=$numShardsPerNode") + (availableMemoryBytes * blockMemoryManagerPercent * + blockMemForDatasetPercent / 100 / 100 / numShardsPerNode).toLong + } else { + storeConfig.shardMemSize + } + logger.info(s"Block Memory for dataset=$ref shard=$shardNum bytesAllocated=$size") + size + } + protected val numGroups = storeConfig.groupsPerShard private val chunkRetentionHours = (storeConfig.diskTTLSeconds / 3600).toInt private[memstore] val pagingEnabled = storeConfig.demandPagingEnabled diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesStore.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesStore.scala index dfa0f1f0cb..c84a7298c9 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesStore.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesStore.scala @@ -75,7 +75,7 @@ trait TimeSeriesStore extends ChunkSource { * @param downsampleConfig configuration for downsampling operation. By default it is disabled. */ def setup(ref: DatasetRef, schemas: Schemas, shard: Int, - storeConf: StoreConfig, + storeConf: StoreConfig, numShards: Int, downsampleConfig: DownsampleConfig = DownsampleConfig.disabled): Unit /** diff --git a/core/src/main/scala/filodb.core/store/IngestionConfig.scala b/core/src/main/scala/filodb.core/store/IngestionConfig.scala index 1d530c133c..06802c329e 100644 --- a/core/src/main/scala/filodb.core/store/IngestionConfig.scala +++ b/core/src/main/scala/filodb.core/store/IngestionConfig.scala @@ -17,6 +17,7 @@ final case class StoreConfig(flushInterval: FiniteDuration, maxBlobBufferSize: Int, // Number of bytes to allocate to chunk storage in each shard shardMemSize: Long, + shardMemPercent: Double, maxBufferPoolSize: Int, groupsPerShard: Int, numPagesPerBlock: Int, @@ -45,6 +46,7 @@ final case class StoreConfig(flushInterval: FiniteDuration, "max-chunks-size" -> maxChunksSize, "max-blob-buffer-size" -> maxBlobBufferSize, "shard-mem-size" -> shardMemSize, + "shard-mem-percent" -> shardMemPercent, "max-buffer-pool-size" -> maxBufferPoolSize, "groups-per-shard" -> groupsPerShard, "max-chunk-time" -> (maxChunkTime.toSeconds + "s"), @@ -81,6 +83,7 @@ object StoreConfig { |max-blob-buffer-size = 15000 |max-buffer-pool-size = 10000 |groups-per-shard = 60 + |shard-mem-percent = 100 # assume only one dataset per node by default |num-block-pages = 100 |failure-retries = 3 |retry-delay = 15 seconds @@ -119,6 +122,7 @@ object StoreConfig { config.getInt("max-chunks-size"), config.getInt("max-blob-buffer-size"), config.getMemorySize("shard-mem-size").toBytes, + config.getDouble("shard-mem-percent"), config.getInt("max-buffer-pool-size"), config.getInt("groups-per-shard"), config.getInt("num-block-pages"), diff --git a/core/src/test/resources/application_test.conf b/core/src/test/resources/application_test.conf index 52e14fbebf..dae84e3bdf 100644 --- a/core/src/test/resources/application_test.conf +++ b/core/src/test/resources/application_test.conf @@ -88,6 +88,15 @@ filodb { spark.dataset-ops-timeout = 15s memstore { + memory-alloc { + automatic-alloc-enabled = false + available-memory-bytes = 1GB + os-memory-needs = 500MB + lucene-memory-percent = 20 + native-memory-manager-percent = 20 + block-memory-manager-percent = 60 + } + flush-task-parallelism = 1 ensure-block-memory-headroom-percent = 5 ensure-tsp-count-headroom-percent = 5 @@ -97,6 +106,7 @@ filodb { track-queries-holding-eviction-lock = false index-faceting-enabled-shard-key-labels = true index-faceting-enabled-for-all-labels = true + } tasks { diff --git a/core/src/test/scala/filodb.core/memstore/DemandPagedChunkStoreSpec.scala b/core/src/test/scala/filodb.core/memstore/DemandPagedChunkStoreSpec.scala index ad13ef95ac..defaf44269 100644 --- a/core/src/test/scala/filodb.core/memstore/DemandPagedChunkStoreSpec.scala +++ b/core/src/test/scala/filodb.core/memstore/DemandPagedChunkStoreSpec.scala @@ -26,7 +26,7 @@ class DemandPagedChunkStoreSpec extends AnyFunSpec with AsyncTest { |shard-mem-size = 200MB""".stripMargin) .withFallback(TestData.sourceConf.getConfig("store")) - memStore.setup(dataset1.ref, Schemas(schema1), 0, StoreConfig(sourceConf)) + memStore.setup(dataset1.ref, Schemas(schema1), 0, StoreConfig(sourceConf), 1) val onDemandPartMaker = memStore.getShardE(dataset1.ref, 0).partitionMaker after { diff --git a/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreForMetadataSpec.scala b/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreForMetadataSpec.scala index 8a3797557b..f1500e0c1e 100644 --- a/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreForMetadataSpec.scala +++ b/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreForMetadataSpec.scala @@ -39,7 +39,7 @@ class TimeSeriesMemStoreForMetadataSpec extends AnyFunSpec with Matchers with Sc val container = createRecordContainer(0, 10) override def beforeAll(): Unit = { - memStore.setup(timeseriesDataset.ref, Schemas(timeseriesSchema), 0, TestData.storeConf) + memStore.setup(timeseriesDataset.ref, Schemas(timeseriesSchema), 0, TestData.storeConf, 1) memStore.ingest(timeseriesDataset.ref, 0, SomeData(container, 0)) memStore.refreshIndexForTesting(timeseriesDataset.ref) } diff --git a/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala b/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala index b199600fbd..1654c81c51 100644 --- a/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala +++ b/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala @@ -46,9 +46,9 @@ class TimeSeriesMemStoreSpec extends AnyFunSpec with Matchers with BeforeAndAfte val schemas1 = Schemas(schema1) it("should detect duplicate setup") { - memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf) + memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf, 1) try { - memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf) + memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf, 1) fail() } catch { case e: ShardAlreadySetup => { } // expected @@ -58,7 +58,7 @@ class TimeSeriesMemStoreSpec extends AnyFunSpec with Matchers with BeforeAndAfte // Look mama! Real-time time series ingestion and querying across multiple partitions! it("should ingest into multiple series and be able to query across all partitions in real time") { - memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf) + memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf, 1) val rawData = multiSeriesData().take(20) val data = records(dataset1, rawData) // 2 records per series x 10 series memStore.ingest(dataset1.ref, 0, data) @@ -81,7 +81,7 @@ class TimeSeriesMemStoreSpec extends AnyFunSpec with Matchers with BeforeAndAfte } it("should ingest into multiple series and query across partitions") { - memStore.setup(dataset1.ref, schemas1, 1, TestData.storeConf) + memStore.setup(dataset1.ref, schemas1, 1, TestData.storeConf, 1) val data = records(dataset1, linearMultiSeries().take(20)) // 2 records per series x 10 series memStore.ingest(dataset1.ref, 1, data) @@ -97,7 +97,7 @@ class TimeSeriesMemStoreSpec extends AnyFunSpec with Matchers with BeforeAndAfte } it("should ingest map/tags column as partition key and aggregate") { - memStore.setup(dataset2.ref, schemas2h, 0, TestData.storeConf) + memStore.setup(dataset2.ref, schemas2h, 0, TestData.storeConf, 1) val data = records(dataset2, withMap(linearMultiSeries().take(20))) // 2 records per series x 10 series memStore.ingest(dataset2.ref, 0, data) @@ -109,7 +109,7 @@ class TimeSeriesMemStoreSpec extends AnyFunSpec with Matchers with BeforeAndAfte } it("should ingest histograms and read them back properly") { - memStore.setup(histDataset.ref, schemas2h, 0, TestData.storeConf) + memStore.setup(histDataset.ref, schemas2h, 0, TestData.storeConf, 1) val data = linearHistSeries().take(40) memStore.ingest(histDataset.ref, 0, records(histDataset, data)) memStore.refreshIndexForTesting(histDataset.ref) @@ -137,7 +137,7 @@ class TimeSeriesMemStoreSpec extends AnyFunSpec with Matchers with BeforeAndAfte it("should ingest multiple schemas simultaneously into one shard") { val ref = dataset2.ref - memStore.setup(ref, schemas2h, 0, TestData.storeConf) + memStore.setup(ref, schemas2h, 0, TestData.storeConf, 1) val data = linearHistSeries().take(40) memStore.ingest(ref, 0, records(histDataset, data)) val data2 = records(dataset2, withMap(linearMultiSeries()).take(30)) // 3 records per series x 10 series @@ -151,14 +151,14 @@ class TimeSeriesMemStoreSpec extends AnyFunSpec with Matchers with BeforeAndAfte } it("should be able to handle nonexistent partition keys") { - memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf) + memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf, 1) val q = memStore.scanRows(dataset1, Seq(1), SinglePartitionScan(Array[Byte]())) q.toBuffer.length should equal (0) } it("should ingest into multiple series and be able to query on one partition in real time") { - memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf) + memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf, 1) val data = multiSeriesData().take(20) // 2 records per series x 10 series memStore.ingest(dataset1.ref, 0, records(dataset1, data)) @@ -174,7 +174,7 @@ class TimeSeriesMemStoreSpec extends AnyFunSpec with Matchers with BeforeAndAfte } it("should query on multiple partitions using filters") { - memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf) + memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf, 1) val data = records(dataset1, linearMultiSeries().take(20)) // 2 records per series x 10 series memStore.ingest(dataset1.ref, 0, data) memStore.refreshIndexForTesting(dataset1.ref) @@ -186,8 +186,8 @@ class TimeSeriesMemStoreSpec extends AnyFunSpec with Matchers with BeforeAndAfte } it("should ingest into multiple shards, getScanSplits, query, get index info from shards") { - memStore.setup(dataset2.ref, schemas2h, 0, TestData.storeConf) - memStore.setup(dataset2.ref, schemas2h, 1, TestData.storeConf) + memStore.setup(dataset2.ref, schemas2h, 0, TestData.storeConf, 2) + memStore.setup(dataset2.ref, schemas2h, 1, TestData.storeConf, 2) val data = records(dataset2, withMap(linearMultiSeries()).take(20)) // 2 records per series x 10 series memStore.ingest(dataset2.ref, 0, data) val data2 = records(dataset2, withMap(linearMultiSeries(200000L, 6), 6).take(20)) // 5 series only @@ -214,7 +214,7 @@ class TimeSeriesMemStoreSpec extends AnyFunSpec with Matchers with BeforeAndAfte } it("should handle errors from ingestStream") { - memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf) + memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf, 1) val errStream = Observable.fromIterable(groupedRecords(dataset1, linearMultiSeries())) .endWithError(new NumberFormatException) val fut = memStore.startIngestion(dataset1.ref, 0, errStream, s) @@ -224,7 +224,7 @@ class TimeSeriesMemStoreSpec extends AnyFunSpec with Matchers with BeforeAndAfte } it("should ingestStream and flush on interval") { - memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf) + memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf, 1) val initChunksWritten = chunksetsWritten val stream = Observable.fromIterable(groupedRecords(dataset1, linearMultiSeries())) @@ -245,7 +245,7 @@ class TimeSeriesMemStoreSpec extends AnyFunSpec with Matchers with BeforeAndAfte it("should flush dirty part keys during start-ingestion, end-ingestion and re-ingestion") { memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf.copy(groupsPerShard = 2, diskTTLSeconds = 1.hour.toSeconds.toInt, - flushInterval = 10.minutes)) + flushInterval = 10.minutes), 1) Thread sleep 1000 val numPartKeysWritten = partKeysWritten val tsShard = memStore.asInstanceOf[TimeSeriesMemStore].getShard(dataset1.ref, 0).get @@ -302,6 +302,54 @@ class TimeSeriesMemStoreSpec extends AnyFunSpec with Matchers with BeforeAndAfte 10.until(20).foreach {i => tsShard.activelyIngesting(i) shouldEqual false} } + it("should configure memory automatically when enabled") { + val colStore = new NullColumnStore() + + val config1 = ConfigFactory.parseString( + """ + |min-num-nodes-in-cluster = 32 + |memstore { + | memory-alloc { + | automatic-alloc-enabled = true + | available-memory-bytes = 1GB + | lucene-memory-percent = 10 + | native-memory-manager-percent = 30 + | block-memory-manager-percent = 60 + | } + |} + |""".stripMargin).withFallback(config) + + val memStore1 = new TimeSeriesMemStore(config1, colStore, new InMemoryMetaStore()) + memStore1.setup(dataset1.ref, schemas1, 0, TestData.storeConf.copy(groupsPerShard = 2, + diskTTLSeconds = 1.hour.toSeconds.toInt, + flushInterval = 10.minutes, shardMemPercent = 40), 256) + + memStore1.ingestionMemory shouldEqual 300000000 // 1GB * 30% + val tsShard = memStore1.getShard(dataset1.ref, 0).get + tsShard.blockMemorySize shouldEqual 30000000 // 1GB * 60% (for block memory) * 40% (for dataset memory) / ceil(256/32) + + memStore1.shutdown() + + // Expand cluster by only changing min-num-nodes-in-cluster + // now each shard should get more memory since fewer shards per node + + val config2 = ConfigFactory.parseString( + """ + |min-num-nodes-in-cluster = 45 + |""".stripMargin).withFallback(config1) + + val memStore2 = new TimeSeriesMemStore(config2, colStore, new InMemoryMetaStore()) + memStore2.setup(dataset1.ref, schemas1, 0, TestData.storeConf.copy(groupsPerShard = 2, + diskTTLSeconds = 1.hour.toSeconds.toInt, + flushInterval = 10.minutes, shardMemPercent = 40), 256) + + memStore2.ingestionMemory shouldEqual 300000000 // 1GB * 30% + val tsShard2 = memStore2.getShard(dataset1.ref, 0).get + tsShard2.blockMemorySize shouldEqual 40000000 // 1GB * 60% (for block memory) * 40% (for dataset memory) / ceil(256/45) + + memStore2.shutdown() + } + it("should recover index data from col store correctly") { val partBuilder = new RecordBuilder(TestData.nativeMem) @@ -323,7 +371,7 @@ class TimeSeriesMemStoreSpec extends AnyFunSpec with Matchers with BeforeAndAfte val memStore = new TimeSeriesMemStore(config, colStore, new InMemoryMetaStore()) memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf.copy(groupsPerShard = 2, diskTTLSeconds = 1.hour.toSeconds.toInt, - flushInterval = 10.minutes)) + flushInterval = 10.minutes), 1) Thread sleep 1000 val tsShard = memStore.asInstanceOf[TimeSeriesMemStore].getShard(dataset1.ref, 0).get @@ -346,7 +394,7 @@ class TimeSeriesMemStoreSpec extends AnyFunSpec with Matchers with BeforeAndAfte } it("should lookupPartitions and return correct PartLookupResult") { - memStore.setup(dataset2.ref, schemas2h, 0, TestData.storeConf) + memStore.setup(dataset2.ref, schemas2h, 0, TestData.storeConf, 1) val data = records(dataset2, withMap(linearMultiSeries().take(20))) // 2 records per series x 10 series memStore.ingest(dataset2.ref, 0, data) @@ -372,7 +420,7 @@ class TimeSeriesMemStoreSpec extends AnyFunSpec with Matchers with BeforeAndAfte // 0 -> 2L, 1 -> 4L, 2 -> 6L, 3 -> 8L // A whole bunch of records should be skipped. However, remember that each group of 5 records gets one offset. - memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf) + memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf, 1) val initChunksWritten = chunksetsWritten val checkpoints = Map(0 -> 2L, 1 -> 21L, 2 -> 6L, 3 -> 8L) @@ -396,7 +444,7 @@ class TimeSeriesMemStoreSpec extends AnyFunSpec with Matchers with BeforeAndAfte } it("should recoverStream after timeout, returns endOffset to start normal ingestion") { - memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf) + memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf, 1) val checkpoints = Map(0 -> 2L, 1 -> 21L, 2 -> 6L, 3 -> 8L) val stream = Observable.never // "never" to mimic no data in stream source @@ -410,7 +458,7 @@ class TimeSeriesMemStoreSpec extends AnyFunSpec with Matchers with BeforeAndAfte } it("should truncate shards properly") { - memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf) + memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf, 1) val data = records(dataset1, multiSeriesData().take(20)) // 2 records per series x 10 series memStore.ingest(dataset1.ref, 0, data) @@ -437,7 +485,7 @@ class TimeSeriesMemStoreSpec extends AnyFunSpec with Matchers with BeforeAndAfte } it("should be able to evict partitions properly on ingestion and on ODP") { - memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf) + memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf, 1) val shard = memStore.getShardE(dataset1.ref, 0) // Ingest normal multi series data with 10 partitions. Should have 10 partitions. @@ -526,7 +574,7 @@ class TimeSeriesMemStoreSpec extends AnyFunSpec with Matchers with BeforeAndAfte it("should assign same previously assigned partId using bloom filter when evicted series starts re-ingesting") { - memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf) + memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf, 1) val shard0 = memStore.getShardE(dataset1.ref, 0) // Ingest normal multi series data with 10 partitions. Should have 10 partitions. @@ -568,7 +616,7 @@ class TimeSeriesMemStoreSpec extends AnyFunSpec with Matchers with BeforeAndAfte try { // Ingest >250 partitions. Note how much memory is left after all the allocations - store2.setup(dataset1.ref, schemas1, 0, TestData.storeConf) + store2.setup(dataset1.ref, schemas1, 0, TestData.storeConf, 1) val shard = store2.getShardE(dataset1.ref, 0) // Ingest normal multi series data with 10 partitions. Should have 10 partitions. diff --git a/core/src/test/scala/filodb.core/store/ColumnStoreSpec.scala b/core/src/test/scala/filodb.core/store/ColumnStoreSpec.scala index 57a15afecf..8257cb8691 100644 --- a/core/src/test/scala/filodb.core/store/ColumnStoreSpec.scala +++ b/core/src/test/scala/filodb.core/store/ColumnStoreSpec.scala @@ -126,7 +126,7 @@ with BeforeAndAfter with BeforeAndAfterAll with ScalaFutures { // TODO: FilteredPartitionScan() for ColumnStores does not work without an index right now ignore should "filter rows written with single partition key" in { import GdeltTestData._ - memStore.setup(dataset2.ref, schemas, 0, TestData.storeConf) + memStore.setup(dataset2.ref, schemas, 0, TestData.storeConf, 1) val stream = Observable.now(records(dataset2)) // Force flush of all groups at end memStore.startIngestion(dataset2.ref, 0, stream, s, Task {}).futureValue @@ -143,7 +143,7 @@ with BeforeAndAfter with BeforeAndAfterAll with ScalaFutures { // "rangeVectors api" should "return Range Vectors for given filter and read all rows" in { ignore should "return Range Vectors for given filter and read all rows" in { import GdeltTestData._ - memStore.setup(dataset2.ref, schemas, 0, TestData.storeConf) + memStore.setup(dataset2.ref, schemas, 0, TestData.storeConf, 1) val stream = Observable.now(records(dataset2)) // Force flush of all groups at end memStore.startIngestion(dataset2.ref, 0, stream, s, Task {}).futureValue diff --git a/jmh/src/main/scala/filodb.jmh/HistogramIngestBenchmark.scala b/jmh/src/main/scala/filodb.jmh/HistogramIngestBenchmark.scala index e4eb963d14..1f5437952b 100644 --- a/jmh/src/main/scala/filodb.jmh/HistogramIngestBenchmark.scala +++ b/jmh/src/main/scala/filodb.jmh/HistogramIngestBenchmark.scala @@ -68,8 +68,8 @@ class HistogramIngestBenchmark { val policy = new FixedMaxPartitionsEvictionPolicy(1000) val memStore = new TimeSeriesMemStore(config, new NullColumnStore, new InMemoryMetaStore(), Some(policy)) val ingestConf = TestData.storeConf.copy(shardMemSize = 512 * 1024 * 1024, maxChunksSize = 100) - memStore.setup(histDataset.ref, Schemas(histDataset.schema), 0, ingestConf) - memStore.setup(promDataset.ref, Schemas(promDataset.schema), 0, ingestConf) + memStore.setup(histDataset.ref, Schemas(histDataset.schema), 0, ingestConf, 1) + memStore.setup(promDataset.ref, Schemas(promDataset.schema), 0, ingestConf, 1) val hShard = memStore.getShardE(histDataset.ref, 0) val pShard = memStore.getShardE(promDataset.ref, 0) diff --git a/jmh/src/main/scala/filodb.jmh/HistogramQueryBenchmark.scala b/jmh/src/main/scala/filodb.jmh/HistogramQueryBenchmark.scala index d87b2b7ca8..545027a85a 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 { histSchemaData.take(10 * 180).foreach(histSchemaBuilder.addFromReader(_, histDataset.schema)) val histSchemas = Schemas(histDataset.schema) - memStore.setup(histDataset.ref, histSchemas, 0, ingestConf) + memStore.setup(histDataset.ref, histSchemas, 0, ingestConf, 1) val hShard = memStore.getShardE(histDataset.ref, 0) histSchemaBuilder.allContainers.foreach { c => hShard.ingest(c, 0) } memStore.refreshIndexForTesting(histDataset.ref) // commit lucene index @@ -66,7 +66,7 @@ class HistogramQueryBenchmark { val promBuilder = new RecordBuilder(MemFactory.onHeapFactory, 4200000) promData.take(10*66*180).foreach(promBuilder.addFromReader(_, promDataset.schema)) - memStore.setup(promDataset.ref, promSchemas, 0, ingestConf) + memStore.setup(promDataset.ref, promSchemas, 0, ingestConf, 1) val pShard = memStore.getShardE(promDataset.ref, 0) promBuilder.allContainers.foreach { c => pShard.ingest(c, 0) } memStore.refreshIndexForTesting(promDataset.ref) // commit lucene index diff --git a/jmh/src/main/scala/filodb.jmh/IngestionBenchmark.scala b/jmh/src/main/scala/filodb.jmh/IngestionBenchmark.scala index 286e607c5e..2d3c86b129 100644 --- a/jmh/src/main/scala/filodb.jmh/IngestionBenchmark.scala +++ b/jmh/src/main/scala/filodb.jmh/IngestionBenchmark.scala @@ -61,7 +61,7 @@ class IngestionBenchmark { val policy = new FixedMaxPartitionsEvictionPolicy(100) val memStore = new TimeSeriesMemStore(config, new NullColumnStore, new InMemoryMetaStore(), Some(policy)) val ingestConf = TestData.storeConf.copy(shardMemSize = 512 * 1024 * 1024, maxChunksSize = 200) - memStore.setup(dataset1.ref, Schemas(dataset1.schema), 0, ingestConf) + memStore.setup(dataset1.ref, Schemas(dataset1.schema), 0, ingestConf, 1) val shard = memStore.getShardE(dataset1.ref, 0) diff --git a/query/src/test/scala/filodb/query/exec/InProcessPlanDispatcherSpec.scala b/query/src/test/scala/filodb/query/exec/InProcessPlanDispatcherSpec.scala index ca6cceb872..607e94d8c3 100644 --- a/query/src/test/scala/filodb/query/exec/InProcessPlanDispatcherSpec.scala +++ b/query/src/test/scala/filodb/query/exec/InProcessPlanDispatcherSpec.scala @@ -36,13 +36,13 @@ class InProcessPlanDispatcherSpec extends AnyFunSpec import filodb.core.{MachineMetricsData => MMD} override def beforeAll(): Unit = { - memStore.setup(timeseriesDataset.ref, Schemas(timeseriesSchema), 0, TestData.storeConf) + memStore.setup(timeseriesDataset.ref, Schemas(timeseriesSchema), 0, TestData.storeConf, 1) memStore.ingest(timeseriesDataset.ref, 0, SomeData(container, 0)) - memStore.setup(MMD.dataset1.ref, Schemas(MMD.schema1), 0, TestData.storeConf) + memStore.setup(MMD.dataset1.ref, Schemas(MMD.schema1), 0, TestData.storeConf, 1) memStore.ingest(MMD.dataset1.ref, 0, mmdSomeData) - memStore.setup(MMD.histDataset.ref, Schemas(MMD.histDataset.schema), 0, TestData.storeConf) + memStore.setup(MMD.histDataset.ref, Schemas(MMD.histDataset.schema), 0, TestData.storeConf, 1) memStore.ingest(MMD.histDataset.ref, 0, MMD.records(MMD.histDataset, histData)) - memStore.setup(MMD.histMaxDS.ref, Schemas(MMD.histMaxDS.schema), 0, TestData.storeConf) + memStore.setup(MMD.histMaxDS.ref, Schemas(MMD.histMaxDS.schema), 0, TestData.storeConf, 1) memStore.ingest(MMD.histMaxDS.ref, 0, MMD.records(MMD.histMaxDS, histMaxData)) memStore.refreshIndexForTesting(timeseriesDataset.ref) memStore.refreshIndexForTesting(MMD.dataset1.ref) diff --git a/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala b/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala index dce7ebbb61..f99a21dbce 100644 --- a/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala @@ -86,7 +86,7 @@ class MetadataExecSpec extends AnyFunSpec with Matchers with ScalaFutures with B tuples.map { t => SeqRowReader(Seq(t._1, t._2, metric, partTagsUTF8)) } .foreach(builder.addFromReader(_, Schemas.promCounter)) } - memStore.setup(timeseriesDatasetMultipleShardKeys.ref, Schemas(Schemas.promCounter), ishard, TestData.storeConf) + memStore.setup(timeseriesDatasetMultipleShardKeys.ref, Schemas(Schemas.promCounter), ishard, TestData.storeConf, 1) memStore.ingest(timeseriesDatasetMultipleShardKeys.ref, ishard, SomeData(builder.allContainers.head, 0)) builder.reset() diff --git a/query/src/test/scala/filodb/query/exec/MultiSchemaPartitionsExecSpec.scala b/query/src/test/scala/filodb/query/exec/MultiSchemaPartitionsExecSpec.scala index 7e9ba0d567..3b9e461acb 100644 --- a/query/src/test/scala/filodb/query/exec/MultiSchemaPartitionsExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/MultiSchemaPartitionsExecSpec.scala @@ -88,16 +88,16 @@ class MultiSchemaPartitionsExecSpec extends AnyFunSpec with Matchers with ScalaF implicit val execTimeout = 5.seconds override def beforeAll(): Unit = { - memStore.setup(dsRef, schemas, 0, TestData.storeConf) + memStore.setup(dsRef, schemas, 0, TestData.storeConf, 2) memStore.ingest(dsRef, 0, SomeData(container, 0)) memStore.ingest(dsRef, 0, MMD.records(MMD.histDataset, histData)) // set up shard, but do not ingest data to simulate an empty shard - memStore.setup(dsRef, schemas, 1, TestData.storeConf) + memStore.setup(dsRef, schemas, 1, TestData.storeConf, 2) - memStore.setup(MMD.dataset1.ref, Schemas(MMD.schema1), 0, TestData.storeConf) + memStore.setup(MMD.dataset1.ref, Schemas(MMD.schema1), 0, TestData.storeConf, 1) memStore.ingest(MMD.dataset1.ref, 0, mmdSomeData) - memStore.setup(MMD.histMaxDS.ref, Schemas(MMD.histMaxDS.schema), 0, TestData.storeConf) + memStore.setup(MMD.histMaxDS.ref, Schemas(MMD.histMaxDS.schema), 0, TestData.storeConf, 1) memStore.ingest(MMD.histMaxDS.ref, 0, MMD.records(MMD.histMaxDS, histMaxData)) memStore.refreshIndexForTesting(dsRef) diff --git a/query/src/test/scala/filodb/query/exec/RemoteMetadataExecSpec.scala b/query/src/test/scala/filodb/query/exec/RemoteMetadataExecSpec.scala index d6df796433..b4a4279529 100644 --- a/query/src/test/scala/filodb/query/exec/RemoteMetadataExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/RemoteMetadataExecSpec.scala @@ -85,7 +85,7 @@ class RemoteMetadataExecSpec extends AnyFunSpec with Matchers with ScalaFutures tuples.map { t => SeqRowReader(Seq(t._1, t._2, metric, partTagsUTF8)) } .foreach(builder.addFromReader(_, Schemas.promCounter)) } - memStore.setup(timeseriesDatasetMultipleShardKeys.ref, Schemas(Schemas.promCounter), ishard, TestData.storeConf) + memStore.setup(timeseriesDatasetMultipleShardKeys.ref, Schemas(Schemas.promCounter), ishard, TestData.storeConf, 1) memStore.ingest(timeseriesDatasetMultipleShardKeys.ref, ishard, SomeData(builder.allContainers.head, 0)) } diff --git a/query/src/test/scala/filodb/query/exec/SplitLocalPartitionDistConcatExecSpec.scala b/query/src/test/scala/filodb/query/exec/SplitLocalPartitionDistConcatExecSpec.scala index 652bbc43ed..aab1bc1697 100644 --- a/query/src/test/scala/filodb/query/exec/SplitLocalPartitionDistConcatExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/SplitLocalPartitionDistConcatExecSpec.scala @@ -86,10 +86,10 @@ class SplitLocalPartitionDistConcatExecSpec extends AnyFunSpec with Matchers wit implicit val execTimeout = 5.seconds override def beforeAll(): Unit = { - memStore.setup(dsRef, schemas, 0, TestData.storeConf) + memStore.setup(dsRef, schemas, 0, TestData.storeConf, 1) memStore.ingest(dsRef, 0, SomeData(container, 0)) - memStore.setup(MMD.dataset1.ref, Schemas(MMD.schema1), 0, TestData.storeConf) + memStore.setup(MMD.dataset1.ref, Schemas(MMD.schema1), 0, TestData.storeConf, 1) memStore.ingest(MMD.dataset1.ref, 0, mmdSomeData) memStore.refreshIndexForTesting(dsRef) diff --git a/spark-jobs/src/test/scala/filodb/downsampler/DownsamplerMainSpec.scala b/spark-jobs/src/test/scala/filodb/downsampler/DownsamplerMainSpec.scala index d66d0eec41..6fa2640d82 100644 --- a/spark-jobs/src/test/scala/filodb/downsampler/DownsamplerMainSpec.scala +++ b/spark-jobs/src/test/scala/filodb/downsampler/DownsamplerMainSpec.scala @@ -1412,7 +1412,7 @@ class DownsamplerMainSpec extends AnyFunSpec with Matchers with BeforeAndAfterAl settings.filodbConfig) downsampleTSStore.setup(batchDownsampler.rawDatasetRef, settings.filodbSettings.schemas, - 0, rawDataStoreConfig, settings.rawDatasetIngestionConfig.downsampleConfig) + 0, rawDataStoreConfig, 1, settings.rawDatasetIngestionConfig.downsampleConfig) downsampleTSStore.recoverIndex(batchDownsampler.rawDatasetRef, 0).futureValue @@ -1459,7 +1459,7 @@ class DownsamplerMainSpec extends AnyFunSpec with Matchers with BeforeAndAfterAl ) downsampleTSStore.setup(batchDownsampler.rawDatasetRef, settings.filodbSettings.schemas, - 0, rawDataStoreConfig, durableIndexSettings.rawDatasetIngestionConfig.downsampleConfig) + 0, rawDataStoreConfig, 1, durableIndexSettings.rawDatasetIngestionConfig.downsampleConfig) val recoveredRecords = downsampleTSStore.recoverIndex(batchDownsampler.rawDatasetRef, 0).futureValue recoveredRecords shouldBe 5 @@ -1474,7 +1474,7 @@ class DownsamplerMainSpec extends AnyFunSpec with Matchers with BeforeAndAfterAl ) downsampleTSStore2.setup(batchDownsampler.rawDatasetRef, settings.filodbSettings.schemas, - 0, rawDataStoreConfig, durableIndexSettings.rawDatasetIngestionConfig.downsampleConfig) + 0, rawDataStoreConfig, 1, durableIndexSettings.rawDatasetIngestionConfig.downsampleConfig) val recoveredRecords2 = downsampleTSStore2.recoverIndex(batchDownsampler.rawDatasetRef, 0).futureValue recoveredRecords2 shouldBe 0 @@ -1509,7 +1509,7 @@ class DownsamplerMainSpec extends AnyFunSpec with Matchers with BeforeAndAfterAl settings.filodbConfig) downsampleTSStore.setup(batchDownsampler.rawDatasetRef, settings.filodbSettings.schemas, - 0, rawDataStoreConfig, settings.rawDatasetIngestionConfig.downsampleConfig) + 0, rawDataStoreConfig, 1, settings.rawDatasetIngestionConfig.downsampleConfig) downsampleTSStore.recoverIndex(batchDownsampler.rawDatasetRef, 0).futureValue @@ -1540,7 +1540,7 @@ class DownsamplerMainSpec extends AnyFunSpec with Matchers with BeforeAndAfterAl settings.filodbConfig) downsampleTSStore.setup(batchDownsampler.rawDatasetRef, settings.filodbSettings.schemas, - 0, rawDataStoreConfig, settings.rawDatasetIngestionConfig.downsampleConfig) + 0, rawDataStoreConfig, 1, settings.rawDatasetIngestionConfig.downsampleConfig) downsampleTSStore.recoverIndex(batchDownsampler.rawDatasetRef, 0).futureValue @@ -1572,7 +1572,7 @@ class DownsamplerMainSpec extends AnyFunSpec with Matchers with BeforeAndAfterAl settings.filodbConfig) downsampleTSStore.setup(batchDownsampler.rawDatasetRef, settings.filodbSettings.schemas, - 0, rawDataStoreConfig, settings.rawDatasetIngestionConfig.downsampleConfig) + 0, rawDataStoreConfig, 1, settings.rawDatasetIngestionConfig.downsampleConfig) downsampleTSStore.recoverIndex(batchDownsampler.rawDatasetRef, 0).futureValue @@ -1602,7 +1602,7 @@ class DownsamplerMainSpec extends AnyFunSpec with Matchers with BeforeAndAfterAl val downsampleTSStore = new DownsampledTimeSeriesStore(downsampleColStore, rawColStore, settings.filodbConfig) downsampleTSStore.setup(batchDownsampler.rawDatasetRef, settings.filodbSettings.schemas, - 0, rawDataStoreConfig, settings.rawDatasetIngestionConfig.downsampleConfig) + 0, rawDataStoreConfig, 1, settings.rawDatasetIngestionConfig.downsampleConfig) downsampleTSStore.recoverIndex(batchDownsampler.rawDatasetRef, 0).futureValue val colFilters = seriesTags.map { case (t, v) => ColumnFilter(t.toString, Equals(v.toString)) }.toSeq val queryFilters = colFilters :+ ColumnFilter("_metric_", Equals(gaugeName)) From 90303aaff71dd435091148871861bb84d719c9ae Mon Sep 17 00:00:00 2001 From: yu-shipit <120689245+yu-shipit@users.noreply.github.com> Date: Thu, 13 Jul 2023 10:35:47 -0700 Subject: [PATCH 03/39] filodb(core) add debugging info for empty histogram. (#1613) * filodb(core) add debugging info for empty histogram. Some queries occasionally hit exceptions because of empty histogram. However, the same exception could not be reproduced later. The hunch is that the bug is caused by a race condition. So, adding additional debugging log to print out the chunk id chunk info and the memory dump. --------- Co-authored-by: Yu Zhang Co-authored-by: alextheimer Co-authored-by: sandeep6189 --- .../query/PartitionTimeRangeReader.scala | 22 +++++++++++++++++-- .../format/vectors/HistogramVector.scala | 5 +++++ 2 files changed, 25 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/filodb.core/query/PartitionTimeRangeReader.scala b/core/src/main/scala/filodb.core/query/PartitionTimeRangeReader.scala index e4b336c5da..a86ad2a56a 100644 --- a/core/src/main/scala/filodb.core/query/PartitionTimeRangeReader.scala +++ b/core/src/main/scala/filodb.core/query/PartitionTimeRangeReader.scala @@ -1,10 +1,12 @@ package filodb.core.query +import com.typesafe.scalalogging.StrictLogging import spire.syntax.cfor._ import filodb.core.metadata.Dataset import filodb.core.store.{ChunkInfoIterator, ChunkSetInfoReader, ReadablePartition} import filodb.memory.format.{vectors => bv, RowReader, TypedIterator, UnsafeUtils, ZeroCopyUTF8String} +import filodb.memory.format.vectors.EmptyHistogramException /** * A RowReader iterator which iterates over a time range in the ReadablePartition. Designed to be relatively memory @@ -16,7 +18,7 @@ final class PartitionTimeRangeReader(part: ReadablePartition, startTime: Long, endTime: Long, infos: ChunkInfoIterator, - columnIDs: Array[Int]) extends RangeVectorCursor { + columnIDs: Array[Int]) extends RangeVectorCursor with StrictLogging { // MinValue = no current chunk private var curChunkID = Long.MinValue private final val vectorIts = new Array[TypedIterator](columnIDs.size) @@ -33,7 +35,23 @@ final class PartitionTimeRangeReader(part: ReadablePartition, def getDouble(columnNo: Int): Double = vectorIts(columnNo).asDoubleIt.next def getFloat(columnNo: Int): Float = ??? def getString(columnNo: Int): String = ??? - override def getHistogram(columnNo: Int): bv.Histogram = vectorIts(columnNo).asHistIt.next + override def getHistogram(columnNo: Int): bv.Histogram = { + try { + vectorIts(columnNo).asHistIt.next + } catch { + case e : EmptyHistogramException => { + var message = s"EmptyHistogramException ${e.getMessage} infos=[" + while (infos.hasNext) { + val info = infos.nextInfo + message += + s"""${info.debugString} """ + } + message += "]" + logger.error(s"message ${message}") + throw new IllegalArgumentException(message) + } + } + } def getAny(columnNo: Int): Any = ??? override def filoUTF8String(columnNo: Int): ZeroCopyUTF8String = vectorIts(columnNo).asUTF8It.next 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 430117cd5f..2a6f230e86 100644 --- a/memory/src/main/scala/filodb.memory/format/vectors/HistogramVector.scala +++ b/memory/src/main/scala/filodb.memory/format/vectors/HistogramVector.scala @@ -556,6 +556,7 @@ final case class HistogramCorrection(lastValue: LongHistogram, correction: LongH trait CounterHistogramReader extends HistogramReader with CounterVectorReader { def correctedValue(n: Int, meta: CorrectionMeta): HistogramWithBuckets } +case class EmptyHistogramException(message: String) extends IllegalArgumentException(message) /** * A reader for SectDelta encoded histograms, including correction/drop functionality @@ -578,6 +579,10 @@ class SectDeltaHistogramReader(acc2: MemoryReader, histVect: Ptr.U8) } override def apply(index: Int): HistogramWithBuckets = { + if (length <= 0) { + throw EmptyHistogramException(s"""length = $length memory=${toHexString(acc, histVect.addr)}""") + } + require(length > 0) val histPtr = locate(index) From 5b05779aa36475bb5c91c7a71ae092795ae79b06 Mon Sep 17 00:00:00 2001 From: yu-shipit <120689245+yu-shipit@users.noreply.github.com> Date: Thu, 13 Jul 2023 10:36:17 -0700 Subject: [PATCH 04/39] fix(core) make the error message more frendly to users. (#1593) Co-authored-by: Yu Zhang --- .../main/scala/filodb.core/binaryrecord2/RecordBuilder.scala | 3 ++- query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala | 4 +++- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/filodb.core/binaryrecord2/RecordBuilder.scala b/core/src/main/scala/filodb.core/binaryrecord2/RecordBuilder.scala index 2d309ac616..3820d0058c 100644 --- a/core/src/main/scala/filodb.core/binaryrecord2/RecordBuilder.scala +++ b/core/src/main/scala/filodb.core/binaryrecord2/RecordBuilder.scala @@ -540,7 +540,8 @@ class RecordBuilder(memFactory: MemFactory, val oldOffset = curRecordOffset if (reuseOneContainer) resetContainerPointers() else newContainer() logger.trace(s"Moving $recordNumBytes bytes from end of old container to new container") - require((containerSize - ContainerHeaderLen) > (recordNumBytes + numBytes), "Record too big for container") + require((containerSize - ContainerHeaderLen) > (recordNumBytes + numBytes), + "The intermediate or final result is too big. For queries, please try to add more query filters or time range.") unsafe.copyMemory(oldBase, oldOffset, curBase, curRecordOffset, recordNumBytes) if (mapOffset != -1L) mapOffset = curRecordOffset + (mapOffset - oldOffset) curRecEndOffset = curRecordOffset + recordNumBytes diff --git a/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala b/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala index f99a21dbce..e50b8ef4cc 100644 --- a/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala @@ -243,7 +243,9 @@ class MetadataExecSpec extends AnyFunSpec with Matchers with ScalaFutures with B val resp = execPlan.execute(memStore, querySessionOverridden).runToFuture.futureValue resp match { case QueryError(_, _, ex: IllegalArgumentException) => - ex.getMessage shouldEqual "requirement failed: Record too big for container" + ex.getMessage shouldEqual "requirement failed:" + + " The intermediate or final result is too big. For queries, please try to" + + " add more query filters or time range." case _ => fail(s"Expected to see an exception for exceeding the default " + s"container limit of ${execPlan.maxRecordContainerSize(queryConfig)}") From a93666d8d9d8e277f39b2fab081a7459a91ff82a Mon Sep 17 00:00:00 2001 From: yu-shipit <120689245+yu-shipit@users.noreply.github.com> Date: Thu, 13 Jul 2023 11:02:40 -0700 Subject: [PATCH 05/39] Revert "filodb(core) add debugging info for empty histogram. (#1613)" (#1623) This reverts commit 90303aaff71dd435091148871861bb84d719c9ae. --- .../query/PartitionTimeRangeReader.scala | 22 ++----------------- .../format/vectors/HistogramVector.scala | 5 ----- 2 files changed, 2 insertions(+), 25 deletions(-) diff --git a/core/src/main/scala/filodb.core/query/PartitionTimeRangeReader.scala b/core/src/main/scala/filodb.core/query/PartitionTimeRangeReader.scala index a86ad2a56a..e4b336c5da 100644 --- a/core/src/main/scala/filodb.core/query/PartitionTimeRangeReader.scala +++ b/core/src/main/scala/filodb.core/query/PartitionTimeRangeReader.scala @@ -1,12 +1,10 @@ package filodb.core.query -import com.typesafe.scalalogging.StrictLogging import spire.syntax.cfor._ import filodb.core.metadata.Dataset import filodb.core.store.{ChunkInfoIterator, ChunkSetInfoReader, ReadablePartition} import filodb.memory.format.{vectors => bv, RowReader, TypedIterator, UnsafeUtils, ZeroCopyUTF8String} -import filodb.memory.format.vectors.EmptyHistogramException /** * A RowReader iterator which iterates over a time range in the ReadablePartition. Designed to be relatively memory @@ -18,7 +16,7 @@ final class PartitionTimeRangeReader(part: ReadablePartition, startTime: Long, endTime: Long, infos: ChunkInfoIterator, - columnIDs: Array[Int]) extends RangeVectorCursor with StrictLogging { + columnIDs: Array[Int]) extends RangeVectorCursor { // MinValue = no current chunk private var curChunkID = Long.MinValue private final val vectorIts = new Array[TypedIterator](columnIDs.size) @@ -35,23 +33,7 @@ final class PartitionTimeRangeReader(part: ReadablePartition, def getDouble(columnNo: Int): Double = vectorIts(columnNo).asDoubleIt.next def getFloat(columnNo: Int): Float = ??? def getString(columnNo: Int): String = ??? - override def getHistogram(columnNo: Int): bv.Histogram = { - try { - vectorIts(columnNo).asHistIt.next - } catch { - case e : EmptyHistogramException => { - var message = s"EmptyHistogramException ${e.getMessage} infos=[" - while (infos.hasNext) { - val info = infos.nextInfo - message += - s"""${info.debugString} """ - } - message += "]" - logger.error(s"message ${message}") - throw new IllegalArgumentException(message) - } - } - } + override def getHistogram(columnNo: Int): bv.Histogram = vectorIts(columnNo).asHistIt.next def getAny(columnNo: Int): Any = ??? override def filoUTF8String(columnNo: Int): ZeroCopyUTF8String = vectorIts(columnNo).asUTF8It.next 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 2a6f230e86..430117cd5f 100644 --- a/memory/src/main/scala/filodb.memory/format/vectors/HistogramVector.scala +++ b/memory/src/main/scala/filodb.memory/format/vectors/HistogramVector.scala @@ -556,7 +556,6 @@ final case class HistogramCorrection(lastValue: LongHistogram, correction: LongH trait CounterHistogramReader extends HistogramReader with CounterVectorReader { def correctedValue(n: Int, meta: CorrectionMeta): HistogramWithBuckets } -case class EmptyHistogramException(message: String) extends IllegalArgumentException(message) /** * A reader for SectDelta encoded histograms, including correction/drop functionality @@ -579,10 +578,6 @@ class SectDeltaHistogramReader(acc2: MemoryReader, histVect: Ptr.U8) } override def apply(index: Int): HistogramWithBuckets = { - if (length <= 0) { - throw EmptyHistogramException(s"""length = $length memory=${toHexString(acc, histVect.addr)}""") - } - require(length > 0) val histPtr = locate(index) From a37bf5fbea901713e6394a27f3c324037963bfa4 Mon Sep 17 00:00:00 2001 From: kvpetrov Date: Fri, 14 Jul 2023 09:48:32 -0700 Subject: [PATCH 06/39] Adding logging statement when warning is produced. (#1625) Co-authored-by: Kier Petrov --- .../scala/filodb/query/exec/MultiSchemaPartitionsExec.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/query/src/main/scala/filodb/query/exec/MultiSchemaPartitionsExec.scala b/query/src/main/scala/filodb/query/exec/MultiSchemaPartitionsExec.scala index c9bde9963a..b4b577aab4 100644 --- a/query/src/main/scala/filodb/query/exec/MultiSchemaPartitionsExec.scala +++ b/query/src/main/scala/filodb/query/exec/MultiSchemaPartitionsExec.scala @@ -150,6 +150,10 @@ final case class MultiSchemaPartitionsExec(queryContext: QueryContext, finalPlan.lookupRes.foreach(plr => if (plr.dataBytesScannedCtr.get() > queryContext.plannerParams.warnLimits.rawScannedBytes) { queryWarnings.updateRawScannedBytes(plr.dataBytesScannedCtr.get()) + val msg = + s"Query scanned ${plr.dataBytesScannedCtr.get()} bytes, which exceeds a max warn limit of " + + s"${queryContext.plannerParams.warnLimits.rawScannedBytes} bytes allowed to be scanned per shard. " + qLogger.info(queryContext.getQueryLogLine(msg)) } ) } From 8ecf6309a00e170a91e3ee62dac429d99d923f75 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Mon, 17 Jul 2023 13:24:18 -0700 Subject: [PATCH 07/39] perf(query): Remove boxed Double allocations from NaN checks during data scans (#1628) Double.isNan involves conversion to boxed java Double Local heap profiling showed that this is a significant allocation. Conversion to the static java.lang.Double.isNan removes these. --- .../src/main/scala/filodb.core/query/RangeVector.scala | 2 +- .../filodb.memory/format/vectors/DoubleVector.scala | 10 +++++----- .../scala/filodb.memory/format/vectors/Histogram.scala | 4 ++-- query/src/main/scala/filodb/query/exec/ExecPlan.scala | 5 +++-- .../scala/filodb/query/exec/PromQlRemoteExec.scala | 6 +++--- .../query/exec/aggregator/SumRowAggregator.scala | 4 ++-- 6 files changed, 16 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/filodb.core/query/RangeVector.scala b/core/src/main/scala/filodb.core/query/RangeVector.scala index bba7fdd240..7e456f1aa8 100644 --- a/core/src/main/scala/filodb.core/query/RangeVector.scala +++ b/core/src/main/scala/filodb.core/query/RangeVector.scala @@ -462,7 +462,7 @@ object SerializedRangeVector extends StrictLogging { val nextRow = rows.next() // Don't encode empty / NaN data over the wire if (!canRemoveEmptyRows(rv.outputRange, schema) || - schema.columns(1).colType == DoubleColumn && !nextRow.getDouble(1).isNaN || + schema.columns(1).colType == DoubleColumn && !java.lang.Double.isNaN(nextRow.getDouble(1)) || schema.columns(1).colType == HistogramColumn && !nextRow.getHistogram(1).isEmpty) { numRows += 1 builder.addFromReader(nextRow, schema, 0) 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 e5db5d6e4a..f749ee236f 100644 --- a/memory/src/main/scala/filodb.memory/format/vectors/DoubleVector.scala +++ b/memory/src/main/scala/filodb.memory/format/vectors/DoubleVector.scala @@ -242,7 +242,7 @@ object DoubleVectorDataReader64 extends DoubleVectorDataReader { val nextDbl = acc.getDouble(addr) // There are many possible values of NaN. Use a function to ignore them reliably. if (!java.lang.Double.isNaN(nextDbl)) { - if (sum.isNaN) sum = 0d + if (java.lang.Double.isNaN(sum)) sum = 0d sum += nextDbl } addr += 8 @@ -323,7 +323,7 @@ extends DoubleVectorDataReader { var last = Double.MinValue cforRange { 0 until len } { pos => var nextVal = it.next - if (nextVal.isNaN) nextVal = 0 // explicit counter reset due to end of time series marker + if (java.lang.Double.isNaN(nextVal)) nextVal = 0 // explicit counter reset due to end of time series marker if (nextVal < last) { // reset! _correction += last _drops += pos @@ -443,9 +443,9 @@ class DoubleCounterAppender(addr: BinaryRegion.NativePointer, maxBytes: Int, dis extends DoubleAppendingVector(addr, maxBytes, dispose) { private var last = Double.MinValue override final def addData(data: Double): AddResponse = { - if (data.isNaN || data < last) + if (java.lang.Double.isNaN(data) || data < last) PrimitiveVectorReader.markDrop(MemoryAccessor.nativePtrAccessor, addr) - if (!data.isNaN) + if (!java.lang.Double.isNaN(data)) last = data super.addData(data) } @@ -544,7 +544,7 @@ object DoubleLongWrapDataReader extends DoubleVectorDataReader { final def changes(acc: MemoryReader, vector: BinaryVectorPtr, start: Int, end: Int, prev: Double, ignorePrev: Boolean = false): (Double, Double) = { - val ignorePrev = if (prev.isNaN) true + val ignorePrev = if (java.lang.Double.isNaN(prev)) true else false val changes = LongBinaryVector(acc, vector).changes(acc, vector, start, end, prev.toLong, ignorePrev) (changes._1.toDouble, changes._2.toDouble) 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 4ebb711d6c..3e8bd77a8b 100644 --- a/memory/src/main/scala/filodb.memory/format/vectors/Histogram.scala +++ b/memory/src/main/scala/filodb.memory/format/vectors/Histogram.scala @@ -228,7 +228,7 @@ final case class MutableHistogram(buckets: HistogramBuckets, values: Array[Doubl // Allow addition when type of bucket is different if (buckets.similarForMath(other.buckets)) { // If it was NaN before, reset to 0 to sum another hist - if (values(0).isNaN) java.util.Arrays.fill(values, 0.0) + if (java.lang.Double.isNaN(values(0))) java.util.Arrays.fill(values, 0.0) cforRange { 0 until numBuckets } { b => values(b) += other.bucketValue(b) } @@ -272,7 +272,7 @@ final case class MutableHistogram(buckets: HistogramBuckets, values: Array[Doubl cforRange { 0 until values.size } { b => // When bucket no longer used NaN will be seen. Non-increasing values can be seen when // newer buckets are introduced and not all instances are updated with that bucket. - if (values(b) < max || values(b).isNaN) values(b) = max // assign previous max + if (values(b) < max || java.lang.Double.isNaN(values(b))) values(b) = max // assign previous max else if (values(b) > max) max = values(b) // update max } } diff --git a/query/src/main/scala/filodb/query/exec/ExecPlan.scala b/query/src/main/scala/filodb/query/exec/ExecPlan.scala index d1b889a6f6..3655f0d153 100644 --- a/query/src/main/scala/filodb/query/exec/ExecPlan.scala +++ b/query/src/main/scala/filodb/query/exec/ExecPlan.scala @@ -439,7 +439,6 @@ trait ExecPlan extends QueryCommand { } } - def makeResult( rv : Observable[RangeVector], recordSchema: RecordSchema, resultSchema: ResultSchema ): Task[QueryResult] = { @@ -530,7 +529,9 @@ trait ExecPlan extends QueryCommand { } protected def queryWithPlanName(queryContext: QueryContext): String = { - s"${this.getClass.getSimpleName}-${queryContext.origQueryParams}" + // Disabling this since it showed up in local method profiles. Re-enable if needed for debugging + // s"${this.getClass.getSimpleName}-${queryContext.origQueryParams}" + s"${queryContext.queryId}:$planId" } def curNodeText(level: Int): String = diff --git a/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala b/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala index f6a3cb264d..9412813801 100644 --- a/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala +++ b/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala @@ -153,7 +153,7 @@ case class PromQlRemoteExec(queryEndpoint: String, } // dont add this size to queryStats since it was already added by callee use dummy QueryStats() SerializedRangeVector(rv, builder, recordSchema.get("default").get, - queryWithPlanName(queryContext), dummyQueryStats) + planId, dummyQueryStats) // TODO: Handle stitching with verbose flag } QueryResult( @@ -235,7 +235,7 @@ case class PromQlRemoteExec(queryEndpoint: String, } // dont add this size to queryStats since it was already added by callee use dummy QueryStats() SerializedRangeVector(rv, builder, recordSchema.get(Avg.entryName).get, - queryWithPlanName(queryContext), dummyQueryStats) + planId, dummyQueryStats) } // TODO: Handle stitching with verbose flag @@ -273,7 +273,7 @@ case class PromQlRemoteExec(queryEndpoint: String, } // dont add this size to queryStats since it was already added by callee use dummy QueryStats() SerializedRangeVector(rv, builder, recordSchema.get(QueryFunctionConstants.stdVal).get, - queryWithPlanName(queryContext), dummyQueryStats) + planId, dummyQueryStats) } // TODO: Handle stitching with verbose flag diff --git a/query/src/main/scala/filodb/query/exec/aggregator/SumRowAggregator.scala b/query/src/main/scala/filodb/query/exec/aggregator/SumRowAggregator.scala index 638d01e949..a482b69664 100644 --- a/query/src/main/scala/filodb/query/exec/aggregator/SumRowAggregator.scala +++ b/query/src/main/scala/filodb/query/exec/aggregator/SumRowAggregator.scala @@ -21,8 +21,8 @@ object SumRowAggregator extends RowAggregator { def map(rvk: RangeVectorKey, item: RowReader, mapInto: MutableRowReader): RowReader = item def reduceAggregate(acc: SumHolder, aggRes: RowReader): SumHolder = { acc.timestamp = aggRes.getLong(0) - if (!aggRes.getDouble(1).isNaN) { - if (acc.sum.isNaN) acc.sum = 0 + if (!java.lang.Double.isNaN(aggRes.getDouble(1))) { + if (java.lang.Double.isNaN(acc.sum)) acc.sum = 0 acc.sum += aggRes.getDouble(1) } acc From f14a13cedaa9dbf62bc800cbfa71b019585f23df Mon Sep 17 00:00:00 2001 From: yu-shipit <120689245+yu-shipit@users.noreply.github.com> Date: Mon, 17 Jul 2023 16:06:38 -0700 Subject: [PATCH 08/39] fix(core) make the error message more frendly to users. (#1593) (#1630) Co-authored-by: Yu Zhang (cherry picked from commit 5b05779aa36475bb5c91c7a71ae092795ae79b06) --- .../main/scala/filodb.core/binaryrecord2/RecordBuilder.scala | 3 ++- query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala | 4 +++- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/filodb.core/binaryrecord2/RecordBuilder.scala b/core/src/main/scala/filodb.core/binaryrecord2/RecordBuilder.scala index 2d309ac616..3820d0058c 100644 --- a/core/src/main/scala/filodb.core/binaryrecord2/RecordBuilder.scala +++ b/core/src/main/scala/filodb.core/binaryrecord2/RecordBuilder.scala @@ -540,7 +540,8 @@ class RecordBuilder(memFactory: MemFactory, val oldOffset = curRecordOffset if (reuseOneContainer) resetContainerPointers() else newContainer() logger.trace(s"Moving $recordNumBytes bytes from end of old container to new container") - require((containerSize - ContainerHeaderLen) > (recordNumBytes + numBytes), "Record too big for container") + require((containerSize - ContainerHeaderLen) > (recordNumBytes + numBytes), + "The intermediate or final result is too big. For queries, please try to add more query filters or time range.") unsafe.copyMemory(oldBase, oldOffset, curBase, curRecordOffset, recordNumBytes) if (mapOffset != -1L) mapOffset = curRecordOffset + (mapOffset - oldOffset) curRecEndOffset = curRecordOffset + recordNumBytes diff --git a/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala b/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala index dce7ebbb61..c0b84f3968 100644 --- a/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala @@ -243,7 +243,9 @@ class MetadataExecSpec extends AnyFunSpec with Matchers with ScalaFutures with B val resp = execPlan.execute(memStore, querySessionOverridden).runToFuture.futureValue resp match { case QueryError(_, _, ex: IllegalArgumentException) => - ex.getMessage shouldEqual "requirement failed: Record too big for container" + ex.getMessage shouldEqual "requirement failed:" + + " The intermediate or final result is too big. For queries, please try to" + + " add more query filters or time range." case _ => fail(s"Expected to see an exception for exceeding the default " + s"container limit of ${execPlan.maxRecordContainerSize(queryConfig)}") From 6ac025558a1f8ae7e879227434b9e5014d28b3a8 Mon Sep 17 00:00:00 2001 From: yu-shipit <120689245+yu-shipit@users.noreply.github.com> Date: Tue, 18 Jul 2023 11:22:09 -0700 Subject: [PATCH 09/39] fix nullpointer happened in cardinality busting job. (#1631) skip busting the problematic entry. Co-authored-by: Yu Zhang --- .../PerShardCardinalityBuster.scala | 32 +++++++++++-------- 1 file changed, 19 insertions(+), 13 deletions(-) diff --git a/spark-jobs/src/main/scala/filodb/cardbuster/PerShardCardinalityBuster.scala b/spark-jobs/src/main/scala/filodb/cardbuster/PerShardCardinalityBuster.scala index f61c8ba822..2a9733350a 100644 --- a/spark-jobs/src/main/scala/filodb/cardbuster/PerShardCardinalityBuster.scala +++ b/spark-jobs/src/main/scala/filodb/cardbuster/PerShardCardinalityBuster.scala @@ -83,22 +83,28 @@ class PerShardCardinalityBuster(dsSettings: DownsamplerSettings, val numCouldNotDelete = AtomicInt(0) val numCandidateKeys = AtomicInt(0) val keysToDelete = candidateKeys.filter { pk => - val rawSchemaId = RecordSchema.schemaID(pk.partKey, UnsafeUtils.arayOffset) - val schema = schemas(rawSchemaId) - val pkPairs = schema.partKeySchema.toStringPairs(pk.partKey, UnsafeUtils.arayOffset) - val willDelete = deleteFilter.exists { filter => // at least one filter should match - filter.forall { case (filterKey, filterValRegex) => // should match all tags in this filter - pkPairs.exists { case (pkKey, pkVal) => - pkKey == filterKey && filterValRegex.matcher(pkVal).matches + try { + val rawSchemaId = RecordSchema.schemaID(pk.partKey, UnsafeUtils.arayOffset) + val schema = schemas(rawSchemaId) + val pkPairs = schema.partKeySchema.toStringPairs(pk.partKey, UnsafeUtils.arayOffset) + val willDelete = deleteFilter.exists { filter => // at least one filter should match + filter.forall { case (filterKey, filterValRegex) => // should match all tags in this filter + pkPairs.exists { case (pkKey, pkVal) => + pkKey == filterKey && filterValRegex.matcher(pkVal).matches + } } } + if (willDelete) { + BusterContext.log.debug(s"Deleting part key $pkPairs from shard=$shard startTime=${pk.startTime} " + + s"endTime=${pk.endTime} split=$split isSimulation=$isSimulation") + } + numCandidateKeys += 1 + willDelete + } catch { + case e : Exception => + BusterContext.log.warn(s"skip busting pk=$pk because of exception $e") + false } - if (willDelete) { - BusterContext.log.debug(s"Deleting part key $pkPairs from shard=$shard startTime=${pk.startTime} " + - s"endTime=${pk.endTime} split=$split isSimulation=$isSimulation") - } - numCandidateKeys += 1 - willDelete } val fut = Observable.fromIteratorUnsafe(keysToDelete) .mapParallelUnordered(numParallelDeletesPerSparkThread.getOrElse(1)) { pk => From b9ea6809f3c2eaa3ab30656626301ced4b2b55b5 Mon Sep 17 00:00:00 2001 From: yu-shipit <120689245+yu-shipit@users.noreply.github.com> Date: Tue, 18 Jul 2023 11:22:52 -0700 Subject: [PATCH 10/39] filodb(core) add debugging info for empty histogram. (#1624) Some queries occasionally hit exceptions because of empty histogram. However, the same exception could not be reproduced later. The hunch is that the bug is caused by a race condition. So, adding additional debugging log to print out the chunk id chunk info and the memory dump. Co-authored-by: Yu Zhang --- .../query/PartitionTimeRangeReader.scala | 22 +++++++++++++++++-- .../format/vectors/HistogramVector.scala | 6 ++++- 2 files changed, 25 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/filodb.core/query/PartitionTimeRangeReader.scala b/core/src/main/scala/filodb.core/query/PartitionTimeRangeReader.scala index e4b336c5da..e48711b368 100644 --- a/core/src/main/scala/filodb.core/query/PartitionTimeRangeReader.scala +++ b/core/src/main/scala/filodb.core/query/PartitionTimeRangeReader.scala @@ -1,10 +1,12 @@ package filodb.core.query +import com.typesafe.scalalogging.StrictLogging import spire.syntax.cfor._ import filodb.core.metadata.Dataset import filodb.core.store.{ChunkInfoIterator, ChunkSetInfoReader, ReadablePartition} import filodb.memory.format.{vectors => bv, RowReader, TypedIterator, UnsafeUtils, ZeroCopyUTF8String} +import filodb.memory.format.vectors.EmptyHistogramException /** * A RowReader iterator which iterates over a time range in the ReadablePartition. Designed to be relatively memory @@ -16,7 +18,7 @@ final class PartitionTimeRangeReader(part: ReadablePartition, startTime: Long, endTime: Long, infos: ChunkInfoIterator, - columnIDs: Array[Int]) extends RangeVectorCursor { + columnIDs: Array[Int]) extends RangeVectorCursor with StrictLogging { // MinValue = no current chunk private var curChunkID = Long.MinValue private final val vectorIts = new Array[TypedIterator](columnIDs.size) @@ -33,7 +35,23 @@ final class PartitionTimeRangeReader(part: ReadablePartition, def getDouble(columnNo: Int): Double = vectorIts(columnNo).asDoubleIt.next def getFloat(columnNo: Int): Float = ??? def getString(columnNo: Int): String = ??? - override def getHistogram(columnNo: Int): bv.Histogram = vectorIts(columnNo).asHistIt.next + override def getHistogram(columnNo: Int): bv.Histogram = { + try { + vectorIts(columnNo).asHistIt.next + } catch { + case e : EmptyHistogramException => { + var message = s"EmptyHistogramException ${e.getMessage} infos=[" + while (infos.hasNext) { + val info = infos.nextInfo + message += + s"""${info.debugString} """ + } + message += "]" + logger.error(s"message $message") + throw new IllegalArgumentException("requirement length > 0 failed.") + } + } + } def getAny(columnNo: Int): Any = ??? override def filoUTF8String(columnNo: Int): ZeroCopyUTF8String = vectorIts(columnNo).asUTF8It.next 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 430117cd5f..d9ca52558c 100644 --- a/memory/src/main/scala/filodb.memory/format/vectors/HistogramVector.scala +++ b/memory/src/main/scala/filodb.memory/format/vectors/HistogramVector.scala @@ -556,6 +556,7 @@ final case class HistogramCorrection(lastValue: LongHistogram, correction: LongH trait CounterHistogramReader extends HistogramReader with CounterVectorReader { def correctedValue(n: Int, meta: CorrectionMeta): HistogramWithBuckets } +case class EmptyHistogramException(message: String) extends IllegalArgumentException(message) /** * A reader for SectDelta encoded histograms, including correction/drop functionality @@ -578,7 +579,10 @@ class SectDeltaHistogramReader(acc2: MemoryReader, histVect: Ptr.U8) } override def apply(index: Int): HistogramWithBuckets = { - require(length > 0) + if (length <= 0) { + throw EmptyHistogramException(s"""length = $length memory=${toHexString(acc, histVect.addr)}""") + } + val histPtr = locate(index) // Just return the base histogram if we are at start of section From eebd5f48e6c76b536cc3a5af3c6c13de5cc15b72 Mon Sep 17 00:00:00 2001 From: alextheimer Date: Wed, 19 Jul 2023 21:35:22 -0700 Subject: [PATCH 11/39] fix(query): prevent list.head on empty list (#1632) --- .../queryplanner/LogicalPlanUtils.scala | 3 ++ .../queryplanner/PlannerHierarchySpec.scala | 30 +++++++++++++++++++ 2 files changed, 33 insertions(+) diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala index ef0fa0cfbe..fdb92fc1b8 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala @@ -423,6 +423,9 @@ object LogicalPlanUtils extends StrictLogging { LogicalPlanUtils.getTargetSchemaIfUnchanging(targetSchemaProvider, filters, interval) } } + if (rsTschemaOpts.isEmpty) { + return None + } // make sure all tschemas are defined, and they all match val referenceSchema = rsTschemaOpts.head if (referenceSchema.isDefined diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala index b7f34a851c..1c03160273 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala @@ -3366,4 +3366,34 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS validatePlan(ep, test.expected, sort = true) } } + + it("should generate correct plan for aggregations/joins without selectors") { + val queryExpectedPairs = Seq( + // FIXME: this first plan is needlessly complex (but will return the correct result) + ("""sum(vector(123)) by(foo)""", + """E~StitchRvsExec() on InProcessPlanDispatcher + |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1634172830,300,1634777330)) + |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on InProcessPlanDispatcher + |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(foo)) + |----T~VectorFunctionMapper(funcParams=List()) + |-----E~ScalarFixedDoubleExec(params = RangeParams(1634172830,300,1634777330), value = 123.0) on InProcessPlanDispatcher + |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634172530)) + |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on InProcessPlanDispatcher + |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(foo)) + |----T~VectorFunctionMapper(funcParams=List()) + |-----E~ScalarFixedDoubleExec(params = RangeParams(1633913330,300,1634172530), value = 123.0) on InProcessPlanDispatcher""".stripMargin), + ("""vector(123) + on(foo) vector(123)""", + """E~BinaryJoinExec(binaryOp=ADD, on=List(foo), ignoring=List()) on InProcessPlanDispatcher + |-T~VectorFunctionMapper(funcParams=List()) + |--E~ScalarFixedDoubleExec(params = RangeParams(1633913330,300,1634777330), value = 123.0) on InProcessPlanDispatcher + |-T~VectorFunctionMapper(funcParams=List()) + |--E~ScalarFixedDoubleExec(params = RangeParams(1633913330,300,1634777330), value = 123.0) on InProcessPlanDispatcher""".stripMargin), + ) + val timeParams = TimeStepParams(startSeconds, step, endSeconds) + for ((query, expected) <- queryExpectedPairs) { + val lp = Parser.queryRangeToLogicalPlan(query, timeParams, Antlr) + val execPlan = rootPlanner.materialize(lp, QueryContext(origQueryParams = queryParams)) + validatePlan(execPlan, expected) + } + } } From 6c1693a64c3d231b42b6c50dda584708d23bdade Mon Sep 17 00:00:00 2001 From: alextheimer Date: Mon, 24 Jul 2023 10:31:24 -0700 Subject: [PATCH 12/39] maint(kafka): update consumer client id (#1633) --- kafka/src/main/scala/filodb/kafka/KafkaIngestionStream.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/kafka/src/main/scala/filodb/kafka/KafkaIngestionStream.scala b/kafka/src/main/scala/filodb/kafka/KafkaIngestionStream.scala index 66c5bb2ae9..07f9e0b409 100644 --- a/kafka/src/main/scala/filodb/kafka/KafkaIngestionStream.scala +++ b/kafka/src/main/scala/filodb/kafka/KafkaIngestionStream.scala @@ -59,6 +59,7 @@ class KafkaIngestionStream(config: Config, if (sourceConfig.LogConfig) logger.info(s"Consumer properties: $props") blocking { + props.put("client.id", s"${props.get("group.id")}.${System.getenv("INSTANCE_ID")}.$shard") val consumer = new KafkaConsumer(props) consumer.assign(List(topicPartition).asJava) offset.foreach { off => consumer.seek(topicPartition, off) } From 5dadfb939512972bca602d7101c07b34790b3279 Mon Sep 17 00:00:00 2001 From: alextheimer Date: Wed, 19 Jul 2023 21:35:22 -0700 Subject: [PATCH 13/39] fix(query): prevent list.head on empty list (#1632) --- .../queryplanner/LogicalPlanUtils.scala | 3 ++ .../queryplanner/PlannerHierarchySpec.scala | 30 +++++++++++++++++++ 2 files changed, 33 insertions(+) diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala index ef0fa0cfbe..fdb92fc1b8 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala @@ -423,6 +423,9 @@ object LogicalPlanUtils extends StrictLogging { LogicalPlanUtils.getTargetSchemaIfUnchanging(targetSchemaProvider, filters, interval) } } + if (rsTschemaOpts.isEmpty) { + return None + } // make sure all tschemas are defined, and they all match val referenceSchema = rsTschemaOpts.head if (referenceSchema.isDefined diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala index b7f34a851c..1c03160273 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala @@ -3366,4 +3366,34 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS validatePlan(ep, test.expected, sort = true) } } + + it("should generate correct plan for aggregations/joins without selectors") { + val queryExpectedPairs = Seq( + // FIXME: this first plan is needlessly complex (but will return the correct result) + ("""sum(vector(123)) by(foo)""", + """E~StitchRvsExec() on InProcessPlanDispatcher + |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1634172830,300,1634777330)) + |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on InProcessPlanDispatcher + |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(foo)) + |----T~VectorFunctionMapper(funcParams=List()) + |-----E~ScalarFixedDoubleExec(params = RangeParams(1634172830,300,1634777330), value = 123.0) on InProcessPlanDispatcher + |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634172530)) + |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on InProcessPlanDispatcher + |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(foo)) + |----T~VectorFunctionMapper(funcParams=List()) + |-----E~ScalarFixedDoubleExec(params = RangeParams(1633913330,300,1634172530), value = 123.0) on InProcessPlanDispatcher""".stripMargin), + ("""vector(123) + on(foo) vector(123)""", + """E~BinaryJoinExec(binaryOp=ADD, on=List(foo), ignoring=List()) on InProcessPlanDispatcher + |-T~VectorFunctionMapper(funcParams=List()) + |--E~ScalarFixedDoubleExec(params = RangeParams(1633913330,300,1634777330), value = 123.0) on InProcessPlanDispatcher + |-T~VectorFunctionMapper(funcParams=List()) + |--E~ScalarFixedDoubleExec(params = RangeParams(1633913330,300,1634777330), value = 123.0) on InProcessPlanDispatcher""".stripMargin), + ) + val timeParams = TimeStepParams(startSeconds, step, endSeconds) + for ((query, expected) <- queryExpectedPairs) { + val lp = Parser.queryRangeToLogicalPlan(query, timeParams, Antlr) + val execPlan = rootPlanner.materialize(lp, QueryContext(origQueryParams = queryParams)) + validatePlan(execPlan, expected) + } + } } From 59cae2a1a373b76ba81b59d8a6ded23d54b3b9e3 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Mon, 24 Jul 2023 13:38:05 -0700 Subject: [PATCH 14/39] fix(core): Consolidate num-nodes duplicate config (#1635) There are two configs for num-nodes used by clustering-v2 and automatic memory alloc code. Consolidating them. --- conf/promperf-filodb-server.conf | 2 +- conf/timeseries-filodb-server.conf | 2 +- .../scala/filodb.coordinator/FilodbSettings.scala | 1 - .../coordinator/v2/FiloDbClusterDiscovery.scala | 11 +++++++---- .../FiloDbClusterDiscoverySpec.scala | 4 ++-- core/src/main/resources/filodb-defaults.conf | 5 +++-- 6 files changed, 14 insertions(+), 11 deletions(-) diff --git a/conf/promperf-filodb-server.conf b/conf/promperf-filodb-server.conf index 9359b8d366..15de43e558 100644 --- a/conf/promperf-filodb-server.conf +++ b/conf/promperf-filodb-server.conf @@ -5,8 +5,8 @@ filodb { "conf/promperf-source.conf" ] + min-num-nodes-in-cluster = 1 cluster-discovery { - num-nodes = 1 failure-detection-interval = 20s host-list = [ "127.0.0.1:2552" diff --git a/conf/timeseries-filodb-server.conf b/conf/timeseries-filodb-server.conf index c36293bd30..c2ce99a7fa 100644 --- a/conf/timeseries-filodb-server.conf +++ b/conf/timeseries-filodb-server.conf @@ -1,9 +1,9 @@ dataset-prometheus = { include required("timeseries-dev-source.conf") } filodb { + min-num-nodes-in-cluster = 2 v2-cluster-enabled = false cluster-discovery { - num-nodes = 2 failure-detection-interval = 20s host-list = [ "127.0.0.1:2552", diff --git a/coordinator/src/main/scala/filodb.coordinator/FilodbSettings.scala b/coordinator/src/main/scala/filodb.coordinator/FilodbSettings.scala index e762dc94ce..4b411b61ac 100755 --- a/coordinator/src/main/scala/filodb.coordinator/FilodbSettings.scala +++ b/coordinator/src/main/scala/filodb.coordinator/FilodbSettings.scala @@ -48,7 +48,6 @@ final class FilodbSettings(val conf: Config) { lazy val datasetConfPaths = config.as[Seq[String]]("dataset-configs") - lazy val numNodes = config.getInt("cluster-discovery.num-nodes") lazy val k8sHostFormat = config.as[Option[String]]("cluster-discovery.k8s-stateful-sets-hostname-format") // used for development mode only diff --git a/coordinator/src/main/scala/filodb/coordinator/v2/FiloDbClusterDiscovery.scala b/coordinator/src/main/scala/filodb/coordinator/v2/FiloDbClusterDiscovery.scala index f91968bf45..9db3f19776 100644 --- a/coordinator/src/main/scala/filodb/coordinator/v2/FiloDbClusterDiscovery.scala +++ b/coordinator/src/main/scala/filodb/coordinator/v2/FiloDbClusterDiscovery.scala @@ -40,11 +40,13 @@ class FiloDbClusterDiscovery(settings: FilodbSettings, } def shardsForOrdinal(ordinal: Int, numShards: Int): Seq[Int] = { - require(ordinal < settings.numNodes, s"Ordinal $ordinal was not expected. Number of nodes is ${settings.numNodes}") - val numShardsPerHost = numShards / settings.numNodes + require(settings.minNumNodes.isDefined, "Minimum Number of Nodes config not provided") + require(ordinal < settings.minNumNodes.get, s"Ordinal $ordinal was not expected. " + + s"Number of nodes is ${settings.minNumNodes.get}") + val numShardsPerHost = numShards / settings.minNumNodes.get // Suppose we have a total of 8 shards and 2 hosts, assuming the hostnames are host-0 and host-1, we will map // host-0 to shard [0,1,2,3] and host-1 to shard [4,5,6,7] - val numExtraShardsToAssign = numShards % settings.numNodes + val numExtraShardsToAssign = numShards % settings.minNumNodes.get val (firstShardThisNode, numShardsThisHost) = if (numExtraShardsToAssign != 0) { logger.warn("For stateful shard assignment, numShards should be a multiple of nodes per shard, " + "using default strategy") @@ -69,8 +71,9 @@ class FiloDbClusterDiscovery(settings: FilodbSettings, def shardsForLocalhost(numShards: Int): Seq[Int] = shardsForOrdinal(ordinalOfLocalhost, numShards) lazy private val hostNames = { + require(settings.minNumNodes.isDefined, "Minimum Number of Nodes config not provided") if (settings.k8sHostFormat.isDefined) { - (0 until settings.numNodes).map(i => String.format(settings.k8sHostFormat.get, i.toString)) + (0 until settings.minNumNodes.get).map(i => String.format(settings.k8sHostFormat.get, i.toString)) } else if (settings.hostList.isDefined) { settings.hostList.get.sorted // sort to make order consistent on all nodes of cluster } else throw new IllegalArgumentException("Cluster Discovery mechanism not defined") diff --git a/coordinator/src/test/scala/filodb.coordinator/FiloDbClusterDiscoverySpec.scala b/coordinator/src/test/scala/filodb.coordinator/FiloDbClusterDiscoverySpec.scala index cade043c61..2c2c729e22 100644 --- a/coordinator/src/test/scala/filodb.coordinator/FiloDbClusterDiscoverySpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/FiloDbClusterDiscoverySpec.scala @@ -11,7 +11,7 @@ class FiloDbClusterDiscoverySpec extends AkkaSpec { val config = ConfigFactory.parseString( """ - |filodb.cluster-discovery.num-nodes = 4 + |filodb.min-num-nodes-in-cluster = 4 |""".stripMargin) val settings = new FilodbSettings(config) @@ -29,7 +29,7 @@ class FiloDbClusterDiscoverySpec extends AkkaSpec { "Should allocate the extra n shards to first n nodes" in { val config = ConfigFactory.parseString( """ - |filodb.cluster-discovery.num-nodes = 5 + |filodb.min-num-nodes-in-cluster = 5 |""".stripMargin) val settings = new FilodbSettings(config) diff --git a/core/src/main/resources/filodb-defaults.conf b/core/src/main/resources/filodb-defaults.conf index a951a14769..9250f870eb 100644 --- a/core/src/main/resources/filodb-defaults.conf +++ b/core/src/main/resources/filodb-defaults.conf @@ -1,7 +1,9 @@ filodb { v2-cluster-enabled = false - # Number of nodes in cluster; used to calculate per-dhard resources based on how many shards assigned to node + # Number of nodes in cluster; used to calculate per-shard resources based + # on how many shards assigned to node. + # Required config if v2-clustering or automatic memory-alloc is enabled # min-num-nodes-in-cluster = 2 cluster-discovery { @@ -9,7 +11,6 @@ filodb { // if FiloDB HTTP API is indeed the query entry point, this should be overridden to small value // so that failure detection is quick. failure-detection-interval = 15 minutes - num-nodes = 2 # one of the two below properties should be enabled # enable this to use the k8s stateful sets mode and its hostname to extract ordinal From ea1644ba2dd6665d0cc42fcde1f782db3c9ba9d4 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Mon, 24 Jul 2023 14:56:04 -0700 Subject: [PATCH 15/39] Fix memory alloc config (#1638) Default alloc configs should sum to 100. --- core/src/main/resources/filodb-defaults.conf | 2 +- .../main/scala/filodb.core/memstore/TimeSeriesMemStore.scala | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/resources/filodb-defaults.conf b/core/src/main/resources/filodb-defaults.conf index 9250f870eb..5112af6ecc 100644 --- a/core/src/main/resources/filodb-defaults.conf +++ b/core/src/main/resources/filodb-defaults.conf @@ -722,7 +722,7 @@ filodb { # (used for storing chunks) # This is divvied amongst datasets on the node per configuration for dataset # The shards of the dataset on the node get even amount of memory from this fraction - block-memory-manager-percent = 73 + block-memory-manager-percent = 71 } # At the cost of some extra heap memory, we can track queries holding shared lock for a long time diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala index 496a026d24..3a178dce69 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala @@ -49,8 +49,8 @@ extends TimeSeriesStore with StrictLogging { val nativeMemoryManagerPercent = filodbConfig.getDouble("memstore.memory-alloc.native-memory-manager-percent") val blockMemoryManagerPercent = filodbConfig.getDouble("memstore.memory-alloc.block-memory-manager-percent") val lucenePercent = filodbConfig.getDouble("memstore.memory-alloc.lucene-memory-percent") - require(nativeMemoryManagerPercent + blockMemoryManagerPercent + lucenePercent == 100.0, - s"Configured Block($nativeMemoryManagerPercent), Native($nativeMemoryManagerPercent) " + + require(Math.abs(nativeMemoryManagerPercent + blockMemoryManagerPercent + lucenePercent - 100) < 0.001, + s"Configured Block($nativeMemoryManagerPercent), Native($blockMemoryManagerPercent) " + s"and Lucene($lucenePercent) memory percents don't sum to 100.0") (availableMemoryBytes * nativeMemoryManagerPercent / 100).toLong } else filodbConfig.getMemorySize("memstore.ingestion-buffer-mem-size").toBytes From 955814e4bf10d9f414db3ca422fcdbb80a472537 Mon Sep 17 00:00:00 2001 From: Amol Nayak Date: Fri, 28 Jul 2023 11:19:58 -0700 Subject: [PATCH 16/39] fix(query) Regex equals .* must ignore the label and match series even without the label (#1639) --- .../memstore/PartKeyLuceneIndex.scala | 8 +-- .../memstore/PartKeyLuceneIndexSpec.scala | 51 +++++++++++++++++++ 2 files changed, 56 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala b/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala index 89b5210ed2..5980821f99 100644 --- a/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala +++ b/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala @@ -817,12 +817,14 @@ class PartKeyLuceneIndex(ref: DatasetRef, logger.info(s"Refreshed index searchers to make reads consistent for dataset=$ref shard=$shardNum") } + //scalastyle:off method.length private def leafFilter(column: String, filter: Filter): Query = { filter match { case EqualsRegex(value) => val regex = removeRegexAnchors(value.toString) - if (regex.nonEmpty) new RegexpQuery(new Term(column, regex), RegExp.NONE) - else leafFilter(column, NotEqualsRegex(".+")) // value="" means the label is absent or has an empty value. + if (regex.replaceAll("\\.\\*", "").nonEmpty) new RegexpQuery(new Term(column, regex), RegExp.NONE) + else leafFilter(column, NotEqualsRegex(".+")) // value="" means the label is absent or has an empty value. + case NotEqualsRegex(value) => val term = new Term(column, removeRegexAnchors(value.toString)) val allDocs = new MatchAllDocsQuery @@ -864,7 +866,7 @@ class PartKeyLuceneIndex(ref: DatasetRef, case _ => throw new UnsupportedOperationException } } - + //scalastyle:on method.length def partIdsFromFilters(columnFilters: Seq[ColumnFilter], startTime: Long, endTime: Long): debox.Buffer[Int] = { diff --git a/core/src/test/scala/filodb.core/memstore/PartKeyLuceneIndexSpec.scala b/core/src/test/scala/filodb.core/memstore/PartKeyLuceneIndexSpec.scala index 6eb71613b2..db55753415 100644 --- a/core/src/test/scala/filodb.core/memstore/PartKeyLuceneIndexSpec.scala +++ b/core/src/test/scala/filodb.core/memstore/PartKeyLuceneIndexSpec.scala @@ -1008,4 +1008,55 @@ class PartKeyLuceneIndexSpec extends AnyFunSpec with Matchers with BeforeAndAfte // close CardinalityTracker to avoid leaking of resources cardTracker.close() } + + it("should match records without label when .* is provided on a non existent label") { + + val pkrs = partKeyFromRecords(dataset6, records(dataset6, readers.take(10)), Some(partBuilder)) + .zipWithIndex.map { case (addr, i) => + val pk = partKeyOnHeap(dataset6.schema.partKeySchema, ZeroPointer, addr) + keyIndex.addPartKey(pk, i, i, i + 10)() + PartKeyLuceneIndexRecord(pk, i, i + 10) + } + keyIndex.refreshReadersBlocking() + + + // Query with just the existing Label name + val filter1 = ColumnFilter("Actor2Code", Equals("GOV".utf8)) + val result1 = keyIndex.partKeyRecordsFromFilters(Seq(filter1), 0, Long.MaxValue) + val expected1 = Seq(pkrs(7), pkrs(8), pkrs(9)) + + result1.map(_.partKey.toSeq) shouldEqual expected1.map(_.partKey.toSeq) + result1.map(p => (p.startTime, p.endTime)) shouldEqual expected1.map(p => (p.startTime, p.endTime)) + + // Query with non existent label name with an empty regex + val filter2 = ColumnFilter("dummy", EqualsRegex(".*".utf8)) + val filter3 = ColumnFilter("Actor2Code", Equals("GOV".utf8)) + val result2 = keyIndex.partKeyRecordsFromFilters(Seq(filter2, filter3), 0, Long.MaxValue) + val expected2 = Seq(pkrs(7), pkrs(8), pkrs(9)) + + result2.map(_.partKey.toSeq) shouldEqual expected2.map(_.partKey.toSeq) + result2.map(p => (p.startTime, p.endTime)) shouldEqual expected2.map(p => (p.startTime, p.endTime)) + + // Query with non existent label name with an regex matching at least 1 character + val filter4 = ColumnFilter("dummy", EqualsRegex(".+".utf8)) + val filter5 = ColumnFilter("Actor2Code", Equals("GOV".utf8)) + val result3 = keyIndex.partKeyRecordsFromFilters(Seq(filter4, filter5), 0, Long.MaxValue) + result3 shouldEqual Seq() + + // Query with non existent label name with an empty regex + val filter6 = ColumnFilter("dummy", EqualsRegex("".utf8)) + val filter7 = ColumnFilter("Actor2Code", Equals("GOV".utf8)) + val result4 = keyIndex.partKeyRecordsFromFilters(Seq(filter6, filter7), 0, Long.MaxValue) + val expected4 = Seq(pkrs(7), pkrs(8), pkrs(9)) + result4.map(_.partKey.toSeq) shouldEqual expected4.map(_.partKey.toSeq) + result4.map(p => (p.startTime, p.endTime)) shouldEqual expected4.map(p => (p.startTime, p.endTime)) + + // Query with non existent label name with an empty equals + val filter8 = ColumnFilter("dummy", Equals("".utf8)) + val filter9 = ColumnFilter("Actor2Code", Equals("GOV".utf8)) + val result5 = keyIndex.partKeyRecordsFromFilters(Seq(filter8, filter9), 0, Long.MaxValue) + val expected5 = Seq(pkrs(7), pkrs(8), pkrs(9)) + result5.map(_.partKey.toSeq) shouldEqual expected5.map(_.partKey.toSeq) + result5.map(p => (p.startTime, p.endTime)) shouldEqual expected5.map(p => (p.startTime, p.endTime)) + } } \ No newline at end of file From f5018ae1bc4ae9e2c6e8fe10b55b4052c96455dd Mon Sep 17 00:00:00 2001 From: yu-shipit <120689245+yu-shipit@users.noreply.github.com> Date: Mon, 31 Jul 2023 00:28:36 -0700 Subject: [PATCH 17/39] fix(core) fix the binary join aggregation across different partitions. (#1629) * fix(core) fix the unless operator for aggregators. For regex shard key we need to aggregate across all nodes. InProcessPlanDispatcher is needed. --------- Co-authored-by: Yu Zhang --- .../queryplanner/LogicalPlanUtils.scala | 4 + .../queryplanner/PlannerHierarchySpec.scala | 131 +++++++++++++++++- 2 files changed, 131 insertions(+), 4 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala index fdb92fc1b8..8a6bcd603e 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala @@ -34,6 +34,10 @@ object LogicalPlanUtils extends StrictLogging { */ def hasDescendantAggregate(lp: LogicalPlan): Boolean = lp match { case _: Aggregate => true + // consider this BinaryJoin example foo + on(h) + bar. + // partition1 has foo{h=1}, bar{h1=2} and partition2 has foo{h=2}, bar{h1=1} + // the binary join cannot happen on a partition locally. InProcessPlanDispatcher is required. + case _: BinaryJoin => true case nonLeaf: NonLeafLogicalPlan => nonLeaf.children.exists(hasDescendantAggregate(_)) case _ => false } diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala index 1c03160273..640b84e118 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala @@ -112,7 +112,8 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS "localPartition", dataset, queryConfig) private val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { - if (shardColumnFilters.nonEmpty) { + // we may have mixed of a regex filter and a non-regex filter. + if (shardColumnFilters.exists(f => f.column == "_ns_" && f.filter.isInstanceOf[EqualsRegex])) { // to ensure that tests dont call something else that is not configured require(shardColumnFilters.exists(f => f.column == "_ns_" && f.filter.isInstanceOf[EqualsRegex] && ( @@ -130,25 +131,30 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("remoteNs"))) ) } + } else if (shardColumnFilters.exists(f => f.column == "_ns_" && f.filter.isInstanceOf[Equals])) { + Seq(shardColumnFilters) } else { Nil } // i.e. filters for a scalar } private val oneRemoteShardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { - if (shardColumnFilters.nonEmpty) { + if (shardColumnFilters.exists(f => f.column == "_ns_" && f.filter.isInstanceOf[EqualsRegex])) { // to ensure that tests dont call something else that is not configured require(shardColumnFilters.exists(f => f.column == "_ns_" && f.filter.isInstanceOf[EqualsRegex] && f.filter.asInstanceOf[EqualsRegex].pattern.toString == ".*remoteNs")) Seq( Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("remoteNs"))) ) - } else { + } else if (shardColumnFilters.nonEmpty) { + Seq(shardColumnFilters) + } + else { Nil } // i.e. filters for a scalar } private val twoRemoteShardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { - if (shardColumnFilters.nonEmpty) { + if (shardColumnFilters.exists(f => f.column == "_ns_" && f.filter.isInstanceOf[EqualsRegex])) { // to ensure that tests dont call something else that is not configured require(shardColumnFilters.exists(f => f.column == "_ns_" && f.filter.isInstanceOf[EqualsRegex] && f.filter.asInstanceOf[EqualsRegex].pattern.toString == "remoteNs.*")) @@ -156,6 +162,8 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("remoteNs0"))), Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("remoteNs1"))), ) + } else if (shardColumnFilters.nonEmpty) { + Seq(shardColumnFilters) } else { Nil } // i.e. filters for a scalar @@ -203,6 +211,121 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS validatePlan(execPlan, expected) } + it("should aggregate through ActorPlanDispatcher for localNs") { + val lp = Parser.queryRangeToLogicalPlan( + """count(foo{_ws_="demo", _ns_="localNs", instance="Inst-1" } + | unless on (instance) + | bar{_ws_ = "demo", _ns_="localNs", instance=~".*"} )""".stripMargin, + TimeStepParams(startSeconds, step, endSeconds), Antlr) + val execPlan = rootPlanner.materialize(lp, QueryContext(origQueryParams = queryParams)) + val expected = + """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192))) + |-T~AggregatePresenter(aggrOp=Count, aggrParams=List(), rangeParams=RangeParams(1634173130,300,1634777330)) + |--E~LocalPartitionReduceAggregateExec(aggrOp=Count, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1888594662],raw) + |---T~AggregateMapReduce(aggrOp=Count, aggrParams=List(), without=List(), by=List()) + |----E~SetOperatorExec(binaryOp=LUnless, on=List(instance), ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1888594662],raw) + |-----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1888594662],raw) + |-----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1888594662],raw) + |-----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,EqualsRegex(.*)), ColumnFilter(_metric_,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1888594662],raw) + |-----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,EqualsRegex(.*)), ColumnFilter(_metric_,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1888594662],raw) + |-T~AggregatePresenter(aggrOp=Count, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634172830)) + |--E~LocalPartitionReduceAggregateExec(aggrOp=Count, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1888594662],downsample) + |---T~AggregateMapReduce(aggrOp=Count, aggrParams=List(), without=List(), by=List()) + |----E~SetOperatorExec(binaryOp=LUnless, on=List(instance), ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1888594662],downsample) + |-----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1888594662],downsample) + |-----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1888594662],downsample) + |-----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,EqualsRegex(.*)), ColumnFilter(_metric_,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1888594662],downsample) + |-----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,EqualsRegex(.*)), ColumnFilter(_metric_,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1888594662],downsample)""".stripMargin + validatePlan(execPlan, expected) + } + + it("should aggregate through InProcessPlanDispatcher for remote _ns_ using regex and see two local Ns") { + val lp = Parser.queryRangeToLogicalPlan( + """count(foo{_ws_="demo", _ns_="localNs", instance="Inst-1" } + | unless on (instance) + | bar{_ws_ = "demo", _ns_ =~"localNs.*", instance=~".*"} )""".stripMargin, + TimeStepParams(startSeconds, step, endSeconds), Antlr) + val execPlan = rootPlanner.materialize(lp, QueryContext(origQueryParams = queryParams)) + val expected = + """T~AggregatePresenter(aggrOp=Count, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634777330)) + |-E~LocalPartitionReduceAggregateExec(aggrOp=Count, aggrParams=List()) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) + |--T~AggregateMapReduce(aggrOp=Count, aggrParams=List(), without=List(), by=List()) + |---E~SetOperatorExec(binaryOp=LUnless, on=List(instance), ignoring=List()) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) + |----E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192))) + |-----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],raw) + |------T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) + |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],raw) + |------T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) + |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],raw) + |-----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],downsample) + |------T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) + |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],downsample) + |------T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) + |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],downsample) + |----E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) + |-----E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192))) + |------E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],raw) + |-------T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) + |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(instance,EqualsRegex(.*)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs1)), ColumnFilter(_metric_,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],raw) + |-------T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) + |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(instance,EqualsRegex(.*)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs1)), ColumnFilter(_metric_,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],raw) + |------E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],downsample) + |-------T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) + |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(instance,EqualsRegex(.*)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs1)), ColumnFilter(_metric_,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],downsample) + |-------T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) + |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(instance,EqualsRegex(.*)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs1)), ColumnFilter(_metric_,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],downsample) + |-----E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192))) + |------E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],raw) + |-------T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) + |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(instance,EqualsRegex(.*)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],raw) + |-------T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) + |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(instance,EqualsRegex(.*)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],raw) + |------E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],downsample) + |-------T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) + |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(instance,EqualsRegex(.*)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],downsample) + |-------T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) + |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(instance,EqualsRegex(.*)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],downsample)""".stripMargin + validatePlan(execPlan, expected) + } + + it("should aggregate through InProcessPlanDispatcher for remote _ns_ using regex and see both remote and local Ns") { + val lp = Parser.queryRangeToLogicalPlan( + """count(foo{_ws_="demo", _ns_="localNs", instance="Inst-1" } + | unless on (instance) + | bar{_ws_ = "demo", _ns_ =~"remoteNs.*", instance=~".*"} )""".stripMargin, + TimeStepParams(startSeconds, step, endSeconds), Antlr) + val execPlan = twoRemoteRootPlanner.materialize(lp, QueryContext(origQueryParams = queryParams, + plannerParams = PlannerParams(processMultiPartition = true))) + val expected = + """T~AggregatePresenter(aggrOp=Count, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634777330)) + |-E~LocalPartitionReduceAggregateExec(aggrOp=Count, aggrParams=List()) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) + |--T~AggregateMapReduce(aggrOp=Count, aggrParams=List(), without=List(), by=List()) + |---E~SetOperatorExec(binaryOp=LUnless, on=List(instance), ignoring=List()) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) + |----E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192))) + |-----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1681100802],raw) + |------T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) + |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1681100802],raw) + |------T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) + |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1681100802],raw) + |-----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1681100802],downsample) + |------T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) + |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1681100802],downsample) + |------T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) + |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1681100802],downsample) + |----E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) + |-----E~PromQlRemoteExec(PromQlQueryParams(bar{instance=~".*",_ws_="demo",_ns_="remoteNs0"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remotePartition-url0, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) + |-----E~PromQlRemoteExec(PromQlQueryParams(bar{instance=~".*",_ws_="demo",_ns_="remoteNs1"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remote1Partition-url1, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin + validatePlan(execPlan, expected) + } + it("should generate plan for one recording rule query") { val lp = Parser.queryRangeToLogicalPlan( """sum(foo:1m{_ws_ = "demo", _ns_ = "localNs", instance = "Inst-1" })""", From 84a185f046c64b971eac33550106f29f08035ee7 Mon Sep 17 00:00:00 2001 From: sandeep6189 Date: Wed, 2 Aug 2023 12:00:08 -0700 Subject: [PATCH 18/39] feat(query): Cardinality V2 API Query Plan changes (#1637) feat(query): Cardinality V2 API Query Plan changes --- .../TenantIngestionMetering.scala | 4 +- .../queryplanner/LongTimeRangePlanner.scala | 28 +++++- .../queryplanner/MultiPartitionPlanner.scala | 9 +- .../queryplanner/SingleClusterPlanner.scala | 2 +- .../queryplanner/SinglePartitionPlanner.scala | 11 +-- .../LongTimeRangePlannerSpec.scala | 35 +++++++ .../MultiPartitionPlannerSpec.scala | 37 +++++++- .../queryplanner/PlannerHierarchySpec.scala | 2 +- .../SinglePartitionPlannerSpec.scala | 68 ++++++++++++-- .../main/scala/filodb/query/LogicalPlan.scala | 21 ++++- .../scala/filodb/query/PromCirceSupport.scala | 22 ++++- .../filodb/query/PromQueryResponse.scala | 15 ++- .../filodb/query/exec/MetadataExecPlan.scala | 69 +++++++++++--- .../query/exec/MetadataRemoteExec.scala | 29 ++++++ .../scala/filodb/query/LogicalPlanSpec.scala | 12 +++ .../filodb/query/PromCirceSupportSpec.scala | 92 +++++++++++++++++++ .../filodb/query/exec/MetadataExecSpec.scala | 61 +++++++----- .../query/exec/RemoteMetadataExecSpec.scala | 50 +++++++++- 18 files changed, 488 insertions(+), 79 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/TenantIngestionMetering.scala b/coordinator/src/main/scala/filodb.coordinator/TenantIngestionMetering.scala index 04e396d125..3a2b737d59 100644 --- a/coordinator/src/main/scala/filodb.coordinator/TenantIngestionMetering.scala +++ b/coordinator/src/main/scala/filodb.coordinator/TenantIngestionMetering.scala @@ -79,11 +79,11 @@ case class TenantIngestionMetering(settings: FilodbSettings, "cluster_type" -> CLUSTER_TYPE) if (CLUSTER_TYPE == "downsample") { - Kamon.gauge(METRIC_LONGTERM).withTags(TagSet.from(tags)).update(data.counts.total.toDouble) + Kamon.gauge(METRIC_LONGTERM).withTags(TagSet.from(tags)).update(data.counts.longTerm.toDouble) } else { Kamon.gauge(METRIC_ACTIVE).withTags(TagSet.from(tags)).update(data.counts.active.toDouble) - Kamon.gauge(METRIC_TOTAL).withTags(TagSet.from(tags)).update(data.counts.total.toDouble) + Kamon.gauge(METRIC_TOTAL).withTags(TagSet.from(tags)).update(data.counts.shortTerm.toDouble) } }) case Success(QueryError(_, _, t)) => logger.warn("QueryError: " + t.getMessage) diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LongTimeRangePlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LongTimeRangePlanner.scala index 0d62aa56f8..cdc7fa88d5 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LongTimeRangePlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LongTimeRangePlanner.scala @@ -190,6 +190,28 @@ import filodb.query.exec._ PlanResult(Seq(execPlan)) } + /** + * Materialize Ts cardinality plan. For v1 version, we only go to raw cluster for back compatibility. For v2 versions, + * we would go to both downsample and raw cluster + * + * @param logicalPlan The TsCardinalities logical plan to materialize + * @param queryContext The QueryContext object + * @return + */ + private def materializeTSCardinalityPlan(queryContext: QueryContext, logicalPlan: TsCardinalities): PlanResult = { + logicalPlan.version match { + case 2 => { + val rawPlan = rawClusterPlanner.materialize(logicalPlan, queryContext) + val dsPlan = downsampleClusterPlanner.materialize(logicalPlan, queryContext) + val stitchedPlan = TsCardReduceExec(queryContext, stitchDispatcher, Seq(rawPlan, dsPlan)) + PlanResult(Seq(stitchedPlan)) + } + // version 1 defaults to raw as done before + case 1 => rawClusterMaterialize(queryContext, logicalPlan) + case _ => throw new UnsupportedOperationException(s"version ${logicalPlan.version} not supported!") + } + } + // scalastyle:off cyclomatic.complexity override def walkLogicalPlanTree(logicalPlan: LogicalPlan, qContext: QueryContext, @@ -199,13 +221,13 @@ import filodb.query.exec._ logicalPlan match { case p: PeriodicSeriesPlan => materializePeriodicSeriesPlan(qContext, p) case lc: LabelCardinality => materializeLabelCardinalityPlan(lc, qContext) + case ts: TsCardinalities => materializeTSCardinalityPlan(qContext, ts) case _: LabelValues | _: ApplyLimitFunction | _: SeriesKeysByFilters | _: ApplyInstantFunctionRaw | _: RawSeries | - _: LabelNames | - _: TsCardinalities => rawClusterMaterialize(qContext, logicalPlan) + _: LabelNames => rawClusterMaterialize(qContext, logicalPlan) } } else logicalPlan match { @@ -219,7 +241,7 @@ import filodb.query.exec._ case lp: BinaryJoin => materializePeriodicSeriesPlan(qContext, lp) case lp: ScalarVectorBinaryOperation => super.materializeScalarVectorBinOp(qContext, lp) case lp: LabelValues => rawClusterMaterialize(qContext, lp) - case lp: TsCardinalities => rawClusterMaterialize(qContext, lp) + case lp: TsCardinalities => materializeTSCardinalityPlan(qContext, lp) case lp: SeriesKeysByFilters => rawClusterMaterialize(qContext, lp) case lp: ApplyMiscellaneousFunction => super.materializeApplyMiscellaneousFunction(qContext, lp) case lp: ApplySortFunction => super.materializeApplySortFunction(qContext, lp) diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala index 668bd625fe..60eba29b63 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala @@ -601,8 +601,6 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider def materializeTsCardinalities(lp: TsCardinalities, qContext: QueryContext): PlanResult = { - import TsCardinalities._ - val queryParams = qContext.origQueryParams.asInstanceOf[PromQlQueryParams] val partitions = if (lp.shardKeyPrefix.size >= 2) { // At least a ws/ns pair is required to select specific partitions. @@ -621,12 +619,7 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider if (p.partitionName.equals(localPartitionName)) localPartitionPlanner.materialize(lp, qContext) else { - val params = Map( - "match[]" -> ("{" + SHARD_KEY_LABELS.zip(lp.shardKeyPrefix) - .map{ case (label, value) => s"""$label="$value""""} - .mkString(",") + "}"), - "numGroupByFields" -> lp.numGroupByFields.toString) - createMetadataRemoteExec(qContext, p, params) + createMetadataRemoteExec(qContext, p, lp.queryParams()) } } if (execPlans.size == 1) { diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala index 810e01ef52..4b31c7b209 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala @@ -832,7 +832,7 @@ class SingleClusterPlanner(val dataset: Dataset, forceInProcess: Boolean): PlanResult = { val metaExec = shardMapperFunc.assignedShards.map{ shard => val dispatcher = dispatcherForShard(shard, forceInProcess, qContext) - exec.TsCardExec(qContext, dispatcher, dsRef, shard, lp.shardKeyPrefix, lp.numGroupByFields) + exec.TsCardExec(qContext, dispatcher, dsRef, shard, lp.shardKeyPrefix, lp.numGroupByFields, clusterName) } PlanResult(metaExec) } diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SinglePartitionPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SinglePartitionPlanner.scala index 3c7e20f569..aab22c2ce2 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SinglePartitionPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SinglePartitionPlanner.scala @@ -11,15 +11,10 @@ import filodb.query.exec._ * distributed across multiple clusters. * * @param planners map of clusters names in the local partition to their Planner objects - * @param defaultPlanner TsCardinalities queries are routed here. - * Note: this is a temporary fix only to support TsCardinalities queries. - * These must be routed to planners according to the data they govern, and - * this information isn't accessible without this parameter. * @param plannerSelector a function that selects the planner name given the metric name * @param dataset a function that selects the planner name given the metric name */ class SinglePartitionPlanner(planners: Map[String, QueryPlanner], - defaultPlanner: String, // TODO: remove this-- see above. plannerSelector: String => String, val dataset: Dataset, val queryConfig: QueryConfig) @@ -74,9 +69,9 @@ class SinglePartitionPlanner(planners: Map[String, QueryPlanner], } private def materializeTsCardinalities(logicalPlan: TsCardinalities, qContext: QueryContext): PlanResult = { - // Delegate to defaultPlanner - planners.get(defaultPlanner).map(p => PlanResult(Seq(p.materialize(logicalPlan, qContext)))) - .getOrElse(PlanResult(Seq())) + val execPlans = logicalPlan.datasets.map(d => planners.get(d)) + .map(x => x.get.materialize(logicalPlan, qContext)) + PlanResult(Seq(TsCardReduceExec(qContext, inProcessPlanDispatcher, execPlans))) } diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/LongTimeRangePlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/LongTimeRangePlannerSpec.scala index 1b1600dc1c..aa3f24edc7 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/LongTimeRangePlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/LongTimeRangePlannerSpec.scala @@ -336,6 +336,41 @@ class LongTimeRangePlannerSpec extends AnyFunSpec with Matchers with PlanValidat binaryJoinExec.rhs.head.isInstanceOf[StitchRvsExec] shouldEqual (true) } + it("tsCardinality should span to both downsample and raw for version 2") { + val logicalPlan = TsCardinalities(Seq("a","b"), 2, 2, Seq("longtime-prometheus")) + + val cardExecPlan = longTermPlanner.materialize( + logicalPlan, + QueryContext(origQueryParams = promQlQueryParams.copy(promQl = ""))).asInstanceOf[TsCardReduceExec] + + cardExecPlan.dispatcher.isInstanceOf[InProcessPlanDispatcher] shouldEqual true + cardExecPlan.children.size shouldEqual 2 + val rawEp = cardExecPlan.children.head.asInstanceOf[MockExecPlan] + val downsampleEp = cardExecPlan.children.last.asInstanceOf[MockExecPlan] + + rawEp.name shouldEqual "raw" + downsampleEp.name shouldEqual "downsample" + } + + it("tsCardinality should throw exception for version > 2") { + val logicalPlan = TsCardinalities(Seq("a", "b"), 2, 3, Seq("longtime-prometheus")) + val ex = intercept[UnsupportedOperationException] { + val cardExecPlan = longTermPlanner.materialize( + logicalPlan, + QueryContext(origQueryParams = promQlQueryParams.copy(promQl = ""))) + } + ex.getMessage.contains("version 3 not supported!") shouldEqual true + } + + it("tsCardinality should span to raw ONLY for version 1") { + val logicalPlan = TsCardinalities(Seq("a", "b"), 2, 1, Seq("longtime-prometheus")) + + val cardRawExecPlan = longTermPlanner.materialize( + logicalPlan, + QueryContext(origQueryParams = promQlQueryParams.copy(promQl = ""))).asInstanceOf[MockExecPlan] + + cardRawExecPlan.name shouldEqual "raw" + } it("should direct overlapping binary join offset queries with vector(0) " + "to both raw & downsample planner and stitch") { diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala index 2b44b64667..5e237b068f 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala @@ -1049,7 +1049,7 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers with PlanValida (endSeconds * 1000) } - it ("should generate correct ExecPlan for TsCardinalities query") { + it ("should generate correct ExecPlan for TsCardinalities query version 1") { def partitions(timeRange: TimeRange): List[PartitionAssignment] = List(PartitionAssignment("remote", "remote-url", TimeRange(startSeconds * 1000, localPartitionStart * 1000 - 1)), @@ -1066,7 +1066,8 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers with PlanValida val engine = new MultiPartitionPlanner(partitionLocationProvider, localPlanner, "local", dataset, queryConfig) val lp = TsCardinalities(Seq("a", "b"), 3) val promQlQueryParams = PromQlQueryParams("", startSeconds, step, endSeconds, Some("/api/v1/metering/cardinality/timeseries")) - val expectedUrlParams = Map("match[]" -> """{_ws_="a",_ns_="b"}""", "numGroupByFields" -> "3") + val expectedUrlParams = Map("match[]" -> """{_ws_="a",_ns_="b"}""", "numGroupByFields" -> "3", "verbose" -> "true", + "datasets" -> "") val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams, plannerParams = PlannerParams(processMultiPartition = true))) @@ -1077,6 +1078,38 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers with PlanValida execPlan.children(1).asInstanceOf[MetadataRemoteExec].urlParams shouldEqual(expectedUrlParams) } + it("should generate correct ExecPlan for TsCardinalities query version 2") { + def partitions(timeRange: TimeRange): List[PartitionAssignment] = + List(PartitionAssignment("remote", "remote-url", + TimeRange(startSeconds * 1000, localPartitionStart * 1000 - 1)), + PartitionAssignment("local", "local-url", TimeRange(localPartitionStart * 1000, endSeconds * 1000))) + + val partitionLocationProvider = new PartitionLocationProvider { + override def getPartitions(routingKey: Map[String, String], timeRange: TimeRange): List[PartitionAssignment] = + partitions(timeRange) + + override def getMetadataPartitions(nonMetricShardKeyFilters: Seq[ColumnFilter], + timeRange: TimeRange):List[PartitionAssignment] = + partitions(timeRange) + } + + val engine = new MultiPartitionPlanner(partitionLocationProvider, localPlanner, "local", + dataset, queryConfig) + val lp = TsCardinalities(Seq("a", "b"), 3, 2, Seq("longtime-prometheus","recordingrules-prometheus_rules_1m")) + val promQlQueryParams = PromQlQueryParams("", startSeconds, step, endSeconds, + Some("/api/v2/metering/cardinality/timeseries")) + val expectedUrlParams = Map("match[]" -> """{_ws_="a",_ns_="b"}""", "numGroupByFields" -> "3","verbose" -> "true", + "datasets" -> "longtime-prometheus,recordingrules-prometheus_rules_1m") + + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams, plannerParams = + PlannerParams(processMultiPartition = true))) + + execPlan.isInstanceOf[TsCardReduceExec] shouldEqual (true) + execPlan.children(0).isInstanceOf[TsCardReduceExec] shouldEqual (true) + execPlan.children(1).isInstanceOf[MetadataRemoteExec] shouldEqual (true) + execPlan.children(1).asInstanceOf[MetadataRemoteExec].urlParams shouldEqual (expectedUrlParams) + } + it ("should generate multipartition BinaryJoin") { def partitions(timeRange: TimeRange): List[PartitionAssignment] = List(PartitionAssignment("remote", "remote-url", TimeRange(timeRange.startMs, timeRange.endMs))) diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala index 640b84e118..3f194654db 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala @@ -63,7 +63,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS if (metricName.contains(":1m")) "recordingRules" else "longTerm" } val planners = Map("longTerm" -> longTermPlanner, "recordingRules" -> recordingRulesPlanner) - val singlePartitionPlanner = new SinglePartitionPlanner(planners, "longTerm", plannerSelector, + val singlePartitionPlanner = new SinglePartitionPlanner(planners, plannerSelector, dataset, queryConfig) private val oneRemotePartitionLocationProvider = new PartitionLocationProvider { diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/SinglePartitionPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/SinglePartitionPlannerSpec.scala index 081fc02001..79098932e6 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/SinglePartitionPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/SinglePartitionPlannerSpec.scala @@ -4,7 +4,6 @@ import akka.actor.ActorSystem import akka.testkit.TestProbe import com.typesafe.config.ConfigFactory import monix.execution.Scheduler - import filodb.coordinator.{ActorPlanDispatcher, ShardMapper} import filodb.core.{DatasetRef, MetricsTestData} import filodb.core.metadata.Schemas @@ -81,10 +80,13 @@ class SinglePartitionPlannerSpec extends AnyFunSpec with Matchers { } val planners = Map("local" -> highAvailabilityPlanner, "rules1" -> rrPlanner1, "rules2" -> rrPlanner2) - val plannerSelector = (metricName: String) => { if (metricName.equals("rr1")) "rules1" - else if (metricName.equals("rr2")) "rules2" else "local" } - val engine = new SinglePartitionPlanner(planners, "local", plannerSelector, dataset, queryConfig) + val plannerSelector = (metricName: String) => { + if (metricName.equals("rr1")) "rules1" + else if (metricName.equals("rr2")) "rules2" else "local" + } + + val engine = new SinglePartitionPlanner(planners, plannerSelector, dataset, queryConfig) it("should generate Exec plan for simple query") { val lp = Parser.queryToLogicalPlan("test{job = \"app\"}", 1000, 1000) @@ -160,7 +162,7 @@ class SinglePartitionPlannerSpec extends AnyFunSpec with Matchers { execPlan.asInstanceOf[PartKeysDistConcatExec].children(2).asInstanceOf[MockExecPlan].name shouldEqual ("rules2") } - it("should generate correct ExecPlan for TsCardinalities") { + it("should generate correct ExecPlan for TsCardinalities version 1") { // Note: this test is expected to break when TsCardinalities.isRoutable = true // Note: unrelated to the above, this test is setup to confirm that a hacky fix to @@ -169,15 +171,63 @@ class SinglePartitionPlannerSpec extends AnyFunSpec with Matchers { val localPlanner = new SingleClusterPlanner( dataset, schemas, localMapper, earliestRetainedTimestampFn = 0, queryConfig, "raw-temp") val planners = Map("raw-temp" -> localPlanner, "rules1" -> rrPlanner1, "rules2" -> rrPlanner2) - val engine = new SinglePartitionPlanner(planners, "raw-temp", plannerSelector, dataset, queryConfig) - val lp = TsCardinalities(Seq("a", "b"), 2) + val engine = new SinglePartitionPlanner(planners, plannerSelector, dataset, queryConfig) + val lp = TsCardinalities(Seq("a", "b"), 2, 1, Seq("raw-temp")) + + // Plan should just contain a single root TsCardReduceExec and its TsCardExec children. + // Currently, queries are routed only to the planner who's name equals the SPP's "defaultPlanner" member. + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams.copy(promQl = ""))) + execPlan.isInstanceOf[TsCardReduceExec] shouldEqual (true) + + // UPDATE: We added another TsCardReduceExec to merge data across different datasets + execPlan.asInstanceOf[TsCardReduceExec].children.length shouldEqual(1) + execPlan.asInstanceOf[TsCardReduceExec].children(0).children.length shouldEqual(32) + execPlan.children.forall(_.isInstanceOf[TsCardReduceExec]) shouldEqual true + execPlan.children(0).children.forall(_.isInstanceOf[TsCardExec]) shouldEqual true + } + + it("should generate correct ExecPlan for TsCardinalities version 2") { + + // Note: this test is expected to break when TsCardinalities.isRoutable = true + // Note: unrelated to the above, this test is setup to confirm that a hacky fix to + // SPP::materializeTsCardinalities is working. See there for additional details. + + val rawPlanner = new SingleClusterPlanner( + dataset, schemas, localMapper, earliestRetainedTimestampFn = 0, queryConfig, "raw") + + val downsamplePlanner = new SingleClusterPlanner( + dataset, schemas, localMapper, earliestRetainedTimestampFn = 0, queryConfig, "downsample") + + val longTermPlanner = new LongTimeRangePlanner(rawPlanner, downsamplePlanner, 0, 0, + InProcessPlanDispatcher(QueryConfig.unitTestingQueryConfig), queryConfig, dataset) + + val planners = Map("longtime" -> longTermPlanner, "rules1" -> rrPlanner1, "rules2" -> rrPlanner2) + + val engine = new SinglePartitionPlanner(planners, plannerSelector, dataset, queryConfig) + val lp = TsCardinalities(Seq("a", "b"), 2, 2, Seq("longtime", "rules1", "rules2")) // Plan should just contain a single root TsCardReduceExec and its TsCardExec children. // Currently, queries are routed only to the planner who's name equals the SPP's "defaultPlanner" member. val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams.copy(promQl = ""))) execPlan.isInstanceOf[TsCardReduceExec] shouldEqual (true) - execPlan.asInstanceOf[TsCardReduceExec].children.length shouldEqual(32) - execPlan.children.forall(_.isInstanceOf[TsCardExec]) shouldEqual true + + // UPDATE: should have 3 children, since we passed 3 datasets + execPlan.asInstanceOf[TsCardReduceExec].children.length shouldEqual (3) + + // the longtime should have 2 children for downsample and raw + val longTermExecPlan = execPlan.asInstanceOf[TsCardReduceExec].children(0) + longTermExecPlan.children.length shouldEqual (2) + longTermExecPlan.children(0).dispatcher.clusterName shouldEqual "raw" + longTermExecPlan.children(1).dispatcher.clusterName shouldEqual "downsample" + + longTermExecPlan.children(0).children.forall(_.isInstanceOf[TsCardExec]) shouldEqual true + longTermExecPlan.children(1).children.forall(_.isInstanceOf[TsCardExec]) shouldEqual true + + // rules plan + val rules1ExecPlan = execPlan.asInstanceOf[TsCardReduceExec].children(1).asInstanceOf[MockExecPlan] + rules1ExecPlan.name shouldEqual "rules1" + val rules2ExecPlan = execPlan.asInstanceOf[TsCardReduceExec].children(2).asInstanceOf[MockExecPlan] + rules2ExecPlan.name shouldEqual "rules2" } it("should generate Exec plan for Scalar query which does not have any metric") { diff --git a/query/src/main/scala/filodb/query/LogicalPlan.scala b/query/src/main/scala/filodb/query/LogicalPlan.scala index 4dc3029308..40f8961c00 100644 --- a/query/src/main/scala/filodb/query/LogicalPlan.scala +++ b/query/src/main/scala/filodb/query/LogicalPlan.scala @@ -2,6 +2,7 @@ package filodb.query import filodb.core.query.{ColumnFilter, RangeParams, RvRange} import filodb.core.query.Filter.Equals +import filodb.query.exec.TsCardExec //scalastyle:off number.of.types //scalastyle:off file.size.limit @@ -181,7 +182,10 @@ object TsCardinalities { * [ws, ns] { 2, 3 } * [ws, ns, metric] { 3 } */ -case class TsCardinalities(shardKeyPrefix: Seq[String], numGroupByFields: Int) extends LogicalPlan { +case class TsCardinalities(shardKeyPrefix: Seq[String], + numGroupByFields: Int, + version: Int = 1, + datasets: Seq[String] = Seq()) extends LogicalPlan { import TsCardinalities._ require(numGroupByFields >= 1 && numGroupByFields <= 3, @@ -196,6 +200,21 @@ case class TsCardinalities(shardKeyPrefix: Seq[String], numGroupByFields: Int) e def filters(): Seq[ColumnFilter] = SHARD_KEY_LABELS.zip(shardKeyPrefix).map{ case (label, value) => ColumnFilter(label, Equals(value))} + + /** + * Helper function to create a Map of all the query parameters to be sent for the remote API call + * @return Immutable Map[String, String] + */ + def queryParams(): Map[String, String] = { + Map( + "match[]" -> ("{" + SHARD_KEY_LABELS.zip(shardKeyPrefix) + .map{ case (label, value) => s"""$label="$value""""} + .mkString(",") + "}"), + "numGroupByFields" -> numGroupByFields.toString, + "verbose" -> "true", // Using this plan to determine if we need to pass additional values in groupKey or not + "datasets" -> datasets.mkString(TsCardExec.PREFIX_DELIM) + ) + } } /** diff --git a/query/src/main/scala/filodb/query/PromCirceSupport.scala b/query/src/main/scala/filodb/query/PromCirceSupport.scala index d1db15b35a..df002ad5b4 100644 --- a/query/src/main/scala/filodb/query/PromCirceSupport.scala +++ b/query/src/main/scala/filodb/query/PromCirceSupport.scala @@ -21,6 +21,8 @@ object PromCirceSupport { Json.fromValues(Seq(group.asJson, cardinality.asJson)) case t @ TsCardinalitiesSampl(group, cardinality) => Json.fromValues(Seq(group.asJson, cardinality.asJson)) + case a @ TsCardinalitiesSamplV2(group, cardinality, dataset, _type) => + Json.fromValues(Seq(group.asJson, cardinality.asJson, dataset.asJson, _type.asJson)) } implicit val decodeAvgSample: Decoder[AvgSampl] = new Decoder[AvgSampl] { @@ -57,10 +59,22 @@ object PromCirceSupport { card <- c.get[Seq[Map[String, String]]]("cardinality") } yield LabelCardinalitySampl(metric, card) } else if (c.downField("group").focus.nonEmpty) { - for { - group <- c.get[Map[String, String]]("group") - card <- c.get[Map[String, Int]]("cardinality") - } yield TsCardinalitiesSampl(group, card) + // V2 Cardinality API also has a dataset field. So we are using it to distinguish + // between the TsCardinalitiesSamplV2 vs TsCardinalitiesSampl response + if (c.downField("dataset").focus.nonEmpty) { + for { + group <- c.get[Map[String, String]]("group") + card <- c.get[Map[String, Int]]("cardinality") + dataset <- c.get[String]("dataset") + _type <- c.get[String]("_type") + } yield TsCardinalitiesSamplV2(group, card, dataset, _type) + } + else { + for { + group <- c.get[Map[String, String]]("group") + card <- c.get[Map[String, Int]]("cardinality") + } yield TsCardinalitiesSampl(group, card) + } } else { throw new IllegalArgumentException("could not decode any expected cardinality-related field") } diff --git a/query/src/main/scala/filodb/query/PromQueryResponse.scala b/query/src/main/scala/filodb/query/PromQueryResponse.scala index 4b77a40982..1b2454f0e9 100644 --- a/query/src/main/scala/filodb/query/PromQueryResponse.scala +++ b/query/src/main/scala/filodb/query/PromQueryResponse.scala @@ -68,4 +68,17 @@ final case class LabelCardinalitySampl(metric: Map[String, String], cardinality: Seq[Map[String, String]]) extends MetadataSampl final case class TsCardinalitiesSampl(group: Map[String, String], - cardinality: Map[String, Int]) extends MetadataSampl \ No newline at end of file + cardinality: Map[String, Int]) extends MetadataSampl + +/** + * @param group map of shardKeyPrefix and its values. Key includes - _ws_, _ns_, __name__ + * @param cardinality map of string and int (cardinality count). Key includes - active, shortTerm, longTerm + * @param dataset This is the user specified dataset for the cureent row. + * Valid value includes - raw, aggregated, recordingrules + * @param _type This is the internal filodb dataset. Example values - prometheus, prometheus_preagg, + * prometheus_rules_longterm, prometheus_rules_1m + */ +final case class TsCardinalitiesSamplV2(group: Map[String, String], + cardinality: Map[String, Int], + dataset: String, + _type: String) extends MetadataSampl \ No newline at end of file diff --git a/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala b/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala index 31a1621496..9e8fec9ce5 100644 --- a/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala +++ b/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala @@ -109,7 +109,7 @@ final case class TsCardReduceExec(queryContext: QueryContext, val taskOfResults = childResponses.flatMap(res => Observable.fromIterable(res._1.result)) .foldLeftL(new mutable.HashMap[ZeroCopyUTF8String, CardCounts])(mapFold) .map{ aggMap => - val it = aggMap.toSeq.sortBy(-_._2.total).map{ case (group, counts) => + val it = aggMap.toSeq.sortBy(-_._2.shortTerm).map{ case (group, counts) => CardRowReader(group, counts) }.iterator IteratorBackedRangeVector(new CustomRangeVectorKey(Map.empty), NoCloseCursor(it), None) @@ -462,9 +462,21 @@ final case object TsCardExec { val PREFIX_DELIM = "," + /** + * This is the V1 schema version of QueryResult for TSCardinalities query + */ + val RESULT_SCHEMA_V1 = ResultSchema(Seq(ColumnInfo("group", ColumnType.StringColumn), + ColumnInfo("active", ColumnType.IntColumn), + ColumnInfo("total", ColumnType.IntColumn)), 1) + + /** + * V2 schema version of QueryResult for TSCardinalities query. One more additional column `longterm` is added + * to represent the cardinality count of downsample clusters + */ val RESULT_SCHEMA = ResultSchema(Seq(ColumnInfo("group", ColumnType.StringColumn), ColumnInfo("active", ColumnType.IntColumn), - ColumnInfo("total", ColumnType.IntColumn)), 1) + ColumnInfo("shortTerm", ColumnType.IntColumn), + ColumnInfo("longTerm", ColumnType.IntColumn)), 1) /** * Convert a shard key prefix to a row's group name. @@ -474,13 +486,28 @@ final case object TsCardExec { prefix.mkString(PREFIX_DELIM).utf8 } - case class CardCounts(active: Int, total: Int) { - if (total < active) { - qLogger.warn(s"CardCounts created with total < active; total: $total, active: $active") + /** + * @param prefix ShardKeyPrefix from the Cardinality Record + * @param datasetName FiloDB dataset name + * @return concatenation of ShardKeyPrefix and filodb dataset + */ + def prefixToGroupWithDataset(prefix: Seq[String], datasetName: String):ZeroCopyUTF8String = { + s"${prefix.mkString(PREFIX_DELIM)}$PREFIX_DELIM$datasetName".utf8 + } + + /** + * @param active Actively (1 hourt) Ingesting Cardinality Count + * @param shortTerm This is the 7 day running Cardinality Count + * @param longTerm upto 6 month running Cardinality Count + */ + case class CardCounts(active: Int, shortTerm: Int, longTerm: Int = 0) { + if (shortTerm < active) { + qLogger.warn(s"CardCounts created with total < active; shortTerm: $shortTerm, active: $active") } def add(other: CardCounts): CardCounts = { CardCounts(active + other.active, - total + other.total) + shortTerm + other.shortTerm, + longTerm + other.longTerm) } } @@ -489,7 +516,8 @@ final case object TsCardExec { override def getBoolean(columnNo: Int): Boolean = ??? override def getInt(columnNo: Int): Int = columnNo match { case 1 => counts.active - case 2 => counts.total + case 2 => counts.shortTerm + case 3 => counts.longTerm case _ => throw new IllegalArgumentException(s"illegal getInt columnNo: $columnNo") } override def getLong(columnNo: Int): Long = ??? @@ -514,8 +542,7 @@ final case object TsCardExec { object RowData { def fromRowReader(rr: RowReader): RowData = { val group = rr.getAny(0).asInstanceOf[ZeroCopyUTF8String] - val counts = CardCounts(rr.getInt(1), - rr.getInt(2)) + val counts = CardCounts(rr.getInt(1), rr.getInt(2), rr.getInt(3)) RowData(group, counts) } } @@ -530,13 +557,17 @@ final case class TsCardExec(queryContext: QueryContext, dataset: DatasetRef, shard: Int, shardKeyPrefix: Seq[String], - numGroupByFields: Int) extends LeafExecPlan with StrictLogging { + numGroupByFields: Int, + clusterName: String) extends LeafExecPlan with StrictLogging { require(numGroupByFields >= 1, "numGroupByFields must be positive") require(numGroupByFields >= shardKeyPrefix.size, s"numGroupByFields ($numGroupByFields) must indicate at least as many " + s"fields as shardKeyPrefix.size (${shardKeyPrefix.size})") + // Making the passed cluster name to lowercase to avoid case complexities for string comparisions + val clusterNameLowercase = clusterName.toLowerCase() + override def enforceSampleLimit: Boolean = false // scalastyle:off method.length @@ -547,15 +578,27 @@ final case class TsCardExec(queryContext: QueryContext, source.checkReadyForQuery(dataset, shard, querySession) source.acquireSharedLock(dataset, shard, querySession) - val rvs = source match { case tsMemStore: TimeSeriesStore => Observable.eval { val cards = tsMemStore.scanTsCardinalities( dataset, Seq(shard), shardKeyPrefix, numGroupByFields) val it = cards.map { card => - CardRowReader(prefixToGroup(card.prefix), - CardCounts(card.value.activeTsCount, card.value.tsCount)) + val groupKey = prefixToGroupWithDataset(card.prefix, dataset.dataset) + + // NOTE: cardinality data from downsample cluster is stored as total count in CardinalityStore. But for the + // user perspective, the cardinality data in downsample is a longterm data. Hence, we are forking the + // data path based on the cluster the data is being served from + if (clusterNameLowercase.contains("downsample")) { + CardRowReader( + groupKey, + CardCounts(0, 0, card.value.tsCount)) + } + else { + CardRowReader( + groupKey, + CardCounts(card.value.activeTsCount, card.value.tsCount)) + } }.iterator IteratorBackedRangeVector(new CustomRangeVectorKey(Map.empty), NoCloseCursor(it), None) } diff --git a/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala b/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala index 4b913c3443..348ff7b8ac 100644 --- a/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala +++ b/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala @@ -68,6 +68,7 @@ case class MetadataRemoteExec(queryEndpoint: String, case _: MetadataMapSampl => mapTypeQueryResponse(response, id) case _: LabelCardinalitySampl => mapLabelCardinalityResponse(response, id) case _: TsCardinalitiesSampl => mapTsCardinalitiesResponse(response, id) + case _: TsCardinalitiesSamplV2 => mapTsCardinalitiesResponseV2(response, id) case _ => labelsQueryResponse(response, id) } } @@ -88,6 +89,34 @@ case class MetadataRemoteExec(queryEndpoint: String, val rv = IteratorBackedRangeVector(CustomRangeVectorKey.empty, NoCloseCursor(rows.iterator), None) // dont add this size to queryStats since it was already added by callee use dummy QueryStats() val srv = SerializedRangeVector(rv, builder, RECORD_SCHEMA, queryWithPlanName(queryContext), dummyQueryStats) + + // NOTE: We are using the RESULT_SCHEMA definitions to determine the iteration of shardKeyPrefix in v1 result. + // Hence, we are sending the older result schema which was used for V1 Cardinality API + QueryResult(id, RESULT_SCHEMA_V1, Seq(srv)) + } + + /** + * @param response Metadata Response from the remote query server API call. + * @param id QueryId + * @return We convert the TsCardinalitiesSamplV2 response to QueryResult which can be appropriately parsed + * by the query service and return the response to the user + */ + private def mapTsCardinalitiesResponseV2(response: MetadataSuccessResponse, id: String): QueryResponse = { + import NoCloseCursor._ + import TsCardinalities._ + import TsCardExec._ + + val RECORD_SCHEMA = SerializedRangeVector.toSchema(RESULT_SCHEMA.columns) + + val rows = response.data.asInstanceOf[Seq[TsCardinalitiesSamplV2]] + .map { ts => + val prefix = SHARD_KEY_LABELS.take(ts.group.size).map(l => ts.group(l)) + val counts = CardCounts(ts.cardinality("active"), ts.cardinality("shortTerm"), ts.cardinality("longTerm")) + CardRowReader(prefixToGroupWithDataset(prefix, ts._type), counts) + } + val rv = IteratorBackedRangeVector(CustomRangeVectorKey.empty, NoCloseCursor(rows.iterator), None) + // dont add this size to queryStats since it was already added by callee use dummy QueryStats() + val srv = SerializedRangeVector(rv, builder, RECORD_SCHEMA, queryWithPlanName(queryContext), dummyQueryStats) QueryResult(id, RESULT_SCHEMA, Seq(srv)) } diff --git a/query/src/test/scala/filodb/query/LogicalPlanSpec.scala b/query/src/test/scala/filodb/query/LogicalPlanSpec.scala index 10f3a140df..e44fd6fa98 100644 --- a/query/src/test/scala/filodb/query/LogicalPlanSpec.scala +++ b/query/src/test/scala/filodb/query/LogicalPlanSpec.scala @@ -323,4 +323,16 @@ class LogicalPlanSpec extends AnyFunSpec with Matchers { TsCardinalities(Seq("a", "b"), 3) TsCardinalities(Seq("a", "b", "c"), 3) } + + it ("TsCardinalities queryParams should have expected values") { + val datasets = Seq("longtime-prometheus", + "recordingrules-prometheus_rules_longterm") + val plan = TsCardinalities(Seq("a","b","c"), 3, 2, datasets) + val queryParamsMap = plan.queryParams() + + queryParamsMap.get("numGroupByFields").get shouldEqual "3" + queryParamsMap.get("datasets").get shouldEqual datasets.mkString(",") + queryParamsMap.get("verbose").get shouldEqual "true" + queryParamsMap.get("match[]").get shouldEqual "{_ws_=\"a\",_ns_=\"b\",__name__=\"c\"}" + } } diff --git a/query/src/test/scala/filodb/query/PromCirceSupportSpec.scala b/query/src/test/scala/filodb/query/PromCirceSupportSpec.scala index ebfdeae638..acbb5c3b48 100644 --- a/query/src/test/scala/filodb/query/PromCirceSupportSpec.scala +++ b/query/src/test/scala/filodb/query/PromCirceSupportSpec.scala @@ -205,6 +205,98 @@ class PromCirceSupportSpec extends AnyFunSpec with Matchers with ScalaFutures { } } + it("should parse TsCardinalitiesSamplV2") { + val expected = Seq( + TsCardinalitiesSamplV2( + Map("_ws_" -> "demo", "_ns_" -> "App-0", "_metric_" -> "heap_usage"), + Map("active" -> 2, "shortTerm" -> 3, "longTerm" -> 5), + "raw", + "prometheus"), + TsCardinalitiesSamplV2( + Map("_ws_" -> "demo", "_ns_" -> "App-1"), + Map("active" -> 6, "shortTerm" -> 8, "longTerm" -> 0), + "recordingrules", + "prometheus_rules_1m"), + TsCardinalitiesSamplV2( + Map("_ws_" -> "demo", "_ns_" -> "App-2"), + Map("active" -> 14, "shortTerm" -> 28, "longTerm" -> 0), + "recordingrules", + "prometheus_rules_longterm"), + TsCardinalitiesSamplV2( + Map("_ws_" -> "demo", "_ns_" -> "App-3", "_metric_" -> "heap_usage:::agg"), + Map("active" -> 11, "shortTerm" -> 22, "longTerm" -> 33), + "aggregated", + "prometheus_preagg") + ) + val inputString = + """{ + | "status": "success", + | "data": [ + | { + | "_type": "prometheus", + | "dataset": "raw", + | "cardinality": { + | "active": 2, + | "longTerm": 5, + | "shortTerm": 3 + | }, + | "group": { + | "_ns_": "App-0", + | "_ws_": "demo", + | "_metric_": "heap_usage" + | } + | }, + | { + | "_type": "prometheus_rules_1m", + | "dataset": "recordingrules", + | "cardinality": { + | "active": 6, + | "longTerm": 0, + | "shortTerm": 8 + | }, + | "group": { + | "_ns_": "App-1", + | "_ws_": "demo" + | } + | }, + | { + | "_type": "prometheus_rules_longterm", + | "dataset": "recordingrules", + | "cardinality": { + | "active": 14, + | "longTerm": 0, + | "shortTerm": 28 + | }, + | "group": { + | "_ns_": "App-2", + | "_ws_": "demo" + | } + | }, + | { + | "_type": "prometheus_preagg", + | "dataset": "aggregated", + | "cardinality": { + | "active": 11, + | "longTerm": 33, + | "shortTerm": 22 + | }, + | "group": { + | "_ns_": "App-3", + | "_ws_": "demo", + | "_metric_": "heap_usage:::agg" + | } + | } + | ], + | "errorType": null, + | "error": null + |}""".stripMargin + + parser.decode[MetadataSuccessResponse](inputString) match { + case Right(response) => response shouldEqual MetadataSuccessResponse(expected) + case Left(ex) => throw ex + } + } + it("should parse aggregateResponse") { val input = """[{ | "status": "success", diff --git a/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala b/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala index e50b8ef4cc..556b6a9221 100644 --- a/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala @@ -367,44 +367,55 @@ class MetadataExecSpec extends AnyFunSpec with Matchers with ScalaFutures with B // and converted to ZeroCopyUTF8Strings. Seq( TestSpec(Seq(), 1, Seq( - Seq("demo") -> CardCounts(4,4), - Seq("demo-A") -> CardCounts(1,1), - Seq("testws") -> CardCounts(1,1) + Seq("demo", "timeseries") -> CardCounts(4,4,4), + Seq("testws", "timeseries") -> CardCounts(1,1,1), + Seq("demo-A", "timeseries") -> CardCounts(1,1,1) )), TestSpec(Seq(), 2, Seq( - Seq("demo", "App-0") -> CardCounts(4,4), - Seq("demo-A", "App-A") -> CardCounts(1,1), - Seq("testws", "testns") -> CardCounts(1,1))), + Seq("demo", "App-0", "timeseries") -> CardCounts(4,4,4), + Seq("testws", "testns", "timeseries") -> CardCounts(1,1,1), + Seq("demo-A", "App-A", "timeseries") -> CardCounts(1,1,1) + )), TestSpec(Seq(), 3, Seq( - Seq("demo", "App-0", "http_req_total") -> CardCounts(2,2), - Seq("testws", "testns", "long_labels_metric") -> CardCounts(1,1), - Seq("demo", "App-0", "http_foo_total") -> CardCounts(1,1), - Seq("demo-A", "App-A", "http_req_total-A") -> CardCounts(1,1), - Seq("demo", "App-0", "http_bar_total") -> CardCounts(1,1))), + Seq("demo", "App-0", "http_req_total", "timeseries") -> CardCounts(2,2,2), + Seq("demo", "App-0", "http_bar_total", "timeseries") -> CardCounts(1,1,1), + Seq("demo", "App-0", "http_foo_total", "timeseries") -> CardCounts(1,1,1), + Seq("demo-A", "App-A", "http_req_total-A", "timeseries") -> CardCounts(1,1,1), + Seq("testws", "testns", "long_labels_metric", "timeseries") -> CardCounts(1,1,1) + )), TestSpec(Seq("demo"), 1, Seq( - Seq("demo") -> CardCounts(4,4))), + Seq("demo", "timeseries") -> CardCounts(4,4,4))), TestSpec(Seq("demo"), 2, Seq( - Seq("demo", "App-0") -> CardCounts(4,4))), + Seq("demo", "App-0", "timeseries") -> CardCounts(4,4,4))), TestSpec(Seq("demo"), 3, Seq( - Seq("demo", "App-0", "http_req_total") -> CardCounts(2,2), - Seq("demo", "App-0", "http_foo_total") -> CardCounts(1,1), - Seq("demo", "App-0", "http_bar_total") -> CardCounts(1,1))), + Seq("demo", "App-0", "http_req_total", "timeseries") -> CardCounts(2,2,2), + Seq("demo", "App-0", "http_bar_total", "timeseries") -> CardCounts(1,1,1), + Seq("demo", "App-0", "http_foo_total", "timeseries") -> CardCounts(1,1,1) + )), TestSpec(Seq("demo", "App-0"), 2, Seq( - Seq("demo", "App-0") -> CardCounts(4,4))), + Seq("demo", "App-0", "timeseries") -> CardCounts(4,4,4) + )), TestSpec(Seq("demo", "App-0"), 3, Seq( - Seq("demo", "App-0", "http_req_total") -> CardCounts(2,2), - Seq("demo", "App-0", "http_foo_total") -> CardCounts(1,1), - Seq("demo", "App-0", "http_bar_total") -> CardCounts(1,1))), + Seq("demo", "App-0", "http_req_total", "timeseries") -> CardCounts(2,2,2), + Seq("demo", "App-0", "http_bar_total", "timeseries") -> CardCounts(1,1,1), + Seq("demo", "App-0", "http_foo_total", "timeseries") -> CardCounts(1,1,1) + )), TestSpec(Seq("demo", "App-0", "http_req_total"), 3, Seq( - Seq("demo", "App-0", "http_req_total") -> CardCounts(2,2))) + Seq("demo", "App-0", "http_req_total", "timeseries") -> CardCounts(2,2,2))) ).foreach{ testSpec => - val leaves = (0 until shardPartKeyLabelValues.size).map{ ishard => - new TsCardExec(QueryContext(), executeDispatcher, - timeseriesDatasetMultipleShardKeys.ref, ishard, testSpec.shardKeyPrefix, testSpec.numGroupByFields) + val leavesRaw = (0 until shardPartKeyLabelValues.size).map{ ishard => + new TsCardExec(QueryContext(), executeDispatcher,timeseriesDatasetMultipleShardKeys.ref, + ishard, testSpec.shardKeyPrefix, testSpec.numGroupByFields, "raw") + }.toSeq + // UPDATE: Simulating the call to downsample cluster to get longterm metrics as well + val leavesDownsample = (0 until shardPartKeyLabelValues.size).map { ishard => + new TsCardExec(QueryContext(), executeDispatcher, timeseriesDatasetMultipleShardKeys.ref, + ishard, testSpec.shardKeyPrefix, testSpec.numGroupByFields, "downsample") }.toSeq - val execPlan = TsCardReduceExec(QueryContext(), executeDispatcher, leaves) + val allLeaves = leavesRaw ++ leavesDownsample + val execPlan = TsCardReduceExec(QueryContext(), executeDispatcher, allLeaves) val resp = execPlan.execute(memStore, querySession).runToFuture.futureValue val result = (resp: @unchecked) match { diff --git a/query/src/test/scala/filodb/query/exec/RemoteMetadataExecSpec.scala b/query/src/test/scala/filodb/query/exec/RemoteMetadataExecSpec.scala index b4a4279529..a13bbb5291 100644 --- a/query/src/test/scala/filodb/query/exec/RemoteMetadataExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/RemoteMetadataExecSpec.scala @@ -247,7 +247,7 @@ class RemoteMetadataExecSpec extends AnyFunSpec with Matchers with ScalaFutures result.toArray shouldEqual jobQueryResult3 } - it ("timeseries cardinality remote exec") { + it ("timeseries cardinality version 1 remote exec") { import TsCardExec._ import TsCardinalities._ @@ -286,5 +286,53 @@ class RemoteMetadataExecSpec extends AnyFunSpec with Matchers with ScalaFutures rows shouldEqual expRows } } + + it("timeseries cardinality version 2 remote exec") { + import TsCardExec._ + import TsCardinalities._ + + val samples = Seq( + TsCardinalitiesSamplV2(Map("_ws_" -> "foo", "_ns_" -> "bar", "__name__" -> "baz"), + Map("active" -> 123, "shortTerm" -> 234, "longTerm" -> 500), "raw", "prometheus"), + TsCardinalitiesSamplV2(Map("_ws_" -> "foo", "_ns_" -> "bar", "__name__" -> "bat"), + Map("active" -> 345, "shortTerm" -> 456, "longTerm" -> 1000), "aggregated", "prometheus_preagg"), + TsCardinalitiesSamplV2(Map("_ws_" -> "foo", "_ns_" -> "bar", "__name__" -> "bak"), + Map("active" -> 567, "shortTerm" -> 678, "longTerm" -> 0), "recordingrules", "prometheus_rules_1m"), + ) + + val testingBackendTsCard: SttpBackend[Future, Nothing] = SttpBackendStub.asynchronousFuture + .whenRequestMatches(_.uri.path.startsWith(List("api", "v2", "metering", "cardinality", "timeseries")) + ) + .thenRespondWrapped(Future { + Response(Right(Right(MetadataSuccessResponse(samples, "success", Option.empty, Option.empty))), + StatusCodes.PartialContent, "", Nil, Nil) + }) + + val exec: MetadataRemoteExec = MetadataRemoteExec( + "http://localhost:31007/api/v2/metering/cardinality/timeseries", 10000L, + Map("match[]" -> """{_ws_="foo", _ns_="bar"}""", "numGroupByFields" -> "3", "verbose" -> "true", + "datasets" -> "longtime-prometheus,longtime-prometheus_preagg,recordingrules-prometheus_rules_1m"), + QueryContext(origQueryParams = PromQlQueryParams("test", 123L, 234L, 15L, + Option("http://localhost:31007/api/v2/metering/cardinality/timeseries"))), + InProcessPlanDispatcher(queryConfig), timeseriesDataset.ref, RemoteHttpClient(configBuilder.build(), + testingBackendTsCard), queryConfig) + + val resp = exec.execute(memStore, querySession).runToFuture.futureValue + val result = (resp: @unchecked) match { + case QueryResult(id, _, response, _, _, _, _) => + // should only contain a single RV where each row describes a single group's cardinalities + response.size shouldEqual 1 + val rows = response.head.rows().map { rr => + RowData.fromRowReader(rr) + }.toSet + val expRows = samples.map { s => + // order the shard keys according to precedence + val prefix = SHARD_KEY_LABELS.map(s.group(_)) + val counts = CardCounts(s.cardinality("active"), s.cardinality("shortTerm"), s.cardinality("longTerm")) + RowData(prefixToGroupWithDataset(prefix, s._type), counts) + }.toSet + rows shouldEqual expRows + } + } } From c7e26a9e44f8f2df23884b3f907a5a9473051b2d Mon Sep 17 00:00:00 2001 From: Amol Nayak Date: Thu, 3 Aug 2023 18:54:00 -0700 Subject: [PATCH 19/39] fix(query) Fix regression with regex match (#1640) --- .../scala/filodb.core/memstore/PartKeyLuceneIndex.scala | 5 ++++- .../scala/filodb.core/memstore/PartKeyLuceneIndexSpec.scala | 6 ++++++ 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala b/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala index 5980821f99..26cea14f83 100644 --- a/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala +++ b/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala @@ -822,7 +822,10 @@ class PartKeyLuceneIndex(ref: DatasetRef, filter match { case EqualsRegex(value) => val regex = removeRegexAnchors(value.toString) - if (regex.replaceAll("\\.\\*", "").nonEmpty) new RegexpQuery(new Term(column, regex), RegExp.NONE) + if (regex.replaceAll("\\.\\*", "") == "") + new MatchAllDocsQuery + else if (regex.nonEmpty) + new RegexpQuery(new Term(column, regex), RegExp.NONE) else leafFilter(column, NotEqualsRegex(".+")) // value="" means the label is absent or has an empty value. case NotEqualsRegex(value) => diff --git a/core/src/test/scala/filodb.core/memstore/PartKeyLuceneIndexSpec.scala b/core/src/test/scala/filodb.core/memstore/PartKeyLuceneIndexSpec.scala index db55753415..9c9854f3e2 100644 --- a/core/src/test/scala/filodb.core/memstore/PartKeyLuceneIndexSpec.scala +++ b/core/src/test/scala/filodb.core/memstore/PartKeyLuceneIndexSpec.scala @@ -1058,5 +1058,11 @@ class PartKeyLuceneIndexSpec extends AnyFunSpec with Matchers with BeforeAndAfte val expected5 = Seq(pkrs(7), pkrs(8), pkrs(9)) result5.map(_.partKey.toSeq) shouldEqual expected5.map(_.partKey.toSeq) result5.map(p => (p.startTime, p.endTime)) shouldEqual expected5.map(p => (p.startTime, p.endTime)) + + + val filter10 = ColumnFilter("Actor2Code", EqualsRegex(".*".utf8)) + val result10= keyIndex.partKeyRecordsFromFilters(Seq(filter10), 0, Long.MaxValue) + result10.map(_.partKey.toSeq) shouldEqual pkrs.map(_.partKey.toSeq) + result10.map(p => (p.startTime, p.endTime)) shouldEqual pkrs.map(p => (p.startTime, p.endTime)) } } \ No newline at end of file From dd5932534a5751ccc4ffb7573a11feef04b8fd2a Mon Sep 17 00:00:00 2001 From: yu-shipit <120689245+yu-shipit@users.noreply.github.com> Date: Fri, 4 Aug 2023 11:41:35 -0700 Subject: [PATCH 20/39] fix(query) support unary operators(+/-) (#1642) convert unary expressions through binary expressions. Co-authored-by: Yu Zhang --- .../filodb/prometheus/ast/Expressions.scala | 10 +++++++- .../filodb/prometheus/parse/ParserSpec.scala | 24 +++++++++++++++++++ 2 files changed, 33 insertions(+), 1 deletion(-) diff --git a/prometheus/src/main/scala/filodb/prometheus/ast/Expressions.scala b/prometheus/src/main/scala/filodb/prometheus/ast/Expressions.scala index 1d0e963f98..553b9162b9 100644 --- a/prometheus/src/main/scala/filodb/prometheus/ast/Expressions.scala +++ b/prometheus/src/main/scala/filodb/prometheus/ast/Expressions.scala @@ -3,8 +3,16 @@ package filodb.prometheus.ast import filodb.core.query.RangeParams import filodb.query._ -case class UnaryExpression(operator: Operator, operand: Expression) extends Expression { +case class UnaryExpression(operator: Operator, operand: Expression) extends Expression with PeriodicSeries { //TODO Need to pass an operator to a series + override def toSeriesPlan(timeParams: TimeRangeParams): PeriodicSeriesPlan = { + if (operator != Add && operator != Sub) { + throw new IllegalArgumentException(s"operator=$operator is not allowed in expression=$operand") + } + // use binary expression to implement the unary operators. + // eg. -foo is implemented through (0 - foo). + BinaryExpression(Scalar(0), operator, None, operand).toSeriesPlan(timeParams) + } } case class PrecedenceExpression(expression: Expression) extends Expression diff --git a/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala b/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala index f4e2e94700..f68edf7b4c 100644 --- a/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala +++ b/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala @@ -555,6 +555,30 @@ class ParserSpec extends AnyFunSpec with Matchers { ) } + it("parse expressions with unary operators") { + parseWithAntlr("-1", """ScalarBinaryOperation(SUB,Left(0.0),Left(1.0),RangeParams(1524855988,1000,1524855988))""") + parseWithAntlr("-foo", """ScalarVectorBinaryOperation(SUB,ScalarFixedDoublePlan(0.0,RangeParams(1524855988,1000,1524855988)),PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(foo))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),true)""") + parseWithAntlr("foo * -1","""BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(foo))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),MUL,OneToOne,ScalarBinaryOperation(SUB,Left(0.0),Left(1.0),RangeParams(1524855988,1000,1524855988)),List(),List(),List())""") + parseWithAntlr("-1 * foo", """BinaryJoin(ScalarBinaryOperation(SUB,Left(0.0),Left(1.0),RangeParams(1524855988,1000,1524855988)),MUL,OneToOne,PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(foo))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),List(),List(),List())""") + parseWithAntlr("-1 * -foo", """BinaryJoin(ScalarBinaryOperation(SUB,Left(0.0),Left(1.0),RangeParams(1524855988,1000,1524855988)),MUL,OneToOne,ScalarVectorBinaryOperation(SUB,ScalarFixedDoublePlan(0.0,RangeParams(1524855988,1000,1524855988)),PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(foo))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),true),List(),List(),List())""") + parseWithAntlr("sum(foo) < -1", """BinaryJoin(Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(foo))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),List(),None),LSS,OneToOne,ScalarBinaryOperation(SUB,Left(0.0),Left(1.0),RangeParams(1524855988,1000,1524855988)),List(),List(),List())""") + parseWithAntlr("-sum(foo) < -1", "BinaryJoin(ScalarVectorBinaryOperation(SUB,ScalarFixedDoublePlan(0.0,RangeParams(1524855988,1000,1524855988)),Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(foo))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),List(),None),true),LSS,OneToOne,ScalarBinaryOperation(SUB,Left(0.0),Left(1.0),RangeParams(1524855988,1000,1524855988)),List(),List(),List())") + parseWithAntlr("sum(-foo) < -1", """BinaryJoin(Aggregate(Sum,ScalarVectorBinaryOperation(SUB,ScalarFixedDoublePlan(0.0,RangeParams(1524855988,1000,1524855988)),PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(foo))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),true),List(),None),LSS,OneToOne,ScalarBinaryOperation(SUB,Left(0.0),Left(1.0),RangeParams(1524855988,1000,1524855988)),List(),List(),List())""") + parseWithAntlr("-sum(-foo) < -1", """BinaryJoin(ScalarVectorBinaryOperation(SUB,ScalarFixedDoublePlan(0.0,RangeParams(1524855988,1000,1524855988)),Aggregate(Sum,ScalarVectorBinaryOperation(SUB,ScalarFixedDoublePlan(0.0,RangeParams(1524855988,1000,1524855988)),PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(foo))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),true),List(),None),true),LSS,OneToOne,ScalarBinaryOperation(SUB,Left(0.0),Left(1.0),RangeParams(1524855988,1000,1524855988)),List(),List(),List())""") + + parseWithAntlr("+1", """ScalarBinaryOperation(ADD,Left(0.0),Left(1.0),RangeParams(1524855988,1000,1524855988))""") + parseWithAntlr("+foo", """ScalarVectorBinaryOperation(ADD,ScalarFixedDoublePlan(0.0,RangeParams(1524855988,1000,1524855988)),PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(foo))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),true)""") + parseWithAntlr("foo * +1", """BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(foo))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),MUL,OneToOne,ScalarBinaryOperation(ADD,Left(0.0),Left(1.0),RangeParams(1524855988,1000,1524855988)),List(),List(),List())""") + parseWithAntlr("+1 * foo", """BinaryJoin(ScalarBinaryOperation(ADD,Left(0.0),Left(1.0),RangeParams(1524855988,1000,1524855988)),MUL,OneToOne,PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(foo))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),List(),List(),List())""") + parseWithAntlr("+1 * +foo", """BinaryJoin(ScalarBinaryOperation(ADD,Left(0.0),Left(1.0),RangeParams(1524855988,1000,1524855988)),MUL,OneToOne,ScalarVectorBinaryOperation(ADD,ScalarFixedDoublePlan(0.0,RangeParams(1524855988,1000,1524855988)),PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(foo))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),true),List(),List(),List())""") + parseWithAntlr("sum(foo) < +1", """BinaryJoin(Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(foo))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),List(),None),LSS,OneToOne,ScalarBinaryOperation(ADD,Left(0.0),Left(1.0),RangeParams(1524855988,1000,1524855988)),List(),List(),List())""") + parseWithAntlr("+sum(foo) < +1", "BinaryJoin(ScalarVectorBinaryOperation(ADD,ScalarFixedDoublePlan(0.0,RangeParams(1524855988,1000,1524855988)),Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(foo))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),List(),None),true),LSS,OneToOne,ScalarBinaryOperation(ADD,Left(0.0),Left(1.0),RangeParams(1524855988,1000,1524855988)),List(),List(),List())") + parseWithAntlr("sum(+foo) < +1", """BinaryJoin(Aggregate(Sum,ScalarVectorBinaryOperation(ADD,ScalarFixedDoublePlan(0.0,RangeParams(1524855988,1000,1524855988)),PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(foo))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),true),List(),None),LSS,OneToOne,ScalarBinaryOperation(ADD,Left(0.0),Left(1.0),RangeParams(1524855988,1000,1524855988)),List(),List(),List())""") + parseWithAntlr("+sum(+foo) < +1", """BinaryJoin(ScalarVectorBinaryOperation(ADD,ScalarFixedDoublePlan(0.0,RangeParams(1524855988,1000,1524855988)),Aggregate(Sum,ScalarVectorBinaryOperation(ADD,ScalarFixedDoublePlan(0.0,RangeParams(1524855988,1000,1524855988)),PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(foo))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),true),List(),None),true),LSS,OneToOne,ScalarBinaryOperation(ADD,Left(0.0),Left(1.0),RangeParams(1524855988,1000,1524855988)),List(),List(),List())""") + + parseWithAntlr("+-1", """ScalarVectorBinaryOperation(ADD,ScalarFixedDoublePlan(0.0,RangeParams(1524855988,1000,1524855988)),ScalarBinaryOperation(SUB,Left(0.0),Left(1.0),RangeParams(1524855988,1000,1524855988)),true)""") + } + it("Should be able to make logical plans for Series Expressions") { val queryToLpString = Map( "http_requests_total + time()" -> "ScalarVectorBinaryOperation(ADD,ScalarTimeBasedPlan(Time,RangeParams(1524855988,1000,1524855988)),PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),false)", From d84c6c8aecb7e7da4978c42e8933a36a9e588dc8 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Wed, 9 Aug 2023 16:02:52 -0700 Subject: [PATCH 21/39] fix(core): Bug in calculating size of SerializedRangeVector (#1643) There was a bug in calculating size of SRV. Earlier, for efficiency purposes, we were calculating the size of the containers associated with the SRV. But actually, the container can home multiple SRVs. So the calculated size for several SRVs at a time can end up wrong with addition of cumulative counts. The fix for now is to calculate the size by going through the records. It introduces a small inefficiency here, but submitting this PR for now since other ways to calculate this were more invasive and risk regression. We can have an optimization of this if really needed later. I have also reduced the number of calls to this method from two to one. The unit tests didn't catch this since earlier since they played with one SRV only. I have now added a unit test that adds multiple SRVs. It failed with earlier code. --- .../binaryrecord2/RecordContainer.scala | 6 ++--- .../binaryrecord2/RecordSchema.scala | 7 ++++- .../memstore/TimeSeriesShard.scala | 10 ++----- .../scala/filodb.core/query/RangeVector.scala | 15 +++++------ .../query/SerializedRangeVectorSpec.scala | 27 ++++++++++++++++--- .../scala/filodb/query/exec/ExecPlan.scala | 10 +++++-- .../aggregator/TopBottomKRowAggregator.scala | 4 +-- .../exec/MultiSchemaPartitionsExecSpec.scala | 5 ++++ .../query/exec/PromQLGrpcRemoteExecSpec.scala | 2 +- 9 files changed, 57 insertions(+), 29 deletions(-) diff --git a/core/src/main/scala/filodb.core/binaryrecord2/RecordContainer.scala b/core/src/main/scala/filodb.core/binaryrecord2/RecordContainer.scala index 86f96146fd..c2da9e672f 100644 --- a/core/src/main/scala/filodb.core/binaryrecord2/RecordContainer.scala +++ b/core/src/main/scala/filodb.core/binaryrecord2/RecordContainer.scala @@ -7,7 +7,7 @@ import scala.reflect.ClassTag import debox.Buffer import filodb.memory.{BinaryRegionConsumer, BinaryRegionLarge} -import filodb.memory.format.{RowReader, UnsafeUtils} +import filodb.memory.format.UnsafeUtils /** * A RecordContainer is a binary, wire-compatible container for BinaryRecords V2. @@ -72,13 +72,13 @@ final class RecordContainer(val base: Any, val offset: Long, maxLength: Int, * Iterates through each BinaryRecord as a RowReader. Results in two allocations: the Iterator * as well as a BinaryRecordRowReader. */ - final def iterate(schema: RecordSchema): Iterator[RowReader] = new Iterator[RowReader] { + final def iterate(schema: RecordSchema): Iterator[BinaryRecordRowReader] = new Iterator[BinaryRecordRowReader] { val reader = new BinaryRecordRowReader(schema, base) val endOffset = offset + 4 + numBytes var curOffset = offset + ContainerHeaderLen final def hasNext: Boolean = curOffset < endOffset - final def next: RowReader = { + final def next: BinaryRecordRowReader = { val recordLen = BinaryRegionLarge.numBytes(base, curOffset) reader.recordOffset = curOffset curOffset += (recordLen + 7) & ~3 // +4, then aligned/rounded up to next 4 bytes diff --git a/core/src/main/scala/filodb.core/binaryrecord2/RecordSchema.scala b/core/src/main/scala/filodb.core/binaryrecord2/RecordSchema.scala index 42dddfd2ca..01a1252f7c 100644 --- a/core/src/main/scala/filodb.core/binaryrecord2/RecordSchema.scala +++ b/core/src/main/scala/filodb.core/binaryrecord2/RecordSchema.scala @@ -624,7 +624,12 @@ trait BinaryRecordRowReaderBase extends RowReader { final class BinaryRecordRowReader(val schema: RecordSchema, var recordBase: Any = UnsafeUtils.ZeroPointer, - var recordOffset: Long = 0L) extends BinaryRecordRowReaderBase + var recordOffset: Long = 0L) extends BinaryRecordRowReaderBase { + def recordLength: Int = { + val len = BinaryRegionLarge.numBytes(recordBase, recordOffset) + (len + 7) & ~3 // +4, then aligned/rounded up to next 4 bytes + } +} final class MultiSchemaBRRowReader(var recordBase: Any = UnsafeUtils.ZeroPointer, var recordOffset: Long = 0L) extends BinaryRecordRowReaderBase { diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala index f11c957d33..045712758e 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala @@ -934,14 +934,8 @@ class TimeSeriesShard(val ref: DatasetRef, markPartAsNotIngesting(p, odp = false) if (storeConfig.meteringEnabled) { val shardKey = p.schema.partKeySchema.colValues(p.partKeyBase, p.partKeyOffset, - p.schema.options.shardKeyColumns) - val newCard = cardTracker.modifyCount(shardKey, 0, -1) - // TODO remove temporary debugging since we are seeing some negative counts - if (newCard.exists(_.value.activeTsCount < 0) && p.partID % 100 < 5) - // log for 5% of the cases to reduce log volume - logger.error(s"For some reason, activeTs count negative when updating card for " + - s"partKey: ${p.stringPartition} newCard: $newCard oldActivelyIngestingSize=$oldActivelyIngestingSize " + - s"newActivelyIngestingSize=${activelyIngesting.size}") + p.schema.options.shardKeyColumns) + cardTracker.modifyCount(shardKey, 0, -1) } } } diff --git a/core/src/main/scala/filodb.core/query/RangeVector.scala b/core/src/main/scala/filodb.core/query/RangeVector.scala index 7e456f1aa8..b338accf1c 100644 --- a/core/src/main/scala/filodb.core/query/RangeVector.scala +++ b/core/src/main/scala/filodb.core/query/RangeVector.scala @@ -213,8 +213,8 @@ sealed trait ScalarSingleValue extends ScalarRangeVector { else it } - // Negligible bytes sent over-the-wire. - override def estimateSerializedRowBytes: Long = 0 + // Negligible bytes sent over-the-wire. Don't bother calculating accurately. + override def estimateSerializedRowBytes: Long = SerializableRangeVector.SizeOfDouble } /** @@ -399,7 +399,10 @@ final class SerializedRangeVector(val key: RangeVectorKey, } else it } - override def estimateSerializedRowBytes: Long = containers.map(_.numBytes).sum + override def estimateSerializedRowBytes: Long = + containers.toIterator.flatMap(_.iterate(schema)) + .slice(startRecordNo, startRecordNo + numRowsSerialized) + .foldLeft(0)(_ + _.recordLength) def containersIterator : Iterator[RecordContainer] = containers.toIterator @@ -487,11 +490,7 @@ object SerializedRangeVector extends StrictLogging { case None => builder.allContainers.toList case Some(firstContainer) => builder.allContainers.dropWhile(_ != firstContainer) } - val srv = new SerializedRangeVector(rv.key, numRows, containers, schema, startRecordNo, rv.outputRange) - val resultSize = srv.estimatedSerializedBytes - SerializedRangeVector.queryResultBytes.record(resultSize) - queryStats.getResultBytesCounter(Nil).addAndGet(resultSize) - srv + new SerializedRangeVector(rv.key, numRows, containers, schema, startRecordNo, rv.outputRange) } finally { queryStats.getCpuNanosCounter(Nil).addAndGet(Utils.currentThreadCpuTimeNanos - startNs) } diff --git a/core/src/test/scala/filodb.core/query/SerializedRangeVectorSpec.scala b/core/src/test/scala/filodb.core/query/SerializedRangeVectorSpec.scala index 46915cc20b..7ab7aa8e6e 100644 --- a/core/src/test/scala/filodb.core/query/SerializedRangeVectorSpec.scala +++ b/core/src/test/scala/filodb.core/query/SerializedRangeVectorSpec.scala @@ -51,9 +51,9 @@ class SerializedRangeVectorSpec extends AnyFunSpec with Matchers { val queryStats = QueryStats() val srv = SerializedRangeVector.apply(rv, builder, recSchema, "someExecPlan", queryStats) queryStats.getCpuNanosCounter(Nil).get() > 0 shouldEqual true - queryStats.getResultBytesCounter(Nil).get() shouldEqual 108 srv.numRows shouldEqual Some(11) srv.numRowsSerialized shouldEqual 4 + srv.estimateSerializedRowBytes shouldEqual 80 // 4 non nan records each of 20 bytes val res = srv.rows.map(r => (r.getLong(0), r.getDouble(1))).toList res.length shouldEqual 11 res.map(_._1) shouldEqual (0 to 1000 by 100) @@ -77,9 +77,9 @@ class SerializedRangeVectorSpec extends AnyFunSpec with Matchers { val queryStats = QueryStats() val srv = SerializedRangeVector.apply(rv, builder, recSchema, "someExecPlan", queryStats) queryStats.getCpuNanosCounter(Nil).get() > 0 shouldEqual true - queryStats.getResultBytesCounter(Nil).get() shouldEqual 248 srv.numRows shouldEqual Some(11) srv.numRowsSerialized shouldEqual 11 + srv.estimateSerializedRowBytes shouldEqual 220 val res = srv.rows.map(r => (r.getLong(0), r.getDouble(1))).toList res.length shouldEqual 11 res.map(_._1) shouldEqual (0 to 1000 by 100) @@ -105,7 +105,6 @@ class SerializedRangeVectorSpec extends AnyFunSpec with Matchers { val queryStats = QueryStats() val srv = SerializedRangeVector.apply(rv, builder, recSchema, "someExecPlan", queryStats) queryStats.getCpuNanosCounter(Nil).get() > 0 shouldEqual true - queryStats.getResultBytesCounter(Nil).get() shouldEqual 188 srv.numRows shouldEqual Some(11) srv.numRowsSerialized shouldEqual 4 val res = srv.rows.map(r => (r.getLong(0), r.getHistogram(1))).toList @@ -114,4 +113,26 @@ class SerializedRangeVectorSpec extends AnyFunSpec with Matchers { res.map(_._2).filterNot(_.isEmpty) shouldEqual Seq(h1, h1, h1, h1) } + it("should calculate estimateSerializedRowBytes correctly when builder is used for several SRVs") { + val builder = SerializedRangeVector.newBuilder() + val recSchema = new RecordSchema(Seq(ColumnInfo("time", ColumnType.TimestampColumn), + ColumnInfo("value", ColumnType.DoubleColumn))) + val keysMap = Map(UTF8Str("key1") -> UTF8Str("val1"), + UTF8Str("key2") -> UTF8Str("val2")) + val key = CustomRangeVectorKey(keysMap) + + (0 to 200).foreach { i => + val rv = toRv(Seq((0, Double.NaN), (100, 1.0), (200, Double.NaN), + (300, 3.0), (400, Double.NaN), + (500, 5.0), (600, 6.0), + (700, Double.NaN), (800, Double.NaN), + (900, Double.NaN), (1000, Double.NaN)), key, + RvRange(1000, 100, 1000)) + val queryStats = QueryStats() + val srv = SerializedRangeVector.apply(rv, builder, recSchema, "someExecPlan", queryStats) + srv.numRowsSerialized shouldEqual 11 + srv.estimateSerializedRowBytes shouldEqual 220 + } + } + } diff --git a/query/src/main/scala/filodb/query/exec/ExecPlan.scala b/query/src/main/scala/filodb/query/exec/ExecPlan.scala index 3655f0d153..a75ffb89ab 100644 --- a/query/src/main/scala/filodb/query/exec/ExecPlan.scala +++ b/query/src/main/scala/filodb/query/exec/ExecPlan.scala @@ -251,7 +251,9 @@ trait ExecPlan extends QueryCommand { // fail the query instead of limiting range vectors and returning incomplete/inaccurate results numResultSamples += srv.numRowsSerialized checkSamplesLimit(numResultSamples, querySession.warnings) - resultSize += srv.estimatedSerializedBytes + val srvBytes = srv.estimatedSerializedBytes + resultSize += srvBytes + querySession.queryStats.getResultBytesCounter(Nil).addAndGet(srvBytes) checkResultBytes(resultSize, querySession.queryConfig, querySession.warnings) srv } @@ -263,6 +265,7 @@ trait ExecPlan extends QueryCommand { MeasurementUnit.time.milliseconds) .withTag("plan", getClass.getSimpleName) .record(Math.max(0, System.currentTimeMillis - startExecute)) + SerializedRangeVector.queryResultBytes.record(resultSize) // recording and adding step1 to queryStats at the end of execution since the grouping // for stats is not formed yet at the beginning querySession.queryStats.getCpuNanosCounter(Nil).getAndAdd(step1CpuTime) @@ -461,7 +464,9 @@ trait ExecPlan extends QueryCommand { // fail the query instead of limiting range vectors and returning incomplete/inaccurate results numResultSamples += srv.numRowsSerialized checkSamplesLimit(numResultSamples, querySession.warnings) - resultSize += srv.estimatedSerializedBytes + val srvBytes = srv.estimatedSerializedBytes + resultSize += srvBytes + querySession.queryStats.getResultBytesCounter(Nil).addAndGet(srvBytes) checkResultBytes(resultSize, querySession.queryConfig, querySession.warnings) srv } @@ -469,6 +474,7 @@ trait ExecPlan extends QueryCommand { .guarantee(Task.eval(span.mark("after-last-materialized-result-rv"))) .toListL .map { r => + SerializedRangeVector.queryResultBytes.record(resultSize) Kamon.histogram("query-execute-time-elapsed-step2-result-materialized", MeasurementUnit.time.milliseconds) .withTag("plan", getClass.getSimpleName) diff --git a/query/src/main/scala/filodb/query/exec/aggregator/TopBottomKRowAggregator.scala b/query/src/main/scala/filodb/query/exec/aggregator/TopBottomKRowAggregator.scala index de4479e8d1..726692fafc 100644 --- a/query/src/main/scala/filodb/query/exec/aggregator/TopBottomKRowAggregator.scala +++ b/query/src/main/scala/filodb/query/exec/aggregator/TopBottomKRowAggregator.scala @@ -158,12 +158,10 @@ class TopBottomKRowAggregator(k: Int, bottomK: Boolean) extends RowAggregator wi resRvs.map { case (key, builder) => val numRows = builder.allContainers.map(_.countRecords()).sum logger.debug(s"TopkPresent before creating SRV key = ${key.labelValues.mkString(",")}") - val srv = new SerializedRangeVector(key, numRows, builder.allContainers, recSchema, 0, + new SerializedRangeVector(key, numRows, builder.allContainers, recSchema, 0, Some(RvRange(rangeParams.startSecs * 1000, rangeParams.stepSecs * 1000, rangeParams.endSecs * 1000))) - queryStats.getResultBytesCounter(Nil).getAndAdd(srv.estimatedSerializedBytes) - srv }.toSeq } finally { queryStats.getCpuNanosCounter(Nil).getAndAdd(Utils.currentThreadCpuTimeNanos - startNs) diff --git a/query/src/test/scala/filodb/query/exec/MultiSchemaPartitionsExecSpec.scala b/query/src/test/scala/filodb/query/exec/MultiSchemaPartitionsExecSpec.scala index 3b9e461acb..092c309583 100644 --- a/query/src/test/scala/filodb/query/exec/MultiSchemaPartitionsExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/MultiSchemaPartitionsExecSpec.scala @@ -138,6 +138,7 @@ class MultiSchemaPartitionsExecSpec extends AnyFunSpec with Matchers with ScalaF val execPlan = MultiSchemaPartitionsExec(QueryContext(), dummyDispatcher, dsRef, 0, filters, TimeRangeChunkScan(startTime, endTime), "_metric_") + querySession.queryStats.clear() // so this can be run as a standalone test val resp = execPlan.execute(memStore, querySession).runToFuture.futureValue val result = resp.asInstanceOf[QueryResult] result.result.size shouldEqual 1 @@ -145,6 +146,10 @@ class MultiSchemaPartitionsExecSpec extends AnyFunSpec with Matchers with ScalaF dataRead shouldEqual tuples.take(11) val partKeyRead = result.result(0).key.labelValues.map(lv => (lv._1.asNewString, lv._2.asNewString)) partKeyRead shouldEqual partKeyKVWithMetric + querySession.queryStats.getResultBytesCounter().get() shouldEqual 297 + querySession.queryStats.getCpuNanosCounter().get() > 0 shouldEqual true + querySession.queryStats.getDataBytesScannedCounter().get() shouldEqual 48 + querySession.queryStats.getTimeSeriesScannedCounter().get() shouldEqual 1 } it("should get empty schema if query returns no results") { diff --git a/query/src/test/scala/filodb/query/exec/PromQLGrpcRemoteExecSpec.scala b/query/src/test/scala/filodb/query/exec/PromQLGrpcRemoteExecSpec.scala index 217cae5fd0..5ecf7fea2b 100644 --- a/query/src/test/scala/filodb/query/exec/PromQLGrpcRemoteExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/PromQLGrpcRemoteExecSpec.scala @@ -157,7 +157,7 @@ class PromQLGrpcRemoteExecSpec extends AnyFunSpec with Matchers with ScalaFuture deserializedSrv.numRowsSerialized shouldEqual 4 val res = deserializedSrv.rows.map(r => (r.getLong(0), r.getDouble(1))).toList deserializedSrv.key shouldEqual rvKey - qr.queryStats.getResultBytesCounter(List()).get()shouldEqual 108 + // queryStats ResultBytes counter increment is not done as part of SRV constructor, so skipping that assertion (qr.queryStats.getCpuNanosCounter(List()).get() > 0) shouldEqual true res.length shouldEqual 11 res.map(_._1) shouldEqual (0 to 1000 by 100) From 38c682c3dbee409b4479304452704d47724df21d Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Wed, 9 Aug 2023 16:12:46 -0700 Subject: [PATCH 22/39] perf(core): ~Two times throughput improvement for Lucene queries with enum and prefix regex filters (#1641) Production profiling is showing that Lucene Regex automata is creating a hotspot in method and allocation profiling. This PR optimizes two kinds of queries * Regex with enumerated values are converted to TermInSetQuery * Regex with prefix is converted to PrefixQuery It also wraps Lucene queries in ConstantScoreQuery to prevent any scoring that may be happening. Observed 2.2x performance improvement in JMH benchmark for specific enum regex query 1.5x performance improvement in JMH benchmark for specific prefix regex query --- .../memstore/PartKeyLuceneIndex.scala | 52 +++++++++++++------ .../memstore/PartKeyLuceneIndexSpec.scala | 28 +++++++++- .../filodb.jmh/PartKeyIndexBenchmark.scala | 27 ++++++++-- run_benchmarks.sh | 3 +- 4 files changed, 86 insertions(+), 24 deletions(-) diff --git a/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala b/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala index 26cea14f83..aa1b66dc72 100644 --- a/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala +++ b/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala @@ -777,7 +777,6 @@ class PartKeyLuceneIndex(ref: DatasetRef, coll.numHits } - def foreachPartKeyMatchingFilter(columnFilters: Seq[ColumnFilter], startTime: Long, endTime: Long, func: (BytesRef) => Unit): Int = { @@ -817,16 +816,40 @@ class PartKeyLuceneIndex(ref: DatasetRef, logger.info(s"Refreshed index searchers to make reads consistent for dataset=$ref shard=$shardNum") } - //scalastyle:off method.length + val regexChars = Array('.', '?', '+', '*', '|', '{', '}', '[', ']', '(', ')', '"', '\\') + val regexCharsMinusPipe = (regexChars.toSet - '|').toArray + + // scalastyle:off method.length private def leafFilter(column: String, filter: Filter): Query = { + + def equalsQuery(value: String): Query = { + if (value.nonEmpty) new TermQuery(new Term(column, value)) + else leafFilter(column, NotEqualsRegex(".+")) // value="" means the label is absent or has an empty value. + } + filter match { case EqualsRegex(value) => val regex = removeRegexAnchors(value.toString) - if (regex.replaceAll("\\.\\*", "") == "") + if (regex == "") { + // if label=~"" then match empty string or label not present condition too + leafFilter(column, NotEqualsRegex(".+")) + } else if (regex.replaceAll("\\.\\*", "") == "") { + // if label=~".*" then match all docs since promQL matches .* with absent label too new MatchAllDocsQuery - else if (regex.nonEmpty) + } else if (regex.forall(c => !regexChars.contains(c))) { + // if all regex special chars absent, then treat like Equals + equalsQuery(regex) + } else if (regex.forall(c => !regexCharsMinusPipe.contains(c))) { + // if pipe is only regex special char present, then convert to IN query + new TermInSetQuery(column, regex.split('|').map(t => new BytesRef(t)): _*) + } else if (regex.endsWith(".*") && regex.length > 2 && + regex.dropRight(2).forall(c => !regexChars.contains(c))) { + // if suffix is .* and no regex special chars present in non-empty prefix, then use prefix query + new PrefixQuery(new Term(column, regex.dropRight(2))) + } else { + // regular non-empty regex query new RegexpQuery(new Term(column, regex), RegExp.NONE) - else leafFilter(column, NotEqualsRegex(".+")) // value="" means the label is absent or has an empty value. + } case NotEqualsRegex(value) => val term = new Term(column, removeRegexAnchors(value.toString)) @@ -835,9 +858,10 @@ class PartKeyLuceneIndex(ref: DatasetRef, booleanQuery.add(allDocs, Occur.FILTER) booleanQuery.add(new RegexpQuery(term, RegExp.NONE), Occur.MUST_NOT) booleanQuery.build() + case Equals(value) => - if (value.toString.nonEmpty) new TermQuery(new Term(column, value.toString)) - else leafFilter(column, NotEqualsRegex(".+")) // value="" means the label is absent or has an empty value. + equalsQuery(value.toString) + case NotEquals(value) => val str = value.toString val term = new Term(column, str) @@ -854,18 +878,14 @@ class PartKeyLuceneIndex(ref: DatasetRef, booleanQuery.build() case In(values) => - if (values.size < 2) - throw new IllegalArgumentException("In filter should have atleast 2 values") - val booleanQuery = new BooleanQuery.Builder - values.foreach { value => - booleanQuery.add(new TermQuery(new Term(column, value.toString)), Occur.SHOULD) - } - booleanQuery.build() + new TermInSetQuery(column, values.toArray.map(t => new BytesRef(t.toString)): _*) + case And(lhs, rhs) => val andQuery = new BooleanQuery.Builder andQuery.add(leafFilter(column, lhs), Occur.FILTER) andQuery.add(leafFilter(column, rhs), Occur.FILTER) andQuery.build() + case _ => throw new UnsupportedOperationException } } @@ -915,7 +935,7 @@ class PartKeyLuceneIndex(ref: DatasetRef, val startExecute = System.nanoTime() val span = Kamon.currentSpan() - val query: BooleanQuery = colFiltersToQuery(columnFilters, startTime, endTime) + val query = colFiltersToQuery(columnFilters, startTime, endTime) logger.debug(s"Querying dataset=$ref shard=$shardNum partKeyIndex with: $query") withNewSearcher(s => s.search(query, collector)) val latency = System.nanoTime - startExecute @@ -932,7 +952,7 @@ class PartKeyLuceneIndex(ref: DatasetRef, booleanQuery.add(LongPoint.newRangeQuery(START_TIME, 0, endTime), Occur.FILTER) booleanQuery.add(LongPoint.newRangeQuery(END_TIME, startTime, Long.MaxValue), Occur.FILTER) val query = booleanQuery.build() - query + new ConstantScoreQuery(query) // disable scoring } def partIdFromPartKeySlow(partKeyBase: Any, diff --git a/core/src/test/scala/filodb.core/memstore/PartKeyLuceneIndexSpec.scala b/core/src/test/scala/filodb.core/memstore/PartKeyLuceneIndexSpec.scala index 9c9854f3e2..e776700b3f 100644 --- a/core/src/test/scala/filodb.core/memstore/PartKeyLuceneIndexSpec.scala +++ b/core/src/test/scala/filodb.core/memstore/PartKeyLuceneIndexSpec.scala @@ -282,7 +282,6 @@ class PartKeyLuceneIndexSpec extends AnyFunSpec with Matchers with BeforeAndAfte } - it("should add part keys and removePartKeys (without partIds) correctly for more than 1024 keys with del count") { // Identical to test // it("should add part keys and fetch partIdsEndedBefore and removePartKeys correctly for more than 1024 keys") @@ -345,7 +344,6 @@ class PartKeyLuceneIndexSpec extends AnyFunSpec with Matchers with BeforeAndAfte // it is no longer required to have partIds in the index on non unit test setup } - it("should update part keys with endtime and parse filters correctly") { val start = System.currentTimeMillis() // Add the first ten keys and row numbers @@ -447,6 +445,32 @@ class PartKeyLuceneIndexSpec extends AnyFunSpec with Matchers with BeforeAndAfte partNums2 shouldEqual debox.Buffer.empty[Int] } + it("should be able to convert pipe regex to TermInSetQuery") { + // Add the first ten keys and row numbers + partKeyFromRecords(dataset6, records(dataset6, readers.take(99)), Some(partBuilder)) + .zipWithIndex.foreach { case (addr, i) => + keyIndex.addPartKey(partKeyOnHeap(dataset6.partKeySchema, ZeroPointer, addr), i, System.currentTimeMillis())() + } + keyIndex.refreshReadersBlocking() + + val filters1 = Seq(ColumnFilter("Actor2Code", EqualsRegex("GOV|KHM|LAB|MED".utf8))) + val partNums1 = keyIndex.partIdsFromFilters(filters1, 0, Long.MaxValue) + partNums1 shouldEqual debox.Buffer(7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 22, 23, 24, 25, 26, 28, 29, 73, 81, 90) + } + + it("should be able to convert prefix regex to PrefixQuery") { + // Add the first ten keys and row numbers + partKeyFromRecords(dataset6, records(dataset6, readers.take(99)), Some(partBuilder)) + .zipWithIndex.foreach { case (addr, i) => + keyIndex.addPartKey(partKeyOnHeap(dataset6.partKeySchema, ZeroPointer, addr), i, System.currentTimeMillis())() + } + keyIndex.refreshReadersBlocking() + + val filters1 = Seq(ColumnFilter("Actor2Name", EqualsRegex("C.*".utf8))) + val partNums1 = keyIndex.partIdsFromFilters(filters1, 0, Long.MaxValue) + partNums1 shouldEqual debox.Buffer(3, 12, 22, 23, 24, 31, 59, 60, 66, 69, 72, 78, 79, 80, 88, 89) + } + it("should ignore unsupported columns and return empty filter") { val index2 = new PartKeyLuceneIndex(dataset1.ref, dataset1.schema.partition, true, true, 0, 1.hour.toMillis) partKeyFromRecords(dataset1, records(dataset1, readers.take(10))).zipWithIndex.foreach { case (addr, i) => diff --git a/jmh/src/main/scala/filodb.jmh/PartKeyIndexBenchmark.scala b/jmh/src/main/scala/filodb.jmh/PartKeyIndexBenchmark.scala index 5fd16ab097..41b036bf0c 100644 --- a/jmh/src/main/scala/filodb.jmh/PartKeyIndexBenchmark.scala +++ b/jmh/src/main/scala/filodb.jmh/PartKeyIndexBenchmark.scala @@ -74,7 +74,7 @@ class PartKeyIndexBenchmark { partKeyIndex.partIdsFromFilters( Seq(ColumnFilter("_ns_", Filter.Equals(s"App-$i")), ColumnFilter("_ws_", Filter.Equals("demo")), - ColumnFilter("host", Filter.EqualsRegex("H0")), + ColumnFilter("host", Filter.Equals("H0")), ColumnFilter("_metric_", Filter.Equals("heap_usage0"))), now, now + 1000) @@ -90,7 +90,7 @@ class PartKeyIndexBenchmark { partKeyIndex.partIdsFromFilters( Seq(ColumnFilter("_ns_", Filter.Equals(s"App-${i + 200}")), ColumnFilter("_ws_", Filter.Equals("demo")), - ColumnFilter("host", Filter.EqualsRegex("H0")), + ColumnFilter("host", Filter.Equals("H0")), ColumnFilter("_metric_", Filter.Equals("heap_usage0"))), now, now + 1000) @@ -101,7 +101,7 @@ class PartKeyIndexBenchmark { @BenchmarkMode(Array(Mode.Throughput)) @OutputTimeUnit(TimeUnit.SECONDS) @OperationsPerInvocation(8) - def partIdsLookupWithSuffixRegexFilters(): Unit = { + def partIdsLookupWithPrefixRegexFilters(): Unit = { cforRange ( 0 until 8 ) { i => partKeyIndex.partIdsFromFilters( Seq(ColumnFilter("_ns_", Filter.Equals(s"App-$i")), @@ -117,7 +117,7 @@ class PartKeyIndexBenchmark { @BenchmarkMode(Array(Mode.Throughput)) @OutputTimeUnit(TimeUnit.SECONDS) @OperationsPerInvocation(8) - def partIdsLookupWithPrefixRegexFilters(): Unit = { + def partIdsLookupWithSuffixRegexFilters(): Unit = { cforRange ( 0 until 8 ) { i => partKeyIndex.partIdsFromFilters( Seq(ColumnFilter("_ns_", Filter.Equals(s"App-$i")), @@ -129,6 +129,25 @@ class PartKeyIndexBenchmark { } } + @Benchmark + @BenchmarkMode(Array(Mode.Throughput)) + @OutputTimeUnit(TimeUnit.SECONDS) + @OperationsPerInvocation(8) + def partIdsLookupWithEnumRegexFilter(): Unit = { + cforRange(0 until 8) { i => + val c = partKeyIndex.partIdsFromFilters( + Seq(ColumnFilter("_ns_", Filter.Equals(s"App-0")), + ColumnFilter("_ws_", Filter.Equals("demo")), + ColumnFilter("_metric_", Filter.Equals("heap_usage0")), + ColumnFilter("instance", + Filter.EqualsRegex("Instance-1|Instance-2|Instance-3|Instance-4|Instance-5|Instance-6|Instance-7|Instance-8|Instance-9|Instance-10|" + + "Instance-11|Instance-12|Instance-13|Instance-14|Instance-15|Instance-16|Instance-17|Instance-18|Instance-19|Instance-20|" + + "Instance-21|Instance-22|Instance-23|Instance-24|Instance-25|Instance-26|Instance-27|Instance-28|Instance-29|Instance-30"))), + now, + now + 1000).length + } + } + @Benchmark @BenchmarkMode(Array(Mode.Throughput)) @OutputTimeUnit(TimeUnit.SECONDS) diff --git a/run_benchmarks.sh b/run_benchmarks.sh index d368e26d59..0572870994 100755 --- a/run_benchmarks.sh +++ b/run_benchmarks.sh @@ -1,7 +1,6 @@ #!/bin/bash -sbt "jmh/jmh:run -rf json -i 15 -wi 10 -f3 -jvmArgsAppend -XX:MaxInlineLevel=20 \ +sbt "jmh/jmh:run -rf json -i 5 -wi 3 -f 1 -jvmArgsAppend -XX:MaxInlineLevel=20 \ -jvmArgsAppend -Xmx4g -jvmArgsAppend -XX:MaxInlineSize=99 \ - -prof jfr:dir=/tmp/filo-jmh \ filodb.jmh.QueryHiCardInMemoryBenchmark \ filodb.jmh.QueryInMemoryBenchmark \ filodb.jmh.QueryAndIngestBenchmark \ From f3352e2ea0dc5b87131943e88c74daef55740d69 Mon Sep 17 00:00:00 2001 From: yu-shipit <120689245+yu-shipit@users.noreply.github.com> Date: Mon, 14 Aug 2023 17:58:19 -0700 Subject: [PATCH 23/39] cherry-pick histogram debugging message and cardinality job nullptr fix (#1645) * fix(core) make the error message more frendly to users. (#1593) Co-authored-by: Yu Zhang (cherry picked from commit 5b05779aa36475bb5c91c7a71ae092795ae79b06) * fix nullpointer happened in cardinality busting job. (#1631) skip busting the problematic entry. Co-authored-by: Yu Zhang (cherry picked from commit 6ac025558a1f8ae7e879227434b9e5014d28b3a8) --- .../PerShardCardinalityBuster.scala | 32 +++++++++++-------- 1 file changed, 19 insertions(+), 13 deletions(-) diff --git a/spark-jobs/src/main/scala/filodb/cardbuster/PerShardCardinalityBuster.scala b/spark-jobs/src/main/scala/filodb/cardbuster/PerShardCardinalityBuster.scala index f61c8ba822..2a9733350a 100644 --- a/spark-jobs/src/main/scala/filodb/cardbuster/PerShardCardinalityBuster.scala +++ b/spark-jobs/src/main/scala/filodb/cardbuster/PerShardCardinalityBuster.scala @@ -83,22 +83,28 @@ class PerShardCardinalityBuster(dsSettings: DownsamplerSettings, val numCouldNotDelete = AtomicInt(0) val numCandidateKeys = AtomicInt(0) val keysToDelete = candidateKeys.filter { pk => - val rawSchemaId = RecordSchema.schemaID(pk.partKey, UnsafeUtils.arayOffset) - val schema = schemas(rawSchemaId) - val pkPairs = schema.partKeySchema.toStringPairs(pk.partKey, UnsafeUtils.arayOffset) - val willDelete = deleteFilter.exists { filter => // at least one filter should match - filter.forall { case (filterKey, filterValRegex) => // should match all tags in this filter - pkPairs.exists { case (pkKey, pkVal) => - pkKey == filterKey && filterValRegex.matcher(pkVal).matches + try { + val rawSchemaId = RecordSchema.schemaID(pk.partKey, UnsafeUtils.arayOffset) + val schema = schemas(rawSchemaId) + val pkPairs = schema.partKeySchema.toStringPairs(pk.partKey, UnsafeUtils.arayOffset) + val willDelete = deleteFilter.exists { filter => // at least one filter should match + filter.forall { case (filterKey, filterValRegex) => // should match all tags in this filter + pkPairs.exists { case (pkKey, pkVal) => + pkKey == filterKey && filterValRegex.matcher(pkVal).matches + } } } + if (willDelete) { + BusterContext.log.debug(s"Deleting part key $pkPairs from shard=$shard startTime=${pk.startTime} " + + s"endTime=${pk.endTime} split=$split isSimulation=$isSimulation") + } + numCandidateKeys += 1 + willDelete + } catch { + case e : Exception => + BusterContext.log.warn(s"skip busting pk=$pk because of exception $e") + false } - if (willDelete) { - BusterContext.log.debug(s"Deleting part key $pkPairs from shard=$shard startTime=${pk.startTime} " + - s"endTime=${pk.endTime} split=$split isSimulation=$isSimulation") - } - numCandidateKeys += 1 - willDelete } val fut = Observable.fromIteratorUnsafe(keysToDelete) .mapParallelUnordered(numParallelDeletesPerSparkThread.getOrElse(1)) { pk => From bfbeb361bbf780135c7ad9df0a9a87344bf1004c Mon Sep 17 00:00:00 2001 From: yu-shipit <120689245+yu-shipit@users.noreply.github.com> Date: Tue, 15 Aug 2023 13:36:23 -0700 Subject: [PATCH 24/39] filodb(core) add debugging info for empty histogram. (#1613) (#1649) * filodb(core) add debugging info for empty histogram. Some queries occasionally hit exceptions because of empty histogram. However, the same exception could not be reproduced later. The hunch is that the bug is caused by a race condition. So, adding additional debugging log to print out the chunk id chunk info and the memory dump. --------- Co-authored-by: Yu Zhang (cherry picked from commit 90303aaff71dd435091148871861bb84d719c9ae) --- .../query/PartitionTimeRangeReader.scala | 22 +++++++++++++++++-- .../format/vectors/HistogramVector.scala | 5 +++++ 2 files changed, 25 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/filodb.core/query/PartitionTimeRangeReader.scala b/core/src/main/scala/filodb.core/query/PartitionTimeRangeReader.scala index e4b336c5da..a86ad2a56a 100644 --- a/core/src/main/scala/filodb.core/query/PartitionTimeRangeReader.scala +++ b/core/src/main/scala/filodb.core/query/PartitionTimeRangeReader.scala @@ -1,10 +1,12 @@ package filodb.core.query +import com.typesafe.scalalogging.StrictLogging import spire.syntax.cfor._ import filodb.core.metadata.Dataset import filodb.core.store.{ChunkInfoIterator, ChunkSetInfoReader, ReadablePartition} import filodb.memory.format.{vectors => bv, RowReader, TypedIterator, UnsafeUtils, ZeroCopyUTF8String} +import filodb.memory.format.vectors.EmptyHistogramException /** * A RowReader iterator which iterates over a time range in the ReadablePartition. Designed to be relatively memory @@ -16,7 +18,7 @@ final class PartitionTimeRangeReader(part: ReadablePartition, startTime: Long, endTime: Long, infos: ChunkInfoIterator, - columnIDs: Array[Int]) extends RangeVectorCursor { + columnIDs: Array[Int]) extends RangeVectorCursor with StrictLogging { // MinValue = no current chunk private var curChunkID = Long.MinValue private final val vectorIts = new Array[TypedIterator](columnIDs.size) @@ -33,7 +35,23 @@ final class PartitionTimeRangeReader(part: ReadablePartition, def getDouble(columnNo: Int): Double = vectorIts(columnNo).asDoubleIt.next def getFloat(columnNo: Int): Float = ??? def getString(columnNo: Int): String = ??? - override def getHistogram(columnNo: Int): bv.Histogram = vectorIts(columnNo).asHistIt.next + override def getHistogram(columnNo: Int): bv.Histogram = { + try { + vectorIts(columnNo).asHistIt.next + } catch { + case e : EmptyHistogramException => { + var message = s"EmptyHistogramException ${e.getMessage} infos=[" + while (infos.hasNext) { + val info = infos.nextInfo + message += + s"""${info.debugString} """ + } + message += "]" + logger.error(s"message ${message}") + throw new IllegalArgumentException(message) + } + } + } def getAny(columnNo: Int): Any = ??? override def filoUTF8String(columnNo: Int): ZeroCopyUTF8String = vectorIts(columnNo).asUTF8It.next 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 430117cd5f..2a6f230e86 100644 --- a/memory/src/main/scala/filodb.memory/format/vectors/HistogramVector.scala +++ b/memory/src/main/scala/filodb.memory/format/vectors/HistogramVector.scala @@ -556,6 +556,7 @@ final case class HistogramCorrection(lastValue: LongHistogram, correction: LongH trait CounterHistogramReader extends HistogramReader with CounterVectorReader { def correctedValue(n: Int, meta: CorrectionMeta): HistogramWithBuckets } +case class EmptyHistogramException(message: String) extends IllegalArgumentException(message) /** * A reader for SectDelta encoded histograms, including correction/drop functionality @@ -578,6 +579,10 @@ class SectDeltaHistogramReader(acc2: MemoryReader, histVect: Ptr.U8) } override def apply(index: Int): HistogramWithBuckets = { + if (length <= 0) { + throw EmptyHistogramException(s"""length = $length memory=${toHexString(acc, histVect.addr)}""") + } + require(length > 0) val histPtr = locate(index) From 523999c68018ff9a2d5106fc58d6a0c1f5d62ec4 Mon Sep 17 00:00:00 2001 From: sandeep6189 Date: Wed, 2 Aug 2023 12:00:08 -0700 Subject: [PATCH 25/39] feat(query): Cardinality V2 API Query Plan changes (#1637) feat(query): Cardinality V2 API Query Plan changes --- .../TenantIngestionMetering.scala | 4 +- .../queryplanner/LongTimeRangePlanner.scala | 28 +++++- .../queryplanner/MultiPartitionPlanner.scala | 9 +- .../queryplanner/SingleClusterPlanner.scala | 2 +- .../queryplanner/SinglePartitionPlanner.scala | 11 +-- .../LongTimeRangePlannerSpec.scala | 35 +++++++ .../MultiPartitionPlannerSpec.scala | 37 +++++++- .../queryplanner/PlannerHierarchySpec.scala | 2 +- .../SinglePartitionPlannerSpec.scala | 68 ++++++++++++-- .../main/scala/filodb/query/LogicalPlan.scala | 21 ++++- .../scala/filodb/query/PromCirceSupport.scala | 22 ++++- .../filodb/query/PromQueryResponse.scala | 15 ++- .../filodb/query/exec/MetadataExecPlan.scala | 69 +++++++++++--- .../query/exec/MetadataRemoteExec.scala | 29 ++++++ .../scala/filodb/query/LogicalPlanSpec.scala | 12 +++ .../filodb/query/PromCirceSupportSpec.scala | 92 +++++++++++++++++++ .../filodb/query/exec/MetadataExecSpec.scala | 61 +++++++----- .../query/exec/RemoteMetadataExecSpec.scala | 50 +++++++++- 18 files changed, 488 insertions(+), 79 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/TenantIngestionMetering.scala b/coordinator/src/main/scala/filodb.coordinator/TenantIngestionMetering.scala index 04e396d125..3a2b737d59 100644 --- a/coordinator/src/main/scala/filodb.coordinator/TenantIngestionMetering.scala +++ b/coordinator/src/main/scala/filodb.coordinator/TenantIngestionMetering.scala @@ -79,11 +79,11 @@ case class TenantIngestionMetering(settings: FilodbSettings, "cluster_type" -> CLUSTER_TYPE) if (CLUSTER_TYPE == "downsample") { - Kamon.gauge(METRIC_LONGTERM).withTags(TagSet.from(tags)).update(data.counts.total.toDouble) + Kamon.gauge(METRIC_LONGTERM).withTags(TagSet.from(tags)).update(data.counts.longTerm.toDouble) } else { Kamon.gauge(METRIC_ACTIVE).withTags(TagSet.from(tags)).update(data.counts.active.toDouble) - Kamon.gauge(METRIC_TOTAL).withTags(TagSet.from(tags)).update(data.counts.total.toDouble) + Kamon.gauge(METRIC_TOTAL).withTags(TagSet.from(tags)).update(data.counts.shortTerm.toDouble) } }) case Success(QueryError(_, _, t)) => logger.warn("QueryError: " + t.getMessage) diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LongTimeRangePlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LongTimeRangePlanner.scala index 0d62aa56f8..cdc7fa88d5 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LongTimeRangePlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LongTimeRangePlanner.scala @@ -190,6 +190,28 @@ import filodb.query.exec._ PlanResult(Seq(execPlan)) } + /** + * Materialize Ts cardinality plan. For v1 version, we only go to raw cluster for back compatibility. For v2 versions, + * we would go to both downsample and raw cluster + * + * @param logicalPlan The TsCardinalities logical plan to materialize + * @param queryContext The QueryContext object + * @return + */ + private def materializeTSCardinalityPlan(queryContext: QueryContext, logicalPlan: TsCardinalities): PlanResult = { + logicalPlan.version match { + case 2 => { + val rawPlan = rawClusterPlanner.materialize(logicalPlan, queryContext) + val dsPlan = downsampleClusterPlanner.materialize(logicalPlan, queryContext) + val stitchedPlan = TsCardReduceExec(queryContext, stitchDispatcher, Seq(rawPlan, dsPlan)) + PlanResult(Seq(stitchedPlan)) + } + // version 1 defaults to raw as done before + case 1 => rawClusterMaterialize(queryContext, logicalPlan) + case _ => throw new UnsupportedOperationException(s"version ${logicalPlan.version} not supported!") + } + } + // scalastyle:off cyclomatic.complexity override def walkLogicalPlanTree(logicalPlan: LogicalPlan, qContext: QueryContext, @@ -199,13 +221,13 @@ import filodb.query.exec._ logicalPlan match { case p: PeriodicSeriesPlan => materializePeriodicSeriesPlan(qContext, p) case lc: LabelCardinality => materializeLabelCardinalityPlan(lc, qContext) + case ts: TsCardinalities => materializeTSCardinalityPlan(qContext, ts) case _: LabelValues | _: ApplyLimitFunction | _: SeriesKeysByFilters | _: ApplyInstantFunctionRaw | _: RawSeries | - _: LabelNames | - _: TsCardinalities => rawClusterMaterialize(qContext, logicalPlan) + _: LabelNames => rawClusterMaterialize(qContext, logicalPlan) } } else logicalPlan match { @@ -219,7 +241,7 @@ import filodb.query.exec._ case lp: BinaryJoin => materializePeriodicSeriesPlan(qContext, lp) case lp: ScalarVectorBinaryOperation => super.materializeScalarVectorBinOp(qContext, lp) case lp: LabelValues => rawClusterMaterialize(qContext, lp) - case lp: TsCardinalities => rawClusterMaterialize(qContext, lp) + case lp: TsCardinalities => materializeTSCardinalityPlan(qContext, lp) case lp: SeriesKeysByFilters => rawClusterMaterialize(qContext, lp) case lp: ApplyMiscellaneousFunction => super.materializeApplyMiscellaneousFunction(qContext, lp) case lp: ApplySortFunction => super.materializeApplySortFunction(qContext, lp) diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala index 668bd625fe..60eba29b63 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala @@ -601,8 +601,6 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider def materializeTsCardinalities(lp: TsCardinalities, qContext: QueryContext): PlanResult = { - import TsCardinalities._ - val queryParams = qContext.origQueryParams.asInstanceOf[PromQlQueryParams] val partitions = if (lp.shardKeyPrefix.size >= 2) { // At least a ws/ns pair is required to select specific partitions. @@ -621,12 +619,7 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider if (p.partitionName.equals(localPartitionName)) localPartitionPlanner.materialize(lp, qContext) else { - val params = Map( - "match[]" -> ("{" + SHARD_KEY_LABELS.zip(lp.shardKeyPrefix) - .map{ case (label, value) => s"""$label="$value""""} - .mkString(",") + "}"), - "numGroupByFields" -> lp.numGroupByFields.toString) - createMetadataRemoteExec(qContext, p, params) + createMetadataRemoteExec(qContext, p, lp.queryParams()) } } if (execPlans.size == 1) { diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala index 810e01ef52..4b31c7b209 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala @@ -832,7 +832,7 @@ class SingleClusterPlanner(val dataset: Dataset, forceInProcess: Boolean): PlanResult = { val metaExec = shardMapperFunc.assignedShards.map{ shard => val dispatcher = dispatcherForShard(shard, forceInProcess, qContext) - exec.TsCardExec(qContext, dispatcher, dsRef, shard, lp.shardKeyPrefix, lp.numGroupByFields) + exec.TsCardExec(qContext, dispatcher, dsRef, shard, lp.shardKeyPrefix, lp.numGroupByFields, clusterName) } PlanResult(metaExec) } diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SinglePartitionPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SinglePartitionPlanner.scala index 3c7e20f569..aab22c2ce2 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SinglePartitionPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SinglePartitionPlanner.scala @@ -11,15 +11,10 @@ import filodb.query.exec._ * distributed across multiple clusters. * * @param planners map of clusters names in the local partition to their Planner objects - * @param defaultPlanner TsCardinalities queries are routed here. - * Note: this is a temporary fix only to support TsCardinalities queries. - * These must be routed to planners according to the data they govern, and - * this information isn't accessible without this parameter. * @param plannerSelector a function that selects the planner name given the metric name * @param dataset a function that selects the planner name given the metric name */ class SinglePartitionPlanner(planners: Map[String, QueryPlanner], - defaultPlanner: String, // TODO: remove this-- see above. plannerSelector: String => String, val dataset: Dataset, val queryConfig: QueryConfig) @@ -74,9 +69,9 @@ class SinglePartitionPlanner(planners: Map[String, QueryPlanner], } private def materializeTsCardinalities(logicalPlan: TsCardinalities, qContext: QueryContext): PlanResult = { - // Delegate to defaultPlanner - planners.get(defaultPlanner).map(p => PlanResult(Seq(p.materialize(logicalPlan, qContext)))) - .getOrElse(PlanResult(Seq())) + val execPlans = logicalPlan.datasets.map(d => planners.get(d)) + .map(x => x.get.materialize(logicalPlan, qContext)) + PlanResult(Seq(TsCardReduceExec(qContext, inProcessPlanDispatcher, execPlans))) } diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/LongTimeRangePlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/LongTimeRangePlannerSpec.scala index 1b1600dc1c..aa3f24edc7 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/LongTimeRangePlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/LongTimeRangePlannerSpec.scala @@ -336,6 +336,41 @@ class LongTimeRangePlannerSpec extends AnyFunSpec with Matchers with PlanValidat binaryJoinExec.rhs.head.isInstanceOf[StitchRvsExec] shouldEqual (true) } + it("tsCardinality should span to both downsample and raw for version 2") { + val logicalPlan = TsCardinalities(Seq("a","b"), 2, 2, Seq("longtime-prometheus")) + + val cardExecPlan = longTermPlanner.materialize( + logicalPlan, + QueryContext(origQueryParams = promQlQueryParams.copy(promQl = ""))).asInstanceOf[TsCardReduceExec] + + cardExecPlan.dispatcher.isInstanceOf[InProcessPlanDispatcher] shouldEqual true + cardExecPlan.children.size shouldEqual 2 + val rawEp = cardExecPlan.children.head.asInstanceOf[MockExecPlan] + val downsampleEp = cardExecPlan.children.last.asInstanceOf[MockExecPlan] + + rawEp.name shouldEqual "raw" + downsampleEp.name shouldEqual "downsample" + } + + it("tsCardinality should throw exception for version > 2") { + val logicalPlan = TsCardinalities(Seq("a", "b"), 2, 3, Seq("longtime-prometheus")) + val ex = intercept[UnsupportedOperationException] { + val cardExecPlan = longTermPlanner.materialize( + logicalPlan, + QueryContext(origQueryParams = promQlQueryParams.copy(promQl = ""))) + } + ex.getMessage.contains("version 3 not supported!") shouldEqual true + } + + it("tsCardinality should span to raw ONLY for version 1") { + val logicalPlan = TsCardinalities(Seq("a", "b"), 2, 1, Seq("longtime-prometheus")) + + val cardRawExecPlan = longTermPlanner.materialize( + logicalPlan, + QueryContext(origQueryParams = promQlQueryParams.copy(promQl = ""))).asInstanceOf[MockExecPlan] + + cardRawExecPlan.name shouldEqual "raw" + } it("should direct overlapping binary join offset queries with vector(0) " + "to both raw & downsample planner and stitch") { diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala index 2b44b64667..5e237b068f 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala @@ -1049,7 +1049,7 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers with PlanValida (endSeconds * 1000) } - it ("should generate correct ExecPlan for TsCardinalities query") { + it ("should generate correct ExecPlan for TsCardinalities query version 1") { def partitions(timeRange: TimeRange): List[PartitionAssignment] = List(PartitionAssignment("remote", "remote-url", TimeRange(startSeconds * 1000, localPartitionStart * 1000 - 1)), @@ -1066,7 +1066,8 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers with PlanValida val engine = new MultiPartitionPlanner(partitionLocationProvider, localPlanner, "local", dataset, queryConfig) val lp = TsCardinalities(Seq("a", "b"), 3) val promQlQueryParams = PromQlQueryParams("", startSeconds, step, endSeconds, Some("/api/v1/metering/cardinality/timeseries")) - val expectedUrlParams = Map("match[]" -> """{_ws_="a",_ns_="b"}""", "numGroupByFields" -> "3") + val expectedUrlParams = Map("match[]" -> """{_ws_="a",_ns_="b"}""", "numGroupByFields" -> "3", "verbose" -> "true", + "datasets" -> "") val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams, plannerParams = PlannerParams(processMultiPartition = true))) @@ -1077,6 +1078,38 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers with PlanValida execPlan.children(1).asInstanceOf[MetadataRemoteExec].urlParams shouldEqual(expectedUrlParams) } + it("should generate correct ExecPlan for TsCardinalities query version 2") { + def partitions(timeRange: TimeRange): List[PartitionAssignment] = + List(PartitionAssignment("remote", "remote-url", + TimeRange(startSeconds * 1000, localPartitionStart * 1000 - 1)), + PartitionAssignment("local", "local-url", TimeRange(localPartitionStart * 1000, endSeconds * 1000))) + + val partitionLocationProvider = new PartitionLocationProvider { + override def getPartitions(routingKey: Map[String, String], timeRange: TimeRange): List[PartitionAssignment] = + partitions(timeRange) + + override def getMetadataPartitions(nonMetricShardKeyFilters: Seq[ColumnFilter], + timeRange: TimeRange):List[PartitionAssignment] = + partitions(timeRange) + } + + val engine = new MultiPartitionPlanner(partitionLocationProvider, localPlanner, "local", + dataset, queryConfig) + val lp = TsCardinalities(Seq("a", "b"), 3, 2, Seq("longtime-prometheus","recordingrules-prometheus_rules_1m")) + val promQlQueryParams = PromQlQueryParams("", startSeconds, step, endSeconds, + Some("/api/v2/metering/cardinality/timeseries")) + val expectedUrlParams = Map("match[]" -> """{_ws_="a",_ns_="b"}""", "numGroupByFields" -> "3","verbose" -> "true", + "datasets" -> "longtime-prometheus,recordingrules-prometheus_rules_1m") + + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams, plannerParams = + PlannerParams(processMultiPartition = true))) + + execPlan.isInstanceOf[TsCardReduceExec] shouldEqual (true) + execPlan.children(0).isInstanceOf[TsCardReduceExec] shouldEqual (true) + execPlan.children(1).isInstanceOf[MetadataRemoteExec] shouldEqual (true) + execPlan.children(1).asInstanceOf[MetadataRemoteExec].urlParams shouldEqual (expectedUrlParams) + } + it ("should generate multipartition BinaryJoin") { def partitions(timeRange: TimeRange): List[PartitionAssignment] = List(PartitionAssignment("remote", "remote-url", TimeRange(timeRange.startMs, timeRange.endMs))) diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala index 1c03160273..14a602ffab 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala @@ -63,7 +63,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS if (metricName.contains(":1m")) "recordingRules" else "longTerm" } val planners = Map("longTerm" -> longTermPlanner, "recordingRules" -> recordingRulesPlanner) - val singlePartitionPlanner = new SinglePartitionPlanner(planners, "longTerm", plannerSelector, + val singlePartitionPlanner = new SinglePartitionPlanner(planners, plannerSelector, dataset, queryConfig) private val oneRemotePartitionLocationProvider = new PartitionLocationProvider { diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/SinglePartitionPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/SinglePartitionPlannerSpec.scala index 081fc02001..79098932e6 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/SinglePartitionPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/SinglePartitionPlannerSpec.scala @@ -4,7 +4,6 @@ import akka.actor.ActorSystem import akka.testkit.TestProbe import com.typesafe.config.ConfigFactory import monix.execution.Scheduler - import filodb.coordinator.{ActorPlanDispatcher, ShardMapper} import filodb.core.{DatasetRef, MetricsTestData} import filodb.core.metadata.Schemas @@ -81,10 +80,13 @@ class SinglePartitionPlannerSpec extends AnyFunSpec with Matchers { } val planners = Map("local" -> highAvailabilityPlanner, "rules1" -> rrPlanner1, "rules2" -> rrPlanner2) - val plannerSelector = (metricName: String) => { if (metricName.equals("rr1")) "rules1" - else if (metricName.equals("rr2")) "rules2" else "local" } - val engine = new SinglePartitionPlanner(planners, "local", plannerSelector, dataset, queryConfig) + val plannerSelector = (metricName: String) => { + if (metricName.equals("rr1")) "rules1" + else if (metricName.equals("rr2")) "rules2" else "local" + } + + val engine = new SinglePartitionPlanner(planners, plannerSelector, dataset, queryConfig) it("should generate Exec plan for simple query") { val lp = Parser.queryToLogicalPlan("test{job = \"app\"}", 1000, 1000) @@ -160,7 +162,7 @@ class SinglePartitionPlannerSpec extends AnyFunSpec with Matchers { execPlan.asInstanceOf[PartKeysDistConcatExec].children(2).asInstanceOf[MockExecPlan].name shouldEqual ("rules2") } - it("should generate correct ExecPlan for TsCardinalities") { + it("should generate correct ExecPlan for TsCardinalities version 1") { // Note: this test is expected to break when TsCardinalities.isRoutable = true // Note: unrelated to the above, this test is setup to confirm that a hacky fix to @@ -169,15 +171,63 @@ class SinglePartitionPlannerSpec extends AnyFunSpec with Matchers { val localPlanner = new SingleClusterPlanner( dataset, schemas, localMapper, earliestRetainedTimestampFn = 0, queryConfig, "raw-temp") val planners = Map("raw-temp" -> localPlanner, "rules1" -> rrPlanner1, "rules2" -> rrPlanner2) - val engine = new SinglePartitionPlanner(planners, "raw-temp", plannerSelector, dataset, queryConfig) - val lp = TsCardinalities(Seq("a", "b"), 2) + val engine = new SinglePartitionPlanner(planners, plannerSelector, dataset, queryConfig) + val lp = TsCardinalities(Seq("a", "b"), 2, 1, Seq("raw-temp")) + + // Plan should just contain a single root TsCardReduceExec and its TsCardExec children. + // Currently, queries are routed only to the planner who's name equals the SPP's "defaultPlanner" member. + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams.copy(promQl = ""))) + execPlan.isInstanceOf[TsCardReduceExec] shouldEqual (true) + + // UPDATE: We added another TsCardReduceExec to merge data across different datasets + execPlan.asInstanceOf[TsCardReduceExec].children.length shouldEqual(1) + execPlan.asInstanceOf[TsCardReduceExec].children(0).children.length shouldEqual(32) + execPlan.children.forall(_.isInstanceOf[TsCardReduceExec]) shouldEqual true + execPlan.children(0).children.forall(_.isInstanceOf[TsCardExec]) shouldEqual true + } + + it("should generate correct ExecPlan for TsCardinalities version 2") { + + // Note: this test is expected to break when TsCardinalities.isRoutable = true + // Note: unrelated to the above, this test is setup to confirm that a hacky fix to + // SPP::materializeTsCardinalities is working. See there for additional details. + + val rawPlanner = new SingleClusterPlanner( + dataset, schemas, localMapper, earliestRetainedTimestampFn = 0, queryConfig, "raw") + + val downsamplePlanner = new SingleClusterPlanner( + dataset, schemas, localMapper, earliestRetainedTimestampFn = 0, queryConfig, "downsample") + + val longTermPlanner = new LongTimeRangePlanner(rawPlanner, downsamplePlanner, 0, 0, + InProcessPlanDispatcher(QueryConfig.unitTestingQueryConfig), queryConfig, dataset) + + val planners = Map("longtime" -> longTermPlanner, "rules1" -> rrPlanner1, "rules2" -> rrPlanner2) + + val engine = new SinglePartitionPlanner(planners, plannerSelector, dataset, queryConfig) + val lp = TsCardinalities(Seq("a", "b"), 2, 2, Seq("longtime", "rules1", "rules2")) // Plan should just contain a single root TsCardReduceExec and its TsCardExec children. // Currently, queries are routed only to the planner who's name equals the SPP's "defaultPlanner" member. val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams.copy(promQl = ""))) execPlan.isInstanceOf[TsCardReduceExec] shouldEqual (true) - execPlan.asInstanceOf[TsCardReduceExec].children.length shouldEqual(32) - execPlan.children.forall(_.isInstanceOf[TsCardExec]) shouldEqual true + + // UPDATE: should have 3 children, since we passed 3 datasets + execPlan.asInstanceOf[TsCardReduceExec].children.length shouldEqual (3) + + // the longtime should have 2 children for downsample and raw + val longTermExecPlan = execPlan.asInstanceOf[TsCardReduceExec].children(0) + longTermExecPlan.children.length shouldEqual (2) + longTermExecPlan.children(0).dispatcher.clusterName shouldEqual "raw" + longTermExecPlan.children(1).dispatcher.clusterName shouldEqual "downsample" + + longTermExecPlan.children(0).children.forall(_.isInstanceOf[TsCardExec]) shouldEqual true + longTermExecPlan.children(1).children.forall(_.isInstanceOf[TsCardExec]) shouldEqual true + + // rules plan + val rules1ExecPlan = execPlan.asInstanceOf[TsCardReduceExec].children(1).asInstanceOf[MockExecPlan] + rules1ExecPlan.name shouldEqual "rules1" + val rules2ExecPlan = execPlan.asInstanceOf[TsCardReduceExec].children(2).asInstanceOf[MockExecPlan] + rules2ExecPlan.name shouldEqual "rules2" } it("should generate Exec plan for Scalar query which does not have any metric") { diff --git a/query/src/main/scala/filodb/query/LogicalPlan.scala b/query/src/main/scala/filodb/query/LogicalPlan.scala index 4dc3029308..40f8961c00 100644 --- a/query/src/main/scala/filodb/query/LogicalPlan.scala +++ b/query/src/main/scala/filodb/query/LogicalPlan.scala @@ -2,6 +2,7 @@ package filodb.query import filodb.core.query.{ColumnFilter, RangeParams, RvRange} import filodb.core.query.Filter.Equals +import filodb.query.exec.TsCardExec //scalastyle:off number.of.types //scalastyle:off file.size.limit @@ -181,7 +182,10 @@ object TsCardinalities { * [ws, ns] { 2, 3 } * [ws, ns, metric] { 3 } */ -case class TsCardinalities(shardKeyPrefix: Seq[String], numGroupByFields: Int) extends LogicalPlan { +case class TsCardinalities(shardKeyPrefix: Seq[String], + numGroupByFields: Int, + version: Int = 1, + datasets: Seq[String] = Seq()) extends LogicalPlan { import TsCardinalities._ require(numGroupByFields >= 1 && numGroupByFields <= 3, @@ -196,6 +200,21 @@ case class TsCardinalities(shardKeyPrefix: Seq[String], numGroupByFields: Int) e def filters(): Seq[ColumnFilter] = SHARD_KEY_LABELS.zip(shardKeyPrefix).map{ case (label, value) => ColumnFilter(label, Equals(value))} + + /** + * Helper function to create a Map of all the query parameters to be sent for the remote API call + * @return Immutable Map[String, String] + */ + def queryParams(): Map[String, String] = { + Map( + "match[]" -> ("{" + SHARD_KEY_LABELS.zip(shardKeyPrefix) + .map{ case (label, value) => s"""$label="$value""""} + .mkString(",") + "}"), + "numGroupByFields" -> numGroupByFields.toString, + "verbose" -> "true", // Using this plan to determine if we need to pass additional values in groupKey or not + "datasets" -> datasets.mkString(TsCardExec.PREFIX_DELIM) + ) + } } /** diff --git a/query/src/main/scala/filodb/query/PromCirceSupport.scala b/query/src/main/scala/filodb/query/PromCirceSupport.scala index d1db15b35a..df002ad5b4 100644 --- a/query/src/main/scala/filodb/query/PromCirceSupport.scala +++ b/query/src/main/scala/filodb/query/PromCirceSupport.scala @@ -21,6 +21,8 @@ object PromCirceSupport { Json.fromValues(Seq(group.asJson, cardinality.asJson)) case t @ TsCardinalitiesSampl(group, cardinality) => Json.fromValues(Seq(group.asJson, cardinality.asJson)) + case a @ TsCardinalitiesSamplV2(group, cardinality, dataset, _type) => + Json.fromValues(Seq(group.asJson, cardinality.asJson, dataset.asJson, _type.asJson)) } implicit val decodeAvgSample: Decoder[AvgSampl] = new Decoder[AvgSampl] { @@ -57,10 +59,22 @@ object PromCirceSupport { card <- c.get[Seq[Map[String, String]]]("cardinality") } yield LabelCardinalitySampl(metric, card) } else if (c.downField("group").focus.nonEmpty) { - for { - group <- c.get[Map[String, String]]("group") - card <- c.get[Map[String, Int]]("cardinality") - } yield TsCardinalitiesSampl(group, card) + // V2 Cardinality API also has a dataset field. So we are using it to distinguish + // between the TsCardinalitiesSamplV2 vs TsCardinalitiesSampl response + if (c.downField("dataset").focus.nonEmpty) { + for { + group <- c.get[Map[String, String]]("group") + card <- c.get[Map[String, Int]]("cardinality") + dataset <- c.get[String]("dataset") + _type <- c.get[String]("_type") + } yield TsCardinalitiesSamplV2(group, card, dataset, _type) + } + else { + for { + group <- c.get[Map[String, String]]("group") + card <- c.get[Map[String, Int]]("cardinality") + } yield TsCardinalitiesSampl(group, card) + } } else { throw new IllegalArgumentException("could not decode any expected cardinality-related field") } diff --git a/query/src/main/scala/filodb/query/PromQueryResponse.scala b/query/src/main/scala/filodb/query/PromQueryResponse.scala index 4b77a40982..1b2454f0e9 100644 --- a/query/src/main/scala/filodb/query/PromQueryResponse.scala +++ b/query/src/main/scala/filodb/query/PromQueryResponse.scala @@ -68,4 +68,17 @@ final case class LabelCardinalitySampl(metric: Map[String, String], cardinality: Seq[Map[String, String]]) extends MetadataSampl final case class TsCardinalitiesSampl(group: Map[String, String], - cardinality: Map[String, Int]) extends MetadataSampl \ No newline at end of file + cardinality: Map[String, Int]) extends MetadataSampl + +/** + * @param group map of shardKeyPrefix and its values. Key includes - _ws_, _ns_, __name__ + * @param cardinality map of string and int (cardinality count). Key includes - active, shortTerm, longTerm + * @param dataset This is the user specified dataset for the cureent row. + * Valid value includes - raw, aggregated, recordingrules + * @param _type This is the internal filodb dataset. Example values - prometheus, prometheus_preagg, + * prometheus_rules_longterm, prometheus_rules_1m + */ +final case class TsCardinalitiesSamplV2(group: Map[String, String], + cardinality: Map[String, Int], + dataset: String, + _type: String) extends MetadataSampl \ No newline at end of file diff --git a/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala b/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala index 31a1621496..9e8fec9ce5 100644 --- a/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala +++ b/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala @@ -109,7 +109,7 @@ final case class TsCardReduceExec(queryContext: QueryContext, val taskOfResults = childResponses.flatMap(res => Observable.fromIterable(res._1.result)) .foldLeftL(new mutable.HashMap[ZeroCopyUTF8String, CardCounts])(mapFold) .map{ aggMap => - val it = aggMap.toSeq.sortBy(-_._2.total).map{ case (group, counts) => + val it = aggMap.toSeq.sortBy(-_._2.shortTerm).map{ case (group, counts) => CardRowReader(group, counts) }.iterator IteratorBackedRangeVector(new CustomRangeVectorKey(Map.empty), NoCloseCursor(it), None) @@ -462,9 +462,21 @@ final case object TsCardExec { val PREFIX_DELIM = "," + /** + * This is the V1 schema version of QueryResult for TSCardinalities query + */ + val RESULT_SCHEMA_V1 = ResultSchema(Seq(ColumnInfo("group", ColumnType.StringColumn), + ColumnInfo("active", ColumnType.IntColumn), + ColumnInfo("total", ColumnType.IntColumn)), 1) + + /** + * V2 schema version of QueryResult for TSCardinalities query. One more additional column `longterm` is added + * to represent the cardinality count of downsample clusters + */ val RESULT_SCHEMA = ResultSchema(Seq(ColumnInfo("group", ColumnType.StringColumn), ColumnInfo("active", ColumnType.IntColumn), - ColumnInfo("total", ColumnType.IntColumn)), 1) + ColumnInfo("shortTerm", ColumnType.IntColumn), + ColumnInfo("longTerm", ColumnType.IntColumn)), 1) /** * Convert a shard key prefix to a row's group name. @@ -474,13 +486,28 @@ final case object TsCardExec { prefix.mkString(PREFIX_DELIM).utf8 } - case class CardCounts(active: Int, total: Int) { - if (total < active) { - qLogger.warn(s"CardCounts created with total < active; total: $total, active: $active") + /** + * @param prefix ShardKeyPrefix from the Cardinality Record + * @param datasetName FiloDB dataset name + * @return concatenation of ShardKeyPrefix and filodb dataset + */ + def prefixToGroupWithDataset(prefix: Seq[String], datasetName: String):ZeroCopyUTF8String = { + s"${prefix.mkString(PREFIX_DELIM)}$PREFIX_DELIM$datasetName".utf8 + } + + /** + * @param active Actively (1 hourt) Ingesting Cardinality Count + * @param shortTerm This is the 7 day running Cardinality Count + * @param longTerm upto 6 month running Cardinality Count + */ + case class CardCounts(active: Int, shortTerm: Int, longTerm: Int = 0) { + if (shortTerm < active) { + qLogger.warn(s"CardCounts created with total < active; shortTerm: $shortTerm, active: $active") } def add(other: CardCounts): CardCounts = { CardCounts(active + other.active, - total + other.total) + shortTerm + other.shortTerm, + longTerm + other.longTerm) } } @@ -489,7 +516,8 @@ final case object TsCardExec { override def getBoolean(columnNo: Int): Boolean = ??? override def getInt(columnNo: Int): Int = columnNo match { case 1 => counts.active - case 2 => counts.total + case 2 => counts.shortTerm + case 3 => counts.longTerm case _ => throw new IllegalArgumentException(s"illegal getInt columnNo: $columnNo") } override def getLong(columnNo: Int): Long = ??? @@ -514,8 +542,7 @@ final case object TsCardExec { object RowData { def fromRowReader(rr: RowReader): RowData = { val group = rr.getAny(0).asInstanceOf[ZeroCopyUTF8String] - val counts = CardCounts(rr.getInt(1), - rr.getInt(2)) + val counts = CardCounts(rr.getInt(1), rr.getInt(2), rr.getInt(3)) RowData(group, counts) } } @@ -530,13 +557,17 @@ final case class TsCardExec(queryContext: QueryContext, dataset: DatasetRef, shard: Int, shardKeyPrefix: Seq[String], - numGroupByFields: Int) extends LeafExecPlan with StrictLogging { + numGroupByFields: Int, + clusterName: String) extends LeafExecPlan with StrictLogging { require(numGroupByFields >= 1, "numGroupByFields must be positive") require(numGroupByFields >= shardKeyPrefix.size, s"numGroupByFields ($numGroupByFields) must indicate at least as many " + s"fields as shardKeyPrefix.size (${shardKeyPrefix.size})") + // Making the passed cluster name to lowercase to avoid case complexities for string comparisions + val clusterNameLowercase = clusterName.toLowerCase() + override def enforceSampleLimit: Boolean = false // scalastyle:off method.length @@ -547,15 +578,27 @@ final case class TsCardExec(queryContext: QueryContext, source.checkReadyForQuery(dataset, shard, querySession) source.acquireSharedLock(dataset, shard, querySession) - val rvs = source match { case tsMemStore: TimeSeriesStore => Observable.eval { val cards = tsMemStore.scanTsCardinalities( dataset, Seq(shard), shardKeyPrefix, numGroupByFields) val it = cards.map { card => - CardRowReader(prefixToGroup(card.prefix), - CardCounts(card.value.activeTsCount, card.value.tsCount)) + val groupKey = prefixToGroupWithDataset(card.prefix, dataset.dataset) + + // NOTE: cardinality data from downsample cluster is stored as total count in CardinalityStore. But for the + // user perspective, the cardinality data in downsample is a longterm data. Hence, we are forking the + // data path based on the cluster the data is being served from + if (clusterNameLowercase.contains("downsample")) { + CardRowReader( + groupKey, + CardCounts(0, 0, card.value.tsCount)) + } + else { + CardRowReader( + groupKey, + CardCounts(card.value.activeTsCount, card.value.tsCount)) + } }.iterator IteratorBackedRangeVector(new CustomRangeVectorKey(Map.empty), NoCloseCursor(it), None) } diff --git a/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala b/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala index 4b913c3443..348ff7b8ac 100644 --- a/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala +++ b/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala @@ -68,6 +68,7 @@ case class MetadataRemoteExec(queryEndpoint: String, case _: MetadataMapSampl => mapTypeQueryResponse(response, id) case _: LabelCardinalitySampl => mapLabelCardinalityResponse(response, id) case _: TsCardinalitiesSampl => mapTsCardinalitiesResponse(response, id) + case _: TsCardinalitiesSamplV2 => mapTsCardinalitiesResponseV2(response, id) case _ => labelsQueryResponse(response, id) } } @@ -88,6 +89,34 @@ case class MetadataRemoteExec(queryEndpoint: String, val rv = IteratorBackedRangeVector(CustomRangeVectorKey.empty, NoCloseCursor(rows.iterator), None) // dont add this size to queryStats since it was already added by callee use dummy QueryStats() val srv = SerializedRangeVector(rv, builder, RECORD_SCHEMA, queryWithPlanName(queryContext), dummyQueryStats) + + // NOTE: We are using the RESULT_SCHEMA definitions to determine the iteration of shardKeyPrefix in v1 result. + // Hence, we are sending the older result schema which was used for V1 Cardinality API + QueryResult(id, RESULT_SCHEMA_V1, Seq(srv)) + } + + /** + * @param response Metadata Response from the remote query server API call. + * @param id QueryId + * @return We convert the TsCardinalitiesSamplV2 response to QueryResult which can be appropriately parsed + * by the query service and return the response to the user + */ + private def mapTsCardinalitiesResponseV2(response: MetadataSuccessResponse, id: String): QueryResponse = { + import NoCloseCursor._ + import TsCardinalities._ + import TsCardExec._ + + val RECORD_SCHEMA = SerializedRangeVector.toSchema(RESULT_SCHEMA.columns) + + val rows = response.data.asInstanceOf[Seq[TsCardinalitiesSamplV2]] + .map { ts => + val prefix = SHARD_KEY_LABELS.take(ts.group.size).map(l => ts.group(l)) + val counts = CardCounts(ts.cardinality("active"), ts.cardinality("shortTerm"), ts.cardinality("longTerm")) + CardRowReader(prefixToGroupWithDataset(prefix, ts._type), counts) + } + val rv = IteratorBackedRangeVector(CustomRangeVectorKey.empty, NoCloseCursor(rows.iterator), None) + // dont add this size to queryStats since it was already added by callee use dummy QueryStats() + val srv = SerializedRangeVector(rv, builder, RECORD_SCHEMA, queryWithPlanName(queryContext), dummyQueryStats) QueryResult(id, RESULT_SCHEMA, Seq(srv)) } diff --git a/query/src/test/scala/filodb/query/LogicalPlanSpec.scala b/query/src/test/scala/filodb/query/LogicalPlanSpec.scala index 10f3a140df..e44fd6fa98 100644 --- a/query/src/test/scala/filodb/query/LogicalPlanSpec.scala +++ b/query/src/test/scala/filodb/query/LogicalPlanSpec.scala @@ -323,4 +323,16 @@ class LogicalPlanSpec extends AnyFunSpec with Matchers { TsCardinalities(Seq("a", "b"), 3) TsCardinalities(Seq("a", "b", "c"), 3) } + + it ("TsCardinalities queryParams should have expected values") { + val datasets = Seq("longtime-prometheus", + "recordingrules-prometheus_rules_longterm") + val plan = TsCardinalities(Seq("a","b","c"), 3, 2, datasets) + val queryParamsMap = plan.queryParams() + + queryParamsMap.get("numGroupByFields").get shouldEqual "3" + queryParamsMap.get("datasets").get shouldEqual datasets.mkString(",") + queryParamsMap.get("verbose").get shouldEqual "true" + queryParamsMap.get("match[]").get shouldEqual "{_ws_=\"a\",_ns_=\"b\",__name__=\"c\"}" + } } diff --git a/query/src/test/scala/filodb/query/PromCirceSupportSpec.scala b/query/src/test/scala/filodb/query/PromCirceSupportSpec.scala index ebfdeae638..acbb5c3b48 100644 --- a/query/src/test/scala/filodb/query/PromCirceSupportSpec.scala +++ b/query/src/test/scala/filodb/query/PromCirceSupportSpec.scala @@ -205,6 +205,98 @@ class PromCirceSupportSpec extends AnyFunSpec with Matchers with ScalaFutures { } } + it("should parse TsCardinalitiesSamplV2") { + val expected = Seq( + TsCardinalitiesSamplV2( + Map("_ws_" -> "demo", "_ns_" -> "App-0", "_metric_" -> "heap_usage"), + Map("active" -> 2, "shortTerm" -> 3, "longTerm" -> 5), + "raw", + "prometheus"), + TsCardinalitiesSamplV2( + Map("_ws_" -> "demo", "_ns_" -> "App-1"), + Map("active" -> 6, "shortTerm" -> 8, "longTerm" -> 0), + "recordingrules", + "prometheus_rules_1m"), + TsCardinalitiesSamplV2( + Map("_ws_" -> "demo", "_ns_" -> "App-2"), + Map("active" -> 14, "shortTerm" -> 28, "longTerm" -> 0), + "recordingrules", + "prometheus_rules_longterm"), + TsCardinalitiesSamplV2( + Map("_ws_" -> "demo", "_ns_" -> "App-3", "_metric_" -> "heap_usage:::agg"), + Map("active" -> 11, "shortTerm" -> 22, "longTerm" -> 33), + "aggregated", + "prometheus_preagg") + ) + val inputString = + """{ + | "status": "success", + | "data": [ + | { + | "_type": "prometheus", + | "dataset": "raw", + | "cardinality": { + | "active": 2, + | "longTerm": 5, + | "shortTerm": 3 + | }, + | "group": { + | "_ns_": "App-0", + | "_ws_": "demo", + | "_metric_": "heap_usage" + | } + | }, + | { + | "_type": "prometheus_rules_1m", + | "dataset": "recordingrules", + | "cardinality": { + | "active": 6, + | "longTerm": 0, + | "shortTerm": 8 + | }, + | "group": { + | "_ns_": "App-1", + | "_ws_": "demo" + | } + | }, + | { + | "_type": "prometheus_rules_longterm", + | "dataset": "recordingrules", + | "cardinality": { + | "active": 14, + | "longTerm": 0, + | "shortTerm": 28 + | }, + | "group": { + | "_ns_": "App-2", + | "_ws_": "demo" + | } + | }, + | { + | "_type": "prometheus_preagg", + | "dataset": "aggregated", + | "cardinality": { + | "active": 11, + | "longTerm": 33, + | "shortTerm": 22 + | }, + | "group": { + | "_ns_": "App-3", + | "_ws_": "demo", + | "_metric_": "heap_usage:::agg" + | } + | } + | ], + | "errorType": null, + | "error": null + |}""".stripMargin + + parser.decode[MetadataSuccessResponse](inputString) match { + case Right(response) => response shouldEqual MetadataSuccessResponse(expected) + case Left(ex) => throw ex + } + } + it("should parse aggregateResponse") { val input = """[{ | "status": "success", diff --git a/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala b/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala index c0b84f3968..58c5c0b294 100644 --- a/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala @@ -367,44 +367,55 @@ class MetadataExecSpec extends AnyFunSpec with Matchers with ScalaFutures with B // and converted to ZeroCopyUTF8Strings. Seq( TestSpec(Seq(), 1, Seq( - Seq("demo") -> CardCounts(4,4), - Seq("demo-A") -> CardCounts(1,1), - Seq("testws") -> CardCounts(1,1) + Seq("demo", "timeseries") -> CardCounts(4,4,4), + Seq("testws", "timeseries") -> CardCounts(1,1,1), + Seq("demo-A", "timeseries") -> CardCounts(1,1,1) )), TestSpec(Seq(), 2, Seq( - Seq("demo", "App-0") -> CardCounts(4,4), - Seq("demo-A", "App-A") -> CardCounts(1,1), - Seq("testws", "testns") -> CardCounts(1,1))), + Seq("demo", "App-0", "timeseries") -> CardCounts(4,4,4), + Seq("testws", "testns", "timeseries") -> CardCounts(1,1,1), + Seq("demo-A", "App-A", "timeseries") -> CardCounts(1,1,1) + )), TestSpec(Seq(), 3, Seq( - Seq("demo", "App-0", "http_req_total") -> CardCounts(2,2), - Seq("testws", "testns", "long_labels_metric") -> CardCounts(1,1), - Seq("demo", "App-0", "http_foo_total") -> CardCounts(1,1), - Seq("demo-A", "App-A", "http_req_total-A") -> CardCounts(1,1), - Seq("demo", "App-0", "http_bar_total") -> CardCounts(1,1))), + Seq("demo", "App-0", "http_req_total", "timeseries") -> CardCounts(2,2,2), + Seq("demo", "App-0", "http_bar_total", "timeseries") -> CardCounts(1,1,1), + Seq("demo", "App-0", "http_foo_total", "timeseries") -> CardCounts(1,1,1), + Seq("demo-A", "App-A", "http_req_total-A", "timeseries") -> CardCounts(1,1,1), + Seq("testws", "testns", "long_labels_metric", "timeseries") -> CardCounts(1,1,1) + )), TestSpec(Seq("demo"), 1, Seq( - Seq("demo") -> CardCounts(4,4))), + Seq("demo", "timeseries") -> CardCounts(4,4,4))), TestSpec(Seq("demo"), 2, Seq( - Seq("demo", "App-0") -> CardCounts(4,4))), + Seq("demo", "App-0", "timeseries") -> CardCounts(4,4,4))), TestSpec(Seq("demo"), 3, Seq( - Seq("demo", "App-0", "http_req_total") -> CardCounts(2,2), - Seq("demo", "App-0", "http_foo_total") -> CardCounts(1,1), - Seq("demo", "App-0", "http_bar_total") -> CardCounts(1,1))), + Seq("demo", "App-0", "http_req_total", "timeseries") -> CardCounts(2,2,2), + Seq("demo", "App-0", "http_bar_total", "timeseries") -> CardCounts(1,1,1), + Seq("demo", "App-0", "http_foo_total", "timeseries") -> CardCounts(1,1,1) + )), TestSpec(Seq("demo", "App-0"), 2, Seq( - Seq("demo", "App-0") -> CardCounts(4,4))), + Seq("demo", "App-0", "timeseries") -> CardCounts(4,4,4) + )), TestSpec(Seq("demo", "App-0"), 3, Seq( - Seq("demo", "App-0", "http_req_total") -> CardCounts(2,2), - Seq("demo", "App-0", "http_foo_total") -> CardCounts(1,1), - Seq("demo", "App-0", "http_bar_total") -> CardCounts(1,1))), + Seq("demo", "App-0", "http_req_total", "timeseries") -> CardCounts(2,2,2), + Seq("demo", "App-0", "http_bar_total", "timeseries") -> CardCounts(1,1,1), + Seq("demo", "App-0", "http_foo_total", "timeseries") -> CardCounts(1,1,1) + )), TestSpec(Seq("demo", "App-0", "http_req_total"), 3, Seq( - Seq("demo", "App-0", "http_req_total") -> CardCounts(2,2))) + Seq("demo", "App-0", "http_req_total", "timeseries") -> CardCounts(2,2,2))) ).foreach{ testSpec => - val leaves = (0 until shardPartKeyLabelValues.size).map{ ishard => - new TsCardExec(QueryContext(), executeDispatcher, - timeseriesDatasetMultipleShardKeys.ref, ishard, testSpec.shardKeyPrefix, testSpec.numGroupByFields) + val leavesRaw = (0 until shardPartKeyLabelValues.size).map{ ishard => + new TsCardExec(QueryContext(), executeDispatcher,timeseriesDatasetMultipleShardKeys.ref, + ishard, testSpec.shardKeyPrefix, testSpec.numGroupByFields, "raw") + }.toSeq + // UPDATE: Simulating the call to downsample cluster to get longterm metrics as well + val leavesDownsample = (0 until shardPartKeyLabelValues.size).map { ishard => + new TsCardExec(QueryContext(), executeDispatcher, timeseriesDatasetMultipleShardKeys.ref, + ishard, testSpec.shardKeyPrefix, testSpec.numGroupByFields, "downsample") }.toSeq - val execPlan = TsCardReduceExec(QueryContext(), executeDispatcher, leaves) + val allLeaves = leavesRaw ++ leavesDownsample + val execPlan = TsCardReduceExec(QueryContext(), executeDispatcher, allLeaves) val resp = execPlan.execute(memStore, querySession).runToFuture.futureValue val result = (resp: @unchecked) match { diff --git a/query/src/test/scala/filodb/query/exec/RemoteMetadataExecSpec.scala b/query/src/test/scala/filodb/query/exec/RemoteMetadataExecSpec.scala index d6df796433..18d92cdda1 100644 --- a/query/src/test/scala/filodb/query/exec/RemoteMetadataExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/RemoteMetadataExecSpec.scala @@ -247,7 +247,7 @@ class RemoteMetadataExecSpec extends AnyFunSpec with Matchers with ScalaFutures result.toArray shouldEqual jobQueryResult3 } - it ("timeseries cardinality remote exec") { + it ("timeseries cardinality version 1 remote exec") { import TsCardExec._ import TsCardinalities._ @@ -286,5 +286,53 @@ class RemoteMetadataExecSpec extends AnyFunSpec with Matchers with ScalaFutures rows shouldEqual expRows } } + + it("timeseries cardinality version 2 remote exec") { + import TsCardExec._ + import TsCardinalities._ + + val samples = Seq( + TsCardinalitiesSamplV2(Map("_ws_" -> "foo", "_ns_" -> "bar", "__name__" -> "baz"), + Map("active" -> 123, "shortTerm" -> 234, "longTerm" -> 500), "raw", "prometheus"), + TsCardinalitiesSamplV2(Map("_ws_" -> "foo", "_ns_" -> "bar", "__name__" -> "bat"), + Map("active" -> 345, "shortTerm" -> 456, "longTerm" -> 1000), "aggregated", "prometheus_preagg"), + TsCardinalitiesSamplV2(Map("_ws_" -> "foo", "_ns_" -> "bar", "__name__" -> "bak"), + Map("active" -> 567, "shortTerm" -> 678, "longTerm" -> 0), "recordingrules", "prometheus_rules_1m"), + ) + + val testingBackendTsCard: SttpBackend[Future, Nothing] = SttpBackendStub.asynchronousFuture + .whenRequestMatches(_.uri.path.startsWith(List("api", "v2", "metering", "cardinality", "timeseries")) + ) + .thenRespondWrapped(Future { + Response(Right(Right(MetadataSuccessResponse(samples, "success", Option.empty, Option.empty))), + StatusCodes.PartialContent, "", Nil, Nil) + }) + + val exec: MetadataRemoteExec = MetadataRemoteExec( + "http://localhost:31007/api/v2/metering/cardinality/timeseries", 10000L, + Map("match[]" -> """{_ws_="foo", _ns_="bar"}""", "numGroupByFields" -> "3", "verbose" -> "true", + "datasets" -> "longtime-prometheus,longtime-prometheus_preagg,recordingrules-prometheus_rules_1m"), + QueryContext(origQueryParams = PromQlQueryParams("test", 123L, 234L, 15L, + Option("http://localhost:31007/api/v2/metering/cardinality/timeseries"))), + InProcessPlanDispatcher(queryConfig), timeseriesDataset.ref, RemoteHttpClient(configBuilder.build(), + testingBackendTsCard), queryConfig) + + val resp = exec.execute(memStore, querySession).runToFuture.futureValue + val result = (resp: @unchecked) match { + case QueryResult(id, _, response, _, _, _, _) => + // should only contain a single RV where each row describes a single group's cardinalities + response.size shouldEqual 1 + val rows = response.head.rows().map { rr => + RowData.fromRowReader(rr) + }.toSet + val expRows = samples.map { s => + // order the shard keys according to precedence + val prefix = SHARD_KEY_LABELS.map(s.group(_)) + val counts = CardCounts(s.cardinality("active"), s.cardinality("shortTerm"), s.cardinality("longTerm")) + RowData(prefixToGroupWithDataset(prefix, s._type), counts) + }.toSet + rows shouldEqual expRows + } + } } From 594ffce1bc7ee89c7a7ec9a9225c329e58be2d07 Mon Sep 17 00:00:00 2001 From: sandeep6189 Date: Wed, 16 Aug 2023 18:05:32 -0700 Subject: [PATCH 26/39] fix(query): Adding user datasets for Cardinality V2 RemoteMetadataExec calls (#1651) * Adding UserDatasets for remote calls * Updating UT --- .../queryplanner/MultiPartitionPlannerSpec.scala | 5 +++-- query/src/main/scala/filodb/query/LogicalPlan.scala | 6 +++--- query/src/test/scala/filodb/query/LogicalPlanSpec.scala | 5 +++-- 3 files changed, 9 insertions(+), 7 deletions(-) diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala index 5e237b068f..640db472f3 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala @@ -1095,11 +1095,12 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers with PlanValida val engine = new MultiPartitionPlanner(partitionLocationProvider, localPlanner, "local", dataset, queryConfig) - val lp = TsCardinalities(Seq("a", "b"), 3, 2, Seq("longtime-prometheus","recordingrules-prometheus_rules_1m")) + val lp = TsCardinalities(Seq("a", "b"), 3, 2, Seq("longtime-prometheus","recordingrules-prometheus_rules_1m") + , "raw,recordingrules") val promQlQueryParams = PromQlQueryParams("", startSeconds, step, endSeconds, Some("/api/v2/metering/cardinality/timeseries")) val expectedUrlParams = Map("match[]" -> """{_ws_="a",_ns_="b"}""", "numGroupByFields" -> "3","verbose" -> "true", - "datasets" -> "longtime-prometheus,recordingrules-prometheus_rules_1m") + "datasets" -> "raw,recordingrules") val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams, plannerParams = PlannerParams(processMultiPartition = true))) diff --git a/query/src/main/scala/filodb/query/LogicalPlan.scala b/query/src/main/scala/filodb/query/LogicalPlan.scala index 40f8961c00..cb33cc11ef 100644 --- a/query/src/main/scala/filodb/query/LogicalPlan.scala +++ b/query/src/main/scala/filodb/query/LogicalPlan.scala @@ -2,7 +2,6 @@ package filodb.query import filodb.core.query.{ColumnFilter, RangeParams, RvRange} import filodb.core.query.Filter.Equals -import filodb.query.exec.TsCardExec //scalastyle:off number.of.types //scalastyle:off file.size.limit @@ -185,7 +184,8 @@ object TsCardinalities { case class TsCardinalities(shardKeyPrefix: Seq[String], numGroupByFields: Int, version: Int = 1, - datasets: Seq[String] = Seq()) extends LogicalPlan { + datasets: Seq[String] = Seq(), + userDatasets: String = "") extends LogicalPlan { import TsCardinalities._ require(numGroupByFields >= 1 && numGroupByFields <= 3, @@ -212,7 +212,7 @@ case class TsCardinalities(shardKeyPrefix: Seq[String], .mkString(",") + "}"), "numGroupByFields" -> numGroupByFields.toString, "verbose" -> "true", // Using this plan to determine if we need to pass additional values in groupKey or not - "datasets" -> datasets.mkString(TsCardExec.PREFIX_DELIM) + "datasets" -> userDatasets ) } } diff --git a/query/src/test/scala/filodb/query/LogicalPlanSpec.scala b/query/src/test/scala/filodb/query/LogicalPlanSpec.scala index e44fd6fa98..67decf5150 100644 --- a/query/src/test/scala/filodb/query/LogicalPlanSpec.scala +++ b/query/src/test/scala/filodb/query/LogicalPlanSpec.scala @@ -327,11 +327,12 @@ class LogicalPlanSpec extends AnyFunSpec with Matchers { it ("TsCardinalities queryParams should have expected values") { val datasets = Seq("longtime-prometheus", "recordingrules-prometheus_rules_longterm") - val plan = TsCardinalities(Seq("a","b","c"), 3, 2, datasets) + val userDatasets = "\"raw\",\"recordingrules\"" + val plan = TsCardinalities(Seq("a","b","c"), 3, 2, datasets, userDatasets) val queryParamsMap = plan.queryParams() queryParamsMap.get("numGroupByFields").get shouldEqual "3" - queryParamsMap.get("datasets").get shouldEqual datasets.mkString(",") + queryParamsMap.get("datasets").get shouldEqual userDatasets queryParamsMap.get("verbose").get shouldEqual "true" queryParamsMap.get("match[]").get shouldEqual "{_ws_=\"a\",_ns_=\"b\",__name__=\"c\"}" } From 89bd6788c6a1ef6dbe1b0b00e62d722c60e6c1fd Mon Sep 17 00:00:00 2001 From: sandeep6189 Date: Fri, 18 Aug 2023 09:47:15 -0700 Subject: [PATCH 27/39] Fix MultiPartition Card Queries (#1652) --- .../queryplanner/MultiPartitionPlanner.scala | 3 ++- .../queryplanner/SingleClusterPlanner.scala | 3 ++- .../filodb/query/exec/MetadataExecPlan.scala | 17 ++++++++++++++--- .../filodb/query/exec/MetadataExecSpec.scala | 4 ++-- 4 files changed, 20 insertions(+), 7 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala index 60eba29b63..f3c7a46d99 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala @@ -619,7 +619,8 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider if (p.partitionName.equals(localPartitionName)) localPartitionPlanner.materialize(lp, qContext) else { - createMetadataRemoteExec(qContext, p, lp.queryParams()) + val newQueryContext = qContext.copy(origQueryParams = queryParams.copy(verbose = true)) + createMetadataRemoteExec(newQueryContext, p, lp.queryParams()) } } if (execPlans.size == 1) { diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala index 4b31c7b209..e0b581b7f8 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala @@ -832,7 +832,8 @@ class SingleClusterPlanner(val dataset: Dataset, forceInProcess: Boolean): PlanResult = { val metaExec = shardMapperFunc.assignedShards.map{ shard => val dispatcher = dispatcherForShard(shard, forceInProcess, qContext) - exec.TsCardExec(qContext, dispatcher, dsRef, shard, lp.shardKeyPrefix, lp.numGroupByFields, clusterName) + exec.TsCardExec(qContext, dispatcher, dsRef, shard, lp.shardKeyPrefix, lp.numGroupByFields, clusterName, + lp.version) } PlanResult(metaExec) } diff --git a/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala b/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala index 9e8fec9ce5..542092c7ac 100644 --- a/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala +++ b/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala @@ -558,7 +558,8 @@ final case class TsCardExec(queryContext: QueryContext, shard: Int, shardKeyPrefix: Seq[String], numGroupByFields: Int, - clusterName: String) extends LeafExecPlan with StrictLogging { + clusterName: String, + version: Int) extends LeafExecPlan with StrictLogging { require(numGroupByFields >= 1, "numGroupByFields must be positive") require(numGroupByFields >= shardKeyPrefix.size, @@ -584,8 +585,14 @@ final case class TsCardExec(queryContext: QueryContext, val cards = tsMemStore.scanTsCardinalities( dataset, Seq(shard), shardKeyPrefix, numGroupByFields) val it = cards.map { card => - val groupKey = prefixToGroupWithDataset(card.prefix, dataset.dataset) + // v1 and v2 cardinality have different schemas and required group key. Hence we are segregating + // w.r.t to the version + val groupKey = + version match { + case 1 => prefixToGroup(card.prefix) + case _ => prefixToGroupWithDataset(card.prefix, dataset.dataset) + } // NOTE: cardinality data from downsample cluster is stored as total count in CardinalityStore. But for the // user perspective, the cardinality data in downsample is a longterm data. Hence, we are forking the // data path based on the cluster the data is being served from @@ -605,7 +612,11 @@ final case class TsCardExec(queryContext: QueryContext, case other => Observable.empty } - ExecResult(rvs, Task.eval(RESULT_SCHEMA)) + // Sending V1 SCHEMA for v1 queries + version match { + case 1 => ExecResult(rvs, Task.eval(RESULT_SCHEMA_V1)) + case _ => ExecResult(rvs, Task.eval(RESULT_SCHEMA)) + } } // scalastyle:on method.length diff --git a/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala b/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala index 556b6a9221..84da458bb7 100644 --- a/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala @@ -406,12 +406,12 @@ class MetadataExecSpec extends AnyFunSpec with Matchers with ScalaFutures with B val leavesRaw = (0 until shardPartKeyLabelValues.size).map{ ishard => new TsCardExec(QueryContext(), executeDispatcher,timeseriesDatasetMultipleShardKeys.ref, - ishard, testSpec.shardKeyPrefix, testSpec.numGroupByFields, "raw") + ishard, testSpec.shardKeyPrefix, testSpec.numGroupByFields, "raw", 2) }.toSeq // UPDATE: Simulating the call to downsample cluster to get longterm metrics as well val leavesDownsample = (0 until shardPartKeyLabelValues.size).map { ishard => new TsCardExec(QueryContext(), executeDispatcher, timeseriesDatasetMultipleShardKeys.ref, - ishard, testSpec.shardKeyPrefix, testSpec.numGroupByFields, "downsample") + ishard, testSpec.shardKeyPrefix, testSpec.numGroupByFields, "downsample", 2) }.toSeq val allLeaves = leavesRaw ++ leavesDownsample From bdcfde70054f102e11eb835037087c32293b9d58 Mon Sep 17 00:00:00 2001 From: sandeep6189 Date: Wed, 16 Aug 2023 18:05:32 -0700 Subject: [PATCH 28/39] fix(query): Adding user datasets for Cardinality V2 RemoteMetadataExec calls (#1651) * Adding UserDatasets for remote calls * Updating UT --- .../queryplanner/MultiPartitionPlannerSpec.scala | 5 +++-- query/src/main/scala/filodb/query/LogicalPlan.scala | 6 +++--- query/src/test/scala/filodb/query/LogicalPlanSpec.scala | 5 +++-- 3 files changed, 9 insertions(+), 7 deletions(-) diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala index 5e237b068f..640db472f3 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala @@ -1095,11 +1095,12 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers with PlanValida val engine = new MultiPartitionPlanner(partitionLocationProvider, localPlanner, "local", dataset, queryConfig) - val lp = TsCardinalities(Seq("a", "b"), 3, 2, Seq("longtime-prometheus","recordingrules-prometheus_rules_1m")) + val lp = TsCardinalities(Seq("a", "b"), 3, 2, Seq("longtime-prometheus","recordingrules-prometheus_rules_1m") + , "raw,recordingrules") val promQlQueryParams = PromQlQueryParams("", startSeconds, step, endSeconds, Some("/api/v2/metering/cardinality/timeseries")) val expectedUrlParams = Map("match[]" -> """{_ws_="a",_ns_="b"}""", "numGroupByFields" -> "3","verbose" -> "true", - "datasets" -> "longtime-prometheus,recordingrules-prometheus_rules_1m") + "datasets" -> "raw,recordingrules") val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams, plannerParams = PlannerParams(processMultiPartition = true))) diff --git a/query/src/main/scala/filodb/query/LogicalPlan.scala b/query/src/main/scala/filodb/query/LogicalPlan.scala index 40f8961c00..cb33cc11ef 100644 --- a/query/src/main/scala/filodb/query/LogicalPlan.scala +++ b/query/src/main/scala/filodb/query/LogicalPlan.scala @@ -2,7 +2,6 @@ package filodb.query import filodb.core.query.{ColumnFilter, RangeParams, RvRange} import filodb.core.query.Filter.Equals -import filodb.query.exec.TsCardExec //scalastyle:off number.of.types //scalastyle:off file.size.limit @@ -185,7 +184,8 @@ object TsCardinalities { case class TsCardinalities(shardKeyPrefix: Seq[String], numGroupByFields: Int, version: Int = 1, - datasets: Seq[String] = Seq()) extends LogicalPlan { + datasets: Seq[String] = Seq(), + userDatasets: String = "") extends LogicalPlan { import TsCardinalities._ require(numGroupByFields >= 1 && numGroupByFields <= 3, @@ -212,7 +212,7 @@ case class TsCardinalities(shardKeyPrefix: Seq[String], .mkString(",") + "}"), "numGroupByFields" -> numGroupByFields.toString, "verbose" -> "true", // Using this plan to determine if we need to pass additional values in groupKey or not - "datasets" -> datasets.mkString(TsCardExec.PREFIX_DELIM) + "datasets" -> userDatasets ) } } diff --git a/query/src/test/scala/filodb/query/LogicalPlanSpec.scala b/query/src/test/scala/filodb/query/LogicalPlanSpec.scala index e44fd6fa98..67decf5150 100644 --- a/query/src/test/scala/filodb/query/LogicalPlanSpec.scala +++ b/query/src/test/scala/filodb/query/LogicalPlanSpec.scala @@ -327,11 +327,12 @@ class LogicalPlanSpec extends AnyFunSpec with Matchers { it ("TsCardinalities queryParams should have expected values") { val datasets = Seq("longtime-prometheus", "recordingrules-prometheus_rules_longterm") - val plan = TsCardinalities(Seq("a","b","c"), 3, 2, datasets) + val userDatasets = "\"raw\",\"recordingrules\"" + val plan = TsCardinalities(Seq("a","b","c"), 3, 2, datasets, userDatasets) val queryParamsMap = plan.queryParams() queryParamsMap.get("numGroupByFields").get shouldEqual "3" - queryParamsMap.get("datasets").get shouldEqual datasets.mkString(",") + queryParamsMap.get("datasets").get shouldEqual userDatasets queryParamsMap.get("verbose").get shouldEqual "true" queryParamsMap.get("match[]").get shouldEqual "{_ws_=\"a\",_ns_=\"b\",__name__=\"c\"}" } From 17fb07767028723e0522f16d57a52c6bb986afea Mon Sep 17 00:00:00 2001 From: sandeep6189 Date: Fri, 18 Aug 2023 09:47:15 -0700 Subject: [PATCH 29/39] Fix MultiPartition Card Queries (#1652) --- .../queryplanner/MultiPartitionPlanner.scala | 3 ++- .../queryplanner/SingleClusterPlanner.scala | 3 ++- .../filodb/query/exec/MetadataExecPlan.scala | 17 ++++++++++++++--- .../filodb/query/exec/MetadataExecSpec.scala | 4 ++-- 4 files changed, 20 insertions(+), 7 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala index 60eba29b63..f3c7a46d99 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala @@ -619,7 +619,8 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider if (p.partitionName.equals(localPartitionName)) localPartitionPlanner.materialize(lp, qContext) else { - createMetadataRemoteExec(qContext, p, lp.queryParams()) + val newQueryContext = qContext.copy(origQueryParams = queryParams.copy(verbose = true)) + createMetadataRemoteExec(newQueryContext, p, lp.queryParams()) } } if (execPlans.size == 1) { diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala index 4b31c7b209..e0b581b7f8 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala @@ -832,7 +832,8 @@ class SingleClusterPlanner(val dataset: Dataset, forceInProcess: Boolean): PlanResult = { val metaExec = shardMapperFunc.assignedShards.map{ shard => val dispatcher = dispatcherForShard(shard, forceInProcess, qContext) - exec.TsCardExec(qContext, dispatcher, dsRef, shard, lp.shardKeyPrefix, lp.numGroupByFields, clusterName) + exec.TsCardExec(qContext, dispatcher, dsRef, shard, lp.shardKeyPrefix, lp.numGroupByFields, clusterName, + lp.version) } PlanResult(metaExec) } diff --git a/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala b/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala index 9e8fec9ce5..542092c7ac 100644 --- a/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala +++ b/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala @@ -558,7 +558,8 @@ final case class TsCardExec(queryContext: QueryContext, shard: Int, shardKeyPrefix: Seq[String], numGroupByFields: Int, - clusterName: String) extends LeafExecPlan with StrictLogging { + clusterName: String, + version: Int) extends LeafExecPlan with StrictLogging { require(numGroupByFields >= 1, "numGroupByFields must be positive") require(numGroupByFields >= shardKeyPrefix.size, @@ -584,8 +585,14 @@ final case class TsCardExec(queryContext: QueryContext, val cards = tsMemStore.scanTsCardinalities( dataset, Seq(shard), shardKeyPrefix, numGroupByFields) val it = cards.map { card => - val groupKey = prefixToGroupWithDataset(card.prefix, dataset.dataset) + // v1 and v2 cardinality have different schemas and required group key. Hence we are segregating + // w.r.t to the version + val groupKey = + version match { + case 1 => prefixToGroup(card.prefix) + case _ => prefixToGroupWithDataset(card.prefix, dataset.dataset) + } // NOTE: cardinality data from downsample cluster is stored as total count in CardinalityStore. But for the // user perspective, the cardinality data in downsample is a longterm data. Hence, we are forking the // data path based on the cluster the data is being served from @@ -605,7 +612,11 @@ final case class TsCardExec(queryContext: QueryContext, case other => Observable.empty } - ExecResult(rvs, Task.eval(RESULT_SCHEMA)) + // Sending V1 SCHEMA for v1 queries + version match { + case 1 => ExecResult(rvs, Task.eval(RESULT_SCHEMA_V1)) + case _ => ExecResult(rvs, Task.eval(RESULT_SCHEMA)) + } } // scalastyle:on method.length diff --git a/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala b/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala index 58c5c0b294..1f19b21fd8 100644 --- a/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala @@ -406,12 +406,12 @@ class MetadataExecSpec extends AnyFunSpec with Matchers with ScalaFutures with B val leavesRaw = (0 until shardPartKeyLabelValues.size).map{ ishard => new TsCardExec(QueryContext(), executeDispatcher,timeseriesDatasetMultipleShardKeys.ref, - ishard, testSpec.shardKeyPrefix, testSpec.numGroupByFields, "raw") + ishard, testSpec.shardKeyPrefix, testSpec.numGroupByFields, "raw", 2) }.toSeq // UPDATE: Simulating the call to downsample cluster to get longterm metrics as well val leavesDownsample = (0 until shardPartKeyLabelValues.size).map { ishard => new TsCardExec(QueryContext(), executeDispatcher, timeseriesDatasetMultipleShardKeys.ref, - ishard, testSpec.shardKeyPrefix, testSpec.numGroupByFields, "downsample") + ishard, testSpec.shardKeyPrefix, testSpec.numGroupByFields, "downsample", 2) }.toSeq val allLeaves = leavesRaw ++ leavesDownsample From 89095e22e2d2471858f1a43e27a5d4c1a571d058 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Mon, 21 Aug 2023 12:17:20 -0700 Subject: [PATCH 30/39] feat(core): Add Query CPU Time for Index Lookups (#1655) Adding CPU Nanos Time consumed for Lucene index lookups in query stats, especially since we are seeing this in frequently as a hotspot in CPU profile information --- .../DownsampledTimeSeriesShard.scala | 13 ++-- .../memstore/TimeSeriesShard.scala | 11 +++- .../aggregator/CountValuesRowAggregator.scala | 59 +++++++++++-------- 3 files changed, 49 insertions(+), 34 deletions(-) diff --git a/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesShard.scala b/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesShard.scala index 7f3aca62e0..66174a3582 100644 --- a/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesShard.scala @@ -1,13 +1,14 @@ package filodb.core.downsample +import java.util +import java.util.concurrent.atomic.AtomicLong + import scala.collection.mutable import scala.concurrent.{ExecutionContext, Future} import scala.concurrent.duration._ import com.typesafe.config.Config import com.typesafe.scalalogging.StrictLogging -import java.util -import java.util.concurrent.atomic.AtomicLong import kamon.Kamon import kamon.metric.MeasurementUnit import kamon.tag.TagSet @@ -16,12 +17,12 @@ import monix.execution.{CancelableFuture, Scheduler, UncaughtExceptionReporter} import monix.reactive.Observable import org.apache.lucene.search.CollectionTerminatedException -import filodb.core.{DatasetRef, Types} +import filodb.core.{DatasetRef, Types, Utils} import filodb.core.binaryrecord2.RecordSchema import filodb.core.memstore._ import filodb.core.memstore.ratelimit.{CardinalityManager, CardinalityRecord, QuotaSource} import filodb.core.metadata.Schemas -import filodb.core.query.{ColumnFilter, Filter, QueryContext, QueryLimitException, QuerySession, QueryWarnings} +import filodb.core.query._ import filodb.core.store._ import filodb.memory.format.{UnsafeUtils, ZeroCopyUTF8String} import filodb.memory.format.ZeroCopyUTF8String._ @@ -341,6 +342,7 @@ class DownsampledTimeSeriesShard(rawDatasetRef: DatasetRef, // Second iteration is for query result evaluation. Loading everything to heap // is expensive, but we do it to handle data sizing for metrics that have // continuous churn. See capDataScannedPerShardCheck method. + val startNs = Utils.currentThreadCpuTimeNanos val recs = partKeyIndex.partKeyRecordsFromFilters(filters, chunkMethod.startTime, chunkMethod.endTime) val _schema = recs.headOption.map { pkRec => RecordSchema.schemaID(pkRec.partKey, UnsafeUtils.arayOffset) @@ -353,9 +355,10 @@ class DownsampledTimeSeriesShard(rawDatasetRef: DatasetRef, val metricGroupBy = deploymentPartitionName +: clusterType +: metricShardKeys.map { col => filters.collectFirst { case ColumnFilter(c, Filter.Equals(filtVal: String)) if c == col => filtVal - }.getOrElse("unknown") + }.getOrElse("multiple") }.toList querySession.queryStats.getTimeSeriesScannedCounter(metricGroupBy).addAndGet(recs.length) + querySession.queryStats.getCpuNanosCounter(metricGroupBy).addAndGet(Utils.currentThreadCpuTimeNanos - startNs) val chunksReadCounter = querySession.queryStats.getDataBytesScannedCounter(metricGroupBy) PartLookupResult(shardNum, chunkMethod, debox.Buffer.empty, diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala index 045712758e..4293fef734 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala @@ -1769,15 +1769,18 @@ class TimeSeriesShard(val ref: DatasetRef, startTime: Long, querySession: QuerySession, limit: Int): Iterator[ZeroCopyUTF8String] = { - if (indexFacetingEnabledAllLabels || + val metricShardKeys = schemas.part.options.shardKeyColumns + val metricGroupBy = deploymentPartitionName +: clusterType +: shardKeyValuesFromFilter(metricShardKeys, filters) + val startNs = Utils.currentThreadCpuTimeNanos + val res = if (indexFacetingEnabledAllLabels || (indexFacetingEnabledShardKeyLabels && schemas.part.options.shardKeyColumns.contains(label))) { partKeyIndex.labelValuesEfficient(filters, startTime, endTime, label, limit).iterator.map(_.utf8) } else { - val metricShardKeys = schemas.part.options.shardKeyColumns - val metricGroupBy = deploymentPartitionName +: clusterType +: shardKeyValuesFromFilter(metricShardKeys, filters) SingleLabelValuesResultIterator(partKeyIndex.partIdsFromFilters(filters, startTime, endTime), label, querySession, metricGroupBy, limit) } + querySession.queryStats.getCpuNanosCounter(metricGroupBy).addAndGet(startNs - Utils.currentThreadCpuTimeNanos) + res } /** @@ -1934,9 +1937,11 @@ class TimeSeriesShard(val ref: DatasetRef, val chunksReadCounter = querySession.queryStats.getDataBytesScannedCounter(metricGroupBy) // No matter if there are filters or not, need to run things through Lucene so we can discover potential // TSPartitions to read back from disk + val startNs = Utils.currentThreadCpuTimeNanos val matches = partKeyIndex.partIdsFromFilters(filters, chunkMethod.startTime, chunkMethod.endTime) shardStats.queryTimeRangeMins.record((chunkMethod.endTime - chunkMethod.startTime) / 60000 ) querySession.queryStats.getTimeSeriesScannedCounter(metricGroupBy).addAndGet(matches.length) + querySession.queryStats.getCpuNanosCounter(metricGroupBy).addAndGet(Utils.currentThreadCpuTimeNanos - startNs) Kamon.currentSpan().tag(s"num-partitions-from-index-$shardNum", matches.length) // first find out which partitions are being queried for data not in memory diff --git a/query/src/main/scala/filodb/query/exec/aggregator/CountValuesRowAggregator.scala b/query/src/main/scala/filodb/query/exec/aggregator/CountValuesRowAggregator.scala index eed6afdd51..22b84ffbed 100644 --- a/query/src/main/scala/filodb/query/exec/aggregator/CountValuesRowAggregator.scala +++ b/query/src/main/scala/filodb/query/exec/aggregator/CountValuesRowAggregator.scala @@ -2,6 +2,7 @@ package filodb.query.exec.aggregator import scala.collection.mutable +import filodb.core.Utils import filodb.core.binaryrecord2.RecordBuilder import filodb.core.memstore.FiloSchedulers import filodb.core.memstore.FiloSchedulers.QuerySchedName @@ -83,36 +84,42 @@ class CountValuesRowAggregator(label: String, limit: Int = 1000) extends RowAggr def present(aggRangeVector: RangeVector, limit: Int, rangeParams: RangeParams, queryStats: QueryStats): Seq[RangeVector] = { - val colSchema = Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), - ColumnInfo("value", ColumnType.DoubleColumn)) - val recSchema = SerializedRangeVector.toSchema(colSchema) - val resRvs = mutable.Map[RangeVectorKey, RecordBuilder]() + val startNs = Utils.currentThreadCpuTimeNanos try { - FiloSchedulers.assertThreadName(QuerySchedName) - // aggRangeVector.rows.take below triggers the ChunkInfoIterator which requires lock/release - ChunkMap.validateNoSharedLocks(s"CountValues-$label") - aggRangeVector.rows.take(limit).foreach { row => - val rowMap = CountValuesSerDeser.deserialize(row.getBlobBase(1), - row.getBlobNumBytes(1), row.getBlobOffset(1)) - rowMap.foreach { (k, v) => - val rvk = CustomRangeVectorKey(aggRangeVector.key.labelValues + - (label.utf8 -> k.toString.utf8)) - val builder = resRvs.getOrElseUpdate(rvk, SerializedRangeVector.newBuilder()) - builder.startNewRecord(recSchema) - builder.addLong(row.getLong(0)) - builder.addDouble(v) - builder.endRecord() + val colSchema = Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), + ColumnInfo("value", ColumnType.DoubleColumn)) + val recSchema = SerializedRangeVector.toSchema(colSchema) + val resRvs = mutable.Map[RangeVectorKey, RecordBuilder]() + try { + FiloSchedulers.assertThreadName(QuerySchedName) + // aggRangeVector.rows.take below triggers the ChunkInfoIterator which requires lock/release + ChunkMap.validateNoSharedLocks(s"CountValues-$label") + aggRangeVector.rows.take(limit).foreach { row => + val rowMap = CountValuesSerDeser.deserialize(row.getBlobBase(1), + row.getBlobNumBytes(1), row.getBlobOffset(1)) + rowMap.foreach { (k, v) => + val rvk = CustomRangeVectorKey(aggRangeVector.key.labelValues + + (label.utf8 -> k.toString.utf8)) + val builder = resRvs.getOrElseUpdate(rvk, SerializedRangeVector.newBuilder()) + builder.startNewRecord(recSchema) + builder.addLong(row.getLong(0)) + builder.addDouble(v) + builder.endRecord() + } } + } finally { + aggRangeVector.rows.close() + ChunkMap.releaseAllSharedLocks() } + resRvs.map { case (key, builder) => + val numRows = builder.allContainers.map(_.countRecords()).sum + val srv = new SerializedRangeVector(key, numRows, builder.allContainers, recSchema, 0, None) + queryStats.getResultBytesCounter(Nil).getAndAdd(srv.estimatedSerializedBytes) + srv + }.toSeq + } finally { + queryStats.getCpuNanosCounter(Nil).getAndAdd(Utils.currentThreadCpuTimeNanos - startNs) } - finally { - aggRangeVector.rows.close() - ChunkMap.releaseAllSharedLocks() - } - resRvs.map { case (key, builder) => - val numRows = builder.allContainers.map(_.countRecords()).sum - new SerializedRangeVector(key, numRows, builder.allContainers, recSchema, 0, None) - }.toSeq } def reductionSchema(source: ResultSchema): ResultSchema = { From 1e56ef9d8679217086fb52042aea1f63f94ef5d5 Mon Sep 17 00:00:00 2001 From: sandeep6189 Date: Wed, 23 Aug 2023 06:29:58 -0700 Subject: [PATCH 31/39] fix(metering): Overriding the cluster name .passed to SingleClusterPlanner for TenantIngestionMetering changes --- .../filodb.coordinator/TenantIngestionMetering.scala | 2 +- .../queryplanner/SingleClusterPlanner.scala | 8 +++++++- query/src/main/scala/filodb/query/LogicalPlan.scala | 3 ++- 3 files changed, 10 insertions(+), 3 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/TenantIngestionMetering.scala b/coordinator/src/main/scala/filodb.coordinator/TenantIngestionMetering.scala index 3a2b737d59..50c7a3ca1d 100644 --- a/coordinator/src/main/scala/filodb.coordinator/TenantIngestionMetering.scala +++ b/coordinator/src/main/scala/filodb.coordinator/TenantIngestionMetering.scala @@ -65,7 +65,7 @@ case class TenantIngestionMetering(settings: FilodbSettings, dsIterProducer().foreach { dsRef => val fut = Client.asyncAsk( coordActorProducer(), - LogicalPlan2Query(dsRef, TsCardinalities(prefix, numGroupByFields)), + LogicalPlan2Query(dsRef, TsCardinalities(prefix, numGroupByFields, 2, overrideClusterName = CLUSTER_TYPE)), ASK_TIMEOUT) fut.onComplete { case Success(QueryResult(_, _, rv, _, _, _, _)) => diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala index e0b581b7f8..abc0cf2827 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala @@ -830,9 +830,15 @@ class SingleClusterPlanner(val dataset: Dataset, private def materializeTsCardinalities(qContext: QueryContext, lp: TsCardinalities, forceInProcess: Boolean): PlanResult = { + // If no clusterName is passed in the logical plan, we use the passed clusterName in the SingleClusterPlanner + // We are using the passed cluster name in logical plan for tenant metering apis + val clusterNameToPass = lp.overrideClusterName match { + case "" => clusterName + case _ => lp.overrideClusterName + } val metaExec = shardMapperFunc.assignedShards.map{ shard => val dispatcher = dispatcherForShard(shard, forceInProcess, qContext) - exec.TsCardExec(qContext, dispatcher, dsRef, shard, lp.shardKeyPrefix, lp.numGroupByFields, clusterName, + exec.TsCardExec(qContext, dispatcher, dsRef, shard, lp.shardKeyPrefix, lp.numGroupByFields, clusterNameToPass, lp.version) } PlanResult(metaExec) diff --git a/query/src/main/scala/filodb/query/LogicalPlan.scala b/query/src/main/scala/filodb/query/LogicalPlan.scala index cb33cc11ef..e6740e758c 100644 --- a/query/src/main/scala/filodb/query/LogicalPlan.scala +++ b/query/src/main/scala/filodb/query/LogicalPlan.scala @@ -185,7 +185,8 @@ case class TsCardinalities(shardKeyPrefix: Seq[String], numGroupByFields: Int, version: Int = 1, datasets: Seq[String] = Seq(), - userDatasets: String = "") extends LogicalPlan { + userDatasets: String = "", + overrideClusterName: String = "") extends LogicalPlan { import TsCardinalities._ require(numGroupByFields >= 1 && numGroupByFields <= 3, From c77a95ad6cebdb26580763cb8e1b9f2e94f9273b Mon Sep 17 00:00:00 2001 From: sandeep6189 Date: Wed, 23 Aug 2023 11:49:09 -0700 Subject: [PATCH 32/39] fix(metering): Overriding the cluster name .passed to SingleClusterPlanner for TenantIngestionMetering changes (#1659) --- .../filodb.coordinator/TenantIngestionMetering.scala | 2 +- .../queryplanner/SingleClusterPlanner.scala | 8 +++++++- query/src/main/scala/filodb/query/LogicalPlan.scala | 3 ++- 3 files changed, 10 insertions(+), 3 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/TenantIngestionMetering.scala b/coordinator/src/main/scala/filodb.coordinator/TenantIngestionMetering.scala index 3a2b737d59..50c7a3ca1d 100644 --- a/coordinator/src/main/scala/filodb.coordinator/TenantIngestionMetering.scala +++ b/coordinator/src/main/scala/filodb.coordinator/TenantIngestionMetering.scala @@ -65,7 +65,7 @@ case class TenantIngestionMetering(settings: FilodbSettings, dsIterProducer().foreach { dsRef => val fut = Client.asyncAsk( coordActorProducer(), - LogicalPlan2Query(dsRef, TsCardinalities(prefix, numGroupByFields)), + LogicalPlan2Query(dsRef, TsCardinalities(prefix, numGroupByFields, 2, overrideClusterName = CLUSTER_TYPE)), ASK_TIMEOUT) fut.onComplete { case Success(QueryResult(_, _, rv, _, _, _, _)) => diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala index e0b581b7f8..abc0cf2827 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala @@ -830,9 +830,15 @@ class SingleClusterPlanner(val dataset: Dataset, private def materializeTsCardinalities(qContext: QueryContext, lp: TsCardinalities, forceInProcess: Boolean): PlanResult = { + // If no clusterName is passed in the logical plan, we use the passed clusterName in the SingleClusterPlanner + // We are using the passed cluster name in logical plan for tenant metering apis + val clusterNameToPass = lp.overrideClusterName match { + case "" => clusterName + case _ => lp.overrideClusterName + } val metaExec = shardMapperFunc.assignedShards.map{ shard => val dispatcher = dispatcherForShard(shard, forceInProcess, qContext) - exec.TsCardExec(qContext, dispatcher, dsRef, shard, lp.shardKeyPrefix, lp.numGroupByFields, clusterName, + exec.TsCardExec(qContext, dispatcher, dsRef, shard, lp.shardKeyPrefix, lp.numGroupByFields, clusterNameToPass, lp.version) } PlanResult(metaExec) diff --git a/query/src/main/scala/filodb/query/LogicalPlan.scala b/query/src/main/scala/filodb/query/LogicalPlan.scala index cb33cc11ef..e6740e758c 100644 --- a/query/src/main/scala/filodb/query/LogicalPlan.scala +++ b/query/src/main/scala/filodb/query/LogicalPlan.scala @@ -185,7 +185,8 @@ case class TsCardinalities(shardKeyPrefix: Seq[String], numGroupByFields: Int, version: Int = 1, datasets: Seq[String] = Seq(), - userDatasets: String = "") extends LogicalPlan { + userDatasets: String = "", + overrideClusterName: String = "") extends LogicalPlan { import TsCardinalities._ require(numGroupByFields >= 1 && numGroupByFields <= 3, From 710c3d2c2e9ba0504555cc1e5628d865c1321cf2 Mon Sep 17 00:00:00 2001 From: Shaik Sher Ali Date: Thu, 24 Aug 2023 22:23:16 -0700 Subject: [PATCH 33/39] misc(core): add downsample support for aggregated data (#1661) --- core/src/main/resources/filodb-defaults.conf | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/core/src/main/resources/filodb-defaults.conf b/core/src/main/resources/filodb-defaults.conf index 5112af6ecc..1d92e10964 100644 --- a/core/src/main/resources/filodb-defaults.conf +++ b/core/src/main/resources/filodb-defaults.conf @@ -196,7 +196,9 @@ filodb { "max:double:detectDrops=false", ] value-column = "sum" - downsamplers = [] // downsampling is disabled by default + downsample-period-marker = "time(0)" + downsamplers = ["tTime(0)", "dSum(1)", "dMin(2)", "dSum(3)", "dMax(4)"] + downsample-schema = "preagg-gauge" } preagg-delta-counter { columns = [ @@ -207,7 +209,9 @@ filodb { "max:double:detectDrops=false", ] value-column = "sum" - downsamplers = [] // downsampling is disabled by default + downsample-period-marker = "time(0)" + downsamplers = ["tTime(0)", "dSum(1)", "dMin(2)", "dSum(3)", "dMax(4)"] + downsample-schema = "preagg-delta-counter" } preagg-delta-histogram { columns = ["timestamp:ts", @@ -216,7 +220,9 @@ filodb { "tscount:double:{detectDrops=false,delta=true}" "h:hist:{counter=false,delta=true}"] value-column = "h" - downsamplers = [] + downsample-period-marker = "time(0)" + downsamplers = ["tTime(0)", "dSum(1)", "dSum(2)", "dSum(3)", "hSum(4)"] + downsample-schema = "preagg-delta-histogram" } # Used for downsampled gauge data From df3922ddb5bb47de72cfb8dc6458caac405ebe26 Mon Sep 17 00:00:00 2001 From: Shaik Sher Ali Date: Thu, 24 Aug 2023 22:23:16 -0700 Subject: [PATCH 34/39] misc(core): add downsample support for aggregated data (#1661) --- core/src/main/resources/filodb-defaults.conf | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/core/src/main/resources/filodb-defaults.conf b/core/src/main/resources/filodb-defaults.conf index f55e234594..e2cac29f02 100644 --- a/core/src/main/resources/filodb-defaults.conf +++ b/core/src/main/resources/filodb-defaults.conf @@ -191,7 +191,9 @@ filodb { "max:double:detectDrops=false", ] value-column = "sum" - downsamplers = [] // downsampling is disabled by default + downsample-period-marker = "time(0)" + downsamplers = ["tTime(0)", "dSum(1)", "dMin(2)", "dSum(3)", "dMax(4)"] + downsample-schema = "preagg-gauge" } preagg-delta-counter { columns = [ @@ -202,7 +204,9 @@ filodb { "max:double:detectDrops=false", ] value-column = "sum" - downsamplers = [] // downsampling is disabled by default + downsample-period-marker = "time(0)" + downsamplers = ["tTime(0)", "dSum(1)", "dMin(2)", "dSum(3)", "dMax(4)"] + downsample-schema = "preagg-delta-counter" } preagg-delta-histogram { columns = ["timestamp:ts", @@ -211,7 +215,9 @@ filodb { "tscount:double:{detectDrops=false,delta=true}" "h:hist:{counter=false,delta=true}"] value-column = "h" - downsamplers = [] + downsample-period-marker = "time(0)" + downsamplers = ["tTime(0)", "dSum(1)", "dSum(2)", "dSum(3)", "hSum(4)"] + downsample-schema = "preagg-delta-histogram" } # Used for downsampled gauge data From 6cb5433b202239fd1ac4baaf201468532444e8a2 Mon Sep 17 00:00:00 2001 From: alextheimer Date: Wed, 30 Aug 2023 15:04:58 -0700 Subject: [PATCH 35/39] maint(core): upgrade to Lucene 9.7.0 (#1662) --- .../memstore/PartKeyLuceneIndex.scala | 50 ++++++++++++------- project/Dependencies.scala | 4 +- 2 files changed, 33 insertions(+), 21 deletions(-) diff --git a/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala b/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala index aa1b66dc72..50b552f7ef 100644 --- a/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala +++ b/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala @@ -24,6 +24,7 @@ import org.apache.lucene.analysis.standard.StandardAnalyzer import org.apache.lucene.document._ import org.apache.lucene.document.Field.Store import org.apache.lucene.facet.{FacetsCollector, FacetsConfig} +import org.apache.lucene.facet.FacetsConfig.DrillDownTermsIndexing import org.apache.lucene.facet.sortedset.{SortedSetDocValuesFacetCounts, SortedSetDocValuesFacetField} import org.apache.lucene.facet.sortedset.DefaultSortedSetDocValuesReaderState import org.apache.lucene.index._ @@ -46,7 +47,11 @@ import filodb.memory.{BinaryRegionLarge, UTF8StringMedium, UTF8StringShort} import filodb.memory.format.{UnsafeUtils, ZeroCopyUTF8String => UTF8Str} object PartKeyLuceneIndex { - final val PART_ID = "__partId__" + // NOTE: these partId fields need to be separate because Lucene 9.7.0 enforces consistent types for document + // field values (i.e. a field cannot have both numeric and string values). Additional details can be found + // here: https://github.com/apache/lucene/pull/11 + final val PART_ID_DV = "__partIdDv__" + final val PART_ID_FIELD = "__partIdField__" final val START_TIME = "__startTime__" final val END_TIME = "__endTime__" final val PART_KEY = "__partKey__" @@ -54,7 +59,7 @@ object PartKeyLuceneIndex { final val FACET_FIELD_PREFIX = "$facet_" final val LABEL_LIST_FACET = FACET_FIELD_PREFIX + LABEL_LIST - final val ignoreIndexNames = HashSet(START_TIME, PART_KEY, END_TIME, PART_ID) + final val ignoreIndexNames = HashSet(START_TIME, PART_KEY, END_TIME, PART_ID_FIELD, PART_ID_DV) val MAX_STR_INTERN_ENTRIES = 10000 val MAX_TERMS_TO_ITERATE = 10000 @@ -279,8 +284,8 @@ class PartKeyLuceneIndex(ref: DatasetRef, var facetsConfig: FacetsConfig = _ val document = new Document() - private[memstore] val partIdField = new StringField(PART_ID, "0", Store.NO) - private val partIdDv = new NumericDocValuesField(PART_ID, 0) + private[memstore] val partIdField = new StringField(PART_ID_FIELD, "0", Store.NO) + private val partIdDv = new NumericDocValuesField(PART_ID_DV, 0) private val partKeyDv = new BinaryDocValuesField(PART_KEY, new BytesRef()) private val startTimeField = new LongPoint(START_TIME, 0L) private val startTimeDv = new NumericDocValuesField(START_TIME, 0L) @@ -303,7 +308,7 @@ class PartKeyLuceneIndex(ref: DatasetRef, if (name.nonEmpty && value.nonEmpty && (always || facetEnabledForLabel(name)) && value.length < FACET_FIELD_MAX_LEN) { - facetsConfig.setRequireDimensionDrillDown(name, false) + facetsConfig.setDrillDownTermsIndexing(name, DrillDownTermsIndexing.NONE) facetsConfig.setIndexFieldName(name, FACET_FIELD_PREFIX + name) document.add(new SortedSetDocValuesFacetField(name, value)) } @@ -328,6 +333,13 @@ class PartKeyLuceneIndex(ref: DatasetRef, partIdDv.setLongValue(partId) document.add(partIdDv) } + + /* + * As of this writing, this documentId will be set as one of two values: + * - In TimeSeriesShard: the string representation of a partId (e.g. "42") + * - In DownsampledTimeSeriesShard: the base64-encoded sha256 of the document ID. This is used to support + * persistence of the downsample index; ephemeral partIds cannot be used. + */ partIdField.setStringValue(documentId) startTimeField.setLongValue(startTime) @@ -468,7 +480,7 @@ class PartKeyLuceneIndex(ref: DatasetRef, cforRange { 0 until partIds.length } { i => terms.add(new BytesRef(partIds(i).toString.getBytes(StandardCharsets.UTF_8))) } - indexWriter.deleteDocuments(new TermInSetQuery(PART_ID, terms)) + indexWriter.deleteDocuments(new TermInSetQuery(PART_ID_FIELD, terms)) } } @@ -498,14 +510,14 @@ class PartKeyLuceneIndex(ref: DatasetRef, .maximumSize(100) .recordStats() .build((key: (IndexReader, String)) => { - new DefaultSortedSetDocValuesReaderState(key._1, FACET_FIELD_PREFIX + key._2) + new DefaultSortedSetDocValuesReaderState(key._1, FACET_FIELD_PREFIX + key._2, new FacetsConfig()) }) private val readerStateCacheNonShardKeys: LoadingCache[(IndexReader, String), DefaultSortedSetDocValuesReaderState] = Caffeine.newBuilder() .maximumSize(200) .recordStats() .build((key: (IndexReader, String)) => { - new DefaultSortedSetDocValuesReaderState(key._1, FACET_FIELD_PREFIX + key._2) + new DefaultSortedSetDocValuesReaderState(key._1, FACET_FIELD_PREFIX + key._2, new FacetsConfig()) }) def labelNamesEfficient(colFilters: Seq[ColumnFilter], startTime: Long, endTime: Long): Seq[String] = { @@ -646,7 +658,7 @@ class PartKeyLuceneIndex(ref: DatasetRef, s"with startTime=$startTime endTime=$endTime into dataset=$ref shard=$shardNum") val doc = luceneDocument.get() val docToAdd = doc.facetsConfig.build(doc.document) - val term = new Term(PART_ID, documentId) + val term = new Term(PART_ID_FIELD, documentId) indexWriter.updateDocument(term, docToAdd) } @@ -711,7 +723,7 @@ class PartKeyLuceneIndex(ref: DatasetRef, */ def partKeyFromPartId(partId: Int): Option[BytesRef] = { val collector = new SinglePartKeyCollector() - withNewSearcher(s => s.search(new TermQuery(new Term(PART_ID, partId.toString)), collector) ) + withNewSearcher(s => s.search(new TermQuery(new Term(PART_ID_FIELD, partId.toString)), collector) ) Option(collector.singleResult) } @@ -720,7 +732,7 @@ class PartKeyLuceneIndex(ref: DatasetRef, */ def startTimeFromPartId(partId: Int): Long = { val collector = new NumericDocValueCollector(PartKeyLuceneIndex.START_TIME) - withNewSearcher(s => s.search(new TermQuery(new Term(PART_ID, partId.toString)), collector)) + withNewSearcher(s => s.search(new TermQuery(new Term(PART_ID_FIELD, partId.toString)), collector)) collector.singleResult } @@ -738,7 +750,7 @@ class PartKeyLuceneIndex(ref: DatasetRef, } // dont use BooleanQuery which will hit the 1024 term limit. Instead use TermInSetQuery which is // more efficient within Lucene - withNewSearcher(s => s.search(new TermInSetQuery(PART_ID, terms), collector)) + withNewSearcher(s => s.search(new TermInSetQuery(PART_ID_FIELD, terms), collector)) span.tag(s"num-partitions-to-page", terms.size()) val latency = System.nanoTime - startExecute span.mark(s"index-startTimes-for-odp-lookup-latency=${latency}ns") @@ -753,7 +765,7 @@ class PartKeyLuceneIndex(ref: DatasetRef, */ def endTimeFromPartId(partId: Int): Long = { val collector = new NumericDocValueCollector(PartKeyLuceneIndex.END_TIME) - withNewSearcher(s => s.search(new TermQuery(new Term(PART_ID, partId.toString)), collector)) + withNewSearcher(s => s.search(new TermQuery(new Term(PART_ID_FIELD, partId.toString)), collector)) collector.singleResult } @@ -804,7 +816,7 @@ class PartKeyLuceneIndex(ref: DatasetRef, logger.debug(s"Updating document ${partKeyString(documentId, partKeyOnHeapBytes, partKeyBytesRefOffset)} " + s"with startTime=$startTime endTime=$endTime into dataset=$ref shard=$shardNum") val docToAdd = doc.facetsConfig.build(doc.document) - indexWriter.updateDocument(new Term(PART_ID, partId.toString), docToAdd) + indexWriter.updateDocument(new Term(PART_ID_FIELD, partId.toString), docToAdd) } /** @@ -1088,7 +1100,7 @@ class SinglePartIdCollector extends SimpleCollector { // gets called for each segment override def doSetNextReader(context: LeafReaderContext): Unit = { - partIdDv = context.reader().getNumericDocValues(PartKeyLuceneIndex.PART_ID) + partIdDv = context.reader().getNumericDocValues(PartKeyLuceneIndex.PART_ID_DV) } // gets called for each matching document in current segment @@ -1126,7 +1138,7 @@ class TopKPartIdsCollector(limit: Int) extends Collector with StrictLogging { def getLeafCollector(context: LeafReaderContext): LeafCollector = { logger.trace("New segment inspected:" + context.id) endTimeDv = DocValues.getNumeric(context.reader, END_TIME) - partIdDv = DocValues.getNumeric(context.reader, PART_ID) + partIdDv = DocValues.getNumeric(context.reader, PART_ID_DV) new LeafCollector() { override def setScorer(scorer: Scorable): Unit = {} @@ -1175,7 +1187,7 @@ class PartIdCollector extends SimpleCollector { override def doSetNextReader(context: LeafReaderContext): Unit = { //set the subarray of the numeric values for all documents in the context - partIdDv = context.reader().getNumericDocValues(PartKeyLuceneIndex.PART_ID) + partIdDv = context.reader().getNumericDocValues(PartKeyLuceneIndex.PART_ID_DV) } override def collect(doc: Int): Unit = { @@ -1196,7 +1208,7 @@ class PartIdStartTimeCollector extends SimpleCollector { override def doSetNextReader(context: LeafReaderContext): Unit = { //set the subarray of the numeric values for all documents in the context - partIdDv = context.reader().getNumericDocValues(PartKeyLuceneIndex.PART_ID) + partIdDv = context.reader().getNumericDocValues(PartKeyLuceneIndex.PART_ID_DV) startTimeDv = context.reader().getNumericDocValues(PartKeyLuceneIndex.START_TIME) } @@ -1245,7 +1257,7 @@ class ActionCollector(action: (Int, BytesRef) => Unit) extends SimpleCollector { override def scoreMode(): ScoreMode = ScoreMode.COMPLETE_NO_SCORES override def doSetNextReader(context: LeafReaderContext): Unit = { - partIdDv = context.reader().getNumericDocValues(PartKeyLuceneIndex.PART_ID) + partIdDv = context.reader().getNumericDocValues(PartKeyLuceneIndex.PART_ID_DV) partKeyDv = context.reader().getBinaryDocValues(PartKeyLuceneIndex.PART_KEY) } diff --git a/project/Dependencies.scala b/project/Dependencies.scala index 81def65677..8c370fae55 100644 --- a/project/Dependencies.scala +++ b/project/Dependencies.scala @@ -73,8 +73,8 @@ object Dependencies { "com.googlecode.javaewah" % "JavaEWAH" % "1.1.6" withJavadoc(), "com.github.rholder.fauxflake" % "fauxflake-core" % "1.1.0", "org.scalactic" %% "scalactic" % "3.2.0" withJavadoc(), - "org.apache.lucene" % "lucene-core" % "8.8.2" withJavadoc(), - "org.apache.lucene" % "lucene-facet" % "8.8.2" withJavadoc(), + "org.apache.lucene" % "lucene-core" % "9.7.0" withJavadoc(), + "org.apache.lucene" % "lucene-facet" % "9.7.0" withJavadoc(), "com.github.alexandrnikitin" %% "bloom-filter" % "0.11.0", "org.rocksdb" % "rocksdbjni" % "6.29.5", "com.esotericsoftware" % "kryo" % "4.0.0" excludeAll(excludeMinlog), From 7adc382501d6c07938552788232e4ff5f16f6a8f Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Fri, 1 Sep 2023 09:37:32 -0700 Subject: [PATCH 36/39] bug(query): Streaming query execution allocated too much mem via RB (#1664) --- .../scala/filodb/query/exec/ExecPlan.scala | 43 +++++++++---------- 1 file changed, 20 insertions(+), 23 deletions(-) diff --git a/query/src/main/scala/filodb/query/exec/ExecPlan.scala b/query/src/main/scala/filodb/query/exec/ExecPlan.scala index a75ffb89ab..5abf255e16 100644 --- a/query/src/main/scala/filodb/query/exec/ExecPlan.scala +++ b/query/src/main/scala/filodb/query/exec/ExecPlan.scala @@ -235,31 +235,28 @@ trait ExecPlan extends QueryCommand { @volatile var numResultSamples = 0 // BEWARE - do not modify concurrently!! @volatile var resultSize = 0L val queryResults = rv.doOnStart(_ => Task.eval(span.mark("before-first-materialized-result-rv"))) - .map { - case srvable: SerializableRangeVector => srvable - case rv: RangeVector => - // materialize, and limit rows per RV - val execPlanString = queryWithPlanName(queryContext) - val builder = SerializedRangeVector.newBuilder(maxRecordContainerSize(querySession.queryConfig)) - val srv = SerializedRangeVector(rv, builder, recordSchema, execPlanString, querySession.queryStats) - if (rv.outputRange.isEmpty) - qLogger.debug(s"Empty rangevector found. Rv class is: ${rv.getClass.getSimpleName}, " + - s"execPlan is: $execPlanString, execPlan children ${this.children}") + .bufferTumbling(querySession.queryConfig.numRvsPerResultMessage) + .map { f => + val builder = SerializedRangeVector.newBuilder(maxRecordContainerSize(querySession.queryConfig)) + val srvs = f.map { + case srvable: SerializableRangeVector => srvable + case rv: RangeVector => + val execPlanString = queryWithPlanName(queryContext) + SerializedRangeVector(rv, builder, recordSchema, execPlanString, querySession.queryStats) + } + .map { srv => + // fail the query instead of limiting range vectors and returning incomplete/inaccurate results + numResultSamples += srv.numRowsSerialized + checkSamplesLimit(numResultSamples, querySession.warnings) + val srvBytes = srv.estimatedSerializedBytes + resultSize += srvBytes + querySession.queryStats.getResultBytesCounter(Nil).addAndGet(srvBytes) + checkResultBytes(resultSize, querySession.queryConfig, querySession.warnings) srv + } + .filter(_.numRowsSerialized > 0) + StreamQueryResult(queryContext.queryId, planId, srvs) } - .map { srv => - // fail the query instead of limiting range vectors and returning incomplete/inaccurate results - numResultSamples += srv.numRowsSerialized - checkSamplesLimit(numResultSamples, querySession.warnings) - val srvBytes = srv.estimatedSerializedBytes - resultSize += srvBytes - querySession.queryStats.getResultBytesCounter(Nil).addAndGet(srvBytes) - checkResultBytes(resultSize, querySession.queryConfig, querySession.warnings) - srv - } - .filter(_.numRowsSerialized > 0) - .bufferTumbling(querySession.queryConfig.numRvsPerResultMessage) - .map(StreamQueryResult(queryContext.queryId, planId, _)) .guarantee(Task.eval { Kamon.histogram("query-execute-time-elapsed-step2-result-materialized", MeasurementUnit.time.milliseconds) From bf8ead0f6e6a9a99ba261c25807921a3863f997f Mon Sep 17 00:00:00 2001 From: sandeep6189 Date: Mon, 11 Sep 2023 14:47:22 -0700 Subject: [PATCH 37/39] perf(card): Adding config support for DS card flushCount and perf logs for cardinality calculation time (#1666) * Adding config support for DS Card flushCount and perf logs for cardinality calculation time --- .../ratelimit/CardinalityManager.scala | 89 ++++++++++++------- 1 file changed, 57 insertions(+), 32 deletions(-) diff --git a/core/src/main/scala/filodb.core/memstore/ratelimit/CardinalityManager.scala b/core/src/main/scala/filodb.core/memstore/ratelimit/CardinalityManager.scala index 92ea92bb19..dfe1d6897c 100644 --- a/core/src/main/scala/filodb.core/memstore/ratelimit/CardinalityManager.scala +++ b/core/src/main/scala/filodb.core/memstore/ratelimit/CardinalityManager.scala @@ -32,6 +32,15 @@ class CardinalityManager(datasetRef: DatasetRef, // physical resources for duplicate calculation private var isCardTriggered: Boolean = false + // number of partKeys to aggregate in memory at any given time, before we write the cardinality information to + // rocksDB. This is done to reduce the time taken to calculate cardinality of partitions with high number of + // TS by reducing the number writes issued to RocksDB ( From our profiling, high number of RocksDB writes lead to + // performance bottlenecks) + // For cardFlushCount = 500,000, we conservatively expect to consume - 128 bytes * 500,000 = ~64 MB + private val cardFlushCount: Option[Int] = + if (filodbConfig.hasPath("card-flush-count")) { + Some(filodbConfig.getInt("card-flush-count")) + } else None /** * `dataset-configs` is an string array where each string is a file path for a dataset config. This function reads @@ -132,9 +141,6 @@ class CardinalityManager(datasetRef: DatasetRef, /** * Triggers cardinalityCount if metering is enabled and the required criteria matches. - * It creates a new instance of CardinalityTracker and uses the PartKeyLuceneIndex to calculate cardinality count - * and store data in a local CardinalityStore. We then close the previous instance of CardinalityTracker and switch - * it with the new one we created in this call. * @param indexRefreshCount The number of time the indexRefresh has already happened. This is used in the logic of * shouldTriggerCardinalityCount */ @@ -143,34 +149,7 @@ class CardinalityManager(datasetRef: DatasetRef, try { if (shouldTriggerCardinalityCount(shardNum, numShardsPerNode, indexRefreshCount)) { isCardTriggered = true - val newCardTracker = getNewCardTracker() - var cardCalculationComplete = false - try { - partKeyIndex.calculateCardinality(partSchema, newCardTracker) - cardCalculationComplete = true - } catch { - case ex: Exception => - logger.error(s"[CardinalityManager]Error while calculating cardinality using" + - s" PartKeyLuceneIndex! shardNum=$shardNum indexRefreshCount=$indexRefreshCount", ex) - // cleanup resources used by the newCardTracker tracker to avoid leaking of resources - newCardTracker.close() - } - if (cardCalculationComplete) { - try { - // close the cardinality store and release the physical resources of the current cardinality store - close() - cardTracker = Some(newCardTracker) - logger.info(s"[CardinalityManager] Triggered cardinality count successfully for" + - s" shardNum=$shardNum indexRefreshCount=$indexRefreshCount") - } catch { - case ex: Exception => - // Very unlikely scenario, but can happen if the disk call fails. - logger.error(s"[CardinalityManager]Error closing card tracker! shardNum=$shardNum", ex) - // setting cardTracker to None in this case, since the exception happened on the close. We - // can't rely on the card store. The next trigger should re-build the card store and card tracker - cardTracker = None - } - } + createNewCardinalityTrackerAndCalculate(indexRefreshCount) isCardTriggered = false } else { @@ -189,6 +168,50 @@ class CardinalityManager(datasetRef: DatasetRef, } } + /** + * Creates a new instance of CardinalityTracker and uses the PartKeyLuceneIndex to calculate cardinality count + * and store data in a local CardinalityStore. We then close the previous instance of CardinalityTracker and switch + * it with the new one we created in this call. + * + * @param indexRefreshCount The number of time the indexRefresh has already happened. This is used in the logic of + * shouldTriggerCardinalityCount + */ + private def createNewCardinalityTrackerAndCalculate(indexRefreshCount: Int): Unit = { + val newCardTracker = getNewCardTracker() + var cardCalculationComplete = false + val startTimeMs = System.currentTimeMillis() + try { + logger.info(s"[CardinalityManager]Triggering cardinality count for shardNum=$shardNum " + + s"indexRefreshCount=$indexRefreshCount") + partKeyIndex.calculateCardinality(partSchema, newCardTracker) + cardCalculationComplete = true + } catch { + case ex: Exception => + logger.error(s"[CardinalityManager]Error while calculating cardinality using" + + s" PartKeyLuceneIndex! shardNum=$shardNum indexRefreshCount=$indexRefreshCount", ex) + // cleanup resources used by the newCardTracker tracker to avoid leaking of resources + newCardTracker.close() + } + if (cardCalculationComplete) { + try { + // close and release the physical resources of the outdated/previous cardinality store + close() + // reassign the cardTracker with the newly created CardinalityTracker object + cardTracker = Some(newCardTracker) + val timeTakenInSeconds = ((System.currentTimeMillis() - startTimeMs)/1000.0) + logger.info(s"[CardinalityManager] Triggered cardinality count successfully for" + + s" shardNum=$shardNum indexRefreshCount=$indexRefreshCount timeTakenInSeconds=$timeTakenInSeconds") + } catch { + case ex: Exception => + // Very unlikely scenario, but can happen if the disk call fails. + logger.error(s"[CardinalityManager]Error closing card tracker! shardNum=$shardNum", ex) + // setting cardTracker to None in this case, since the exception happened on the close. We + // can't rely on the card store. The next trigger should re-build the card store and card tracker + cardTracker = None + } + } + } + /** * Helper method to create a CardinalityTracker instance * @@ -197,7 +220,9 @@ class CardinalityManager(datasetRef: DatasetRef, private def getNewCardTracker(): CardinalityTracker = { val cardStore = new RocksDbCardinalityStore(datasetRef, shardNum) val defaultQuota = quotaSource.getDefaults(datasetRef) - val tracker = new CardinalityTracker(datasetRef, shardNum, shardKeyLen, defaultQuota, cardStore) + logger.info(s"[CardinalityManager] Creating new CardinalityTracker with flushCount=$cardFlushCount") + val tracker = new CardinalityTracker(datasetRef, shardNum, shardKeyLen, defaultQuota, cardStore, + flushCount = cardFlushCount) quotaSource.getQuotas(datasetRef).foreach { q => tracker.setQuota(q.shardKeyPrefix, q.quota) } From 50d28ead53f1016df6dbf635eadf771a784b2009 Mon Sep 17 00:00:00 2001 From: yu-shipit <120689245+yu-shipit@users.noreply.github.com> Date: Wed, 13 Sep 2023 11:44:04 -0700 Subject: [PATCH 38/39] fix(core) fix the binary join aggregation across different partitions. (#1629) (#1668) * fix(core) fix the unless operator for aggregators. For regex shard key we need to aggregate across all nodes. InProcessPlanDispatcher is needed. --------- Co-authored-by: Yu Zhang (cherry picked from commit f5018ae1bc4ae9e2c6e8fe10b55b4052c96455dd) --- .../queryplanner/LogicalPlanUtils.scala | 4 + .../queryplanner/PlannerHierarchySpec.scala | 131 +++++++++++++++++- 2 files changed, 131 insertions(+), 4 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala index fdb92fc1b8..8a6bcd603e 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala @@ -34,6 +34,10 @@ object LogicalPlanUtils extends StrictLogging { */ def hasDescendantAggregate(lp: LogicalPlan): Boolean = lp match { case _: Aggregate => true + // consider this BinaryJoin example foo + on(h) + bar. + // partition1 has foo{h=1}, bar{h1=2} and partition2 has foo{h=2}, bar{h1=1} + // the binary join cannot happen on a partition locally. InProcessPlanDispatcher is required. + case _: BinaryJoin => true case nonLeaf: NonLeafLogicalPlan => nonLeaf.children.exists(hasDescendantAggregate(_)) case _ => false } diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala index 14a602ffab..3f194654db 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala @@ -112,7 +112,8 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS "localPartition", dataset, queryConfig) private val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { - if (shardColumnFilters.nonEmpty) { + // we may have mixed of a regex filter and a non-regex filter. + if (shardColumnFilters.exists(f => f.column == "_ns_" && f.filter.isInstanceOf[EqualsRegex])) { // to ensure that tests dont call something else that is not configured require(shardColumnFilters.exists(f => f.column == "_ns_" && f.filter.isInstanceOf[EqualsRegex] && ( @@ -130,25 +131,30 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("remoteNs"))) ) } + } else if (shardColumnFilters.exists(f => f.column == "_ns_" && f.filter.isInstanceOf[Equals])) { + Seq(shardColumnFilters) } else { Nil } // i.e. filters for a scalar } private val oneRemoteShardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { - if (shardColumnFilters.nonEmpty) { + if (shardColumnFilters.exists(f => f.column == "_ns_" && f.filter.isInstanceOf[EqualsRegex])) { // to ensure that tests dont call something else that is not configured require(shardColumnFilters.exists(f => f.column == "_ns_" && f.filter.isInstanceOf[EqualsRegex] && f.filter.asInstanceOf[EqualsRegex].pattern.toString == ".*remoteNs")) Seq( Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("remoteNs"))) ) - } else { + } else if (shardColumnFilters.nonEmpty) { + Seq(shardColumnFilters) + } + else { Nil } // i.e. filters for a scalar } private val twoRemoteShardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { - if (shardColumnFilters.nonEmpty) { + if (shardColumnFilters.exists(f => f.column == "_ns_" && f.filter.isInstanceOf[EqualsRegex])) { // to ensure that tests dont call something else that is not configured require(shardColumnFilters.exists(f => f.column == "_ns_" && f.filter.isInstanceOf[EqualsRegex] && f.filter.asInstanceOf[EqualsRegex].pattern.toString == "remoteNs.*")) @@ -156,6 +162,8 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("remoteNs0"))), Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("remoteNs1"))), ) + } else if (shardColumnFilters.nonEmpty) { + Seq(shardColumnFilters) } else { Nil } // i.e. filters for a scalar @@ -203,6 +211,121 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS validatePlan(execPlan, expected) } + it("should aggregate through ActorPlanDispatcher for localNs") { + val lp = Parser.queryRangeToLogicalPlan( + """count(foo{_ws_="demo", _ns_="localNs", instance="Inst-1" } + | unless on (instance) + | bar{_ws_ = "demo", _ns_="localNs", instance=~".*"} )""".stripMargin, + TimeStepParams(startSeconds, step, endSeconds), Antlr) + val execPlan = rootPlanner.materialize(lp, QueryContext(origQueryParams = queryParams)) + val expected = + """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192))) + |-T~AggregatePresenter(aggrOp=Count, aggrParams=List(), rangeParams=RangeParams(1634173130,300,1634777330)) + |--E~LocalPartitionReduceAggregateExec(aggrOp=Count, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1888594662],raw) + |---T~AggregateMapReduce(aggrOp=Count, aggrParams=List(), without=List(), by=List()) + |----E~SetOperatorExec(binaryOp=LUnless, on=List(instance), ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1888594662],raw) + |-----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1888594662],raw) + |-----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1888594662],raw) + |-----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,EqualsRegex(.*)), ColumnFilter(_metric_,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1888594662],raw) + |-----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,EqualsRegex(.*)), ColumnFilter(_metric_,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1888594662],raw) + |-T~AggregatePresenter(aggrOp=Count, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634172830)) + |--E~LocalPartitionReduceAggregateExec(aggrOp=Count, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1888594662],downsample) + |---T~AggregateMapReduce(aggrOp=Count, aggrParams=List(), without=List(), by=List()) + |----E~SetOperatorExec(binaryOp=LUnless, on=List(instance), ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1888594662],downsample) + |-----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1888594662],downsample) + |-----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1888594662],downsample) + |-----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,EqualsRegex(.*)), ColumnFilter(_metric_,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1888594662],downsample) + |-----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,EqualsRegex(.*)), ColumnFilter(_metric_,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1888594662],downsample)""".stripMargin + validatePlan(execPlan, expected) + } + + it("should aggregate through InProcessPlanDispatcher for remote _ns_ using regex and see two local Ns") { + val lp = Parser.queryRangeToLogicalPlan( + """count(foo{_ws_="demo", _ns_="localNs", instance="Inst-1" } + | unless on (instance) + | bar{_ws_ = "demo", _ns_ =~"localNs.*", instance=~".*"} )""".stripMargin, + TimeStepParams(startSeconds, step, endSeconds), Antlr) + val execPlan = rootPlanner.materialize(lp, QueryContext(origQueryParams = queryParams)) + val expected = + """T~AggregatePresenter(aggrOp=Count, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634777330)) + |-E~LocalPartitionReduceAggregateExec(aggrOp=Count, aggrParams=List()) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) + |--T~AggregateMapReduce(aggrOp=Count, aggrParams=List(), without=List(), by=List()) + |---E~SetOperatorExec(binaryOp=LUnless, on=List(instance), ignoring=List()) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) + |----E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192))) + |-----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],raw) + |------T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) + |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],raw) + |------T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) + |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],raw) + |-----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],downsample) + |------T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) + |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],downsample) + |------T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) + |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],downsample) + |----E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) + |-----E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192))) + |------E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],raw) + |-------T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) + |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(instance,EqualsRegex(.*)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs1)), ColumnFilter(_metric_,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],raw) + |-------T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) + |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(instance,EqualsRegex(.*)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs1)), ColumnFilter(_metric_,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],raw) + |------E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],downsample) + |-------T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) + |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(instance,EqualsRegex(.*)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs1)), ColumnFilter(_metric_,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],downsample) + |-------T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) + |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(instance,EqualsRegex(.*)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs1)), ColumnFilter(_metric_,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],downsample) + |-----E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192))) + |------E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],raw) + |-------T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) + |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(instance,EqualsRegex(.*)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],raw) + |-------T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) + |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(instance,EqualsRegex(.*)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],raw) + |------E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],downsample) + |-------T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) + |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(instance,EqualsRegex(.*)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],downsample) + |-------T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) + |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(instance,EqualsRegex(.*)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],downsample)""".stripMargin + validatePlan(execPlan, expected) + } + + it("should aggregate through InProcessPlanDispatcher for remote _ns_ using regex and see both remote and local Ns") { + val lp = Parser.queryRangeToLogicalPlan( + """count(foo{_ws_="demo", _ns_="localNs", instance="Inst-1" } + | unless on (instance) + | bar{_ws_ = "demo", _ns_ =~"remoteNs.*", instance=~".*"} )""".stripMargin, + TimeStepParams(startSeconds, step, endSeconds), Antlr) + val execPlan = twoRemoteRootPlanner.materialize(lp, QueryContext(origQueryParams = queryParams, + plannerParams = PlannerParams(processMultiPartition = true))) + val expected = + """T~AggregatePresenter(aggrOp=Count, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634777330)) + |-E~LocalPartitionReduceAggregateExec(aggrOp=Count, aggrParams=List()) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) + |--T~AggregateMapReduce(aggrOp=Count, aggrParams=List(), without=List(), by=List()) + |---E~SetOperatorExec(binaryOp=LUnless, on=List(instance), ignoring=List()) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) + |----E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192))) + |-----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1681100802],raw) + |------T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) + |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1681100802],raw) + |------T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) + |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1681100802],raw) + |-----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1681100802],downsample) + |------T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) + |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1681100802],downsample) + |------T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) + |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1681100802],downsample) + |----E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) + |-----E~PromQlRemoteExec(PromQlQueryParams(bar{instance=~".*",_ws_="demo",_ns_="remoteNs0"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remotePartition-url0, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) + |-----E~PromQlRemoteExec(PromQlQueryParams(bar{instance=~".*",_ws_="demo",_ns_="remoteNs1"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remote1Partition-url1, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin + validatePlan(execPlan, expected) + } + it("should generate plan for one recording rule query") { val lp = Parser.queryRangeToLogicalPlan( """sum(foo:1m{_ws_ = "demo", _ns_ = "localNs", instance = "Inst-1" })""", From 78a33f615d02d00133a4499390ec245ffc7637a4 Mon Sep 17 00:00:00 2001 From: Yu Zhang Date: Tue, 19 Sep 2023 14:03:01 -0700 Subject: [PATCH 39/39] Bump filodb version to 0.9.23. --- version.sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/version.sbt b/version.sbt index c051096049..1e976d7caf 100644 --- a/version.sbt +++ b/version.sbt @@ -1 +1 @@ -version in ThisBuild := "0.9.22.integration-SNAPSHOT" +version in ThisBuild := "0.9.23.integration-SNAPSHOT"