diff --git a/.travis.yml b/.travis.yml index e2e672db87..7dd2ea1535 100644 --- a/.travis.yml +++ b/.travis.yml @@ -2,7 +2,7 @@ language: scala dist: trusty env: global: - _JAVA_OPTIONS="-Dakka.test.timefactor=3 -XX:MaxMetaspaceSize=256m" + _JAVA_OPTIONS="-Dakka.test.timefactor=3 -XX:MaxMetaspaceSize=512m" scala: - 2.11.12 jdk: diff --git a/cassandra/src/main/scala/filodb.cassandra/columnstore/CassandraColumnStore.scala b/cassandra/src/main/scala/filodb.cassandra/columnstore/CassandraColumnStore.scala index de2f9059cc..c0fb415d06 100644 --- a/cassandra/src/main/scala/filodb.cassandra/columnstore/CassandraColumnStore.scala +++ b/cassandra/src/main/scala/filodb.cassandra/columnstore/CassandraColumnStore.scala @@ -13,6 +13,7 @@ import com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap import com.typesafe.config.Config import com.typesafe.scalalogging.StrictLogging import kamon.Kamon +import kamon.metric.MeasurementUnit import monix.eval.Task import monix.execution.Scheduler import monix.reactive.Observable @@ -56,7 +57,6 @@ extends ColumnStore with CassandraChunkSource with StrictLogging { import collection.JavaConverters._ import filodb.core.store._ - import Perftools._ logger.info(s"Starting CassandraColumnStore with config ${cassandraConfig.withoutPath("password")}") @@ -68,6 +68,13 @@ extends ColumnStore with CassandraChunkSource with StrictLogging { val sinkStats = new ChunkSinkStats + val writeChunksetLatency = Kamon.histogram("cass-write-chunkset-latency", MeasurementUnit.time.milliseconds) + .withoutTags() + val writePksLatency = Kamon.histogram("cass-write-part-keys-latency", MeasurementUnit.time.milliseconds) + .withoutTags() + val readChunksBatchLatency = Kamon.histogram("cassandra-per-batch-chunk-read-latency", + MeasurementUnit.time.milliseconds).withoutTags() + def initialize(dataset: DatasetRef, numShards: Int): Future[Response] = { val chunkTable = getOrCreateChunkTable(dataset) val partitionKeysByUpdateTimeTable = getOrCreatePartitionKeysByUpdateTimeTable(dataset) @@ -133,49 +140,45 @@ extends ColumnStore with CassandraChunkSource with StrictLogging { chunksets: Observable[ChunkSet], diskTimeToLiveSeconds: Int = 259200): Future[Response] = { chunksets.mapAsync(writeParallelism) { chunkset => - val span = Kamon.spanBuilder("write-chunkset").asChildOf(Kamon.currentSpan()).start() - val partBytes = BinaryRegionLarge.asNewByteArray(chunkset.partition) - val future = - for { writeChunksResp <- writeChunks(ref, partBytes, chunkset, diskTimeToLiveSeconds) - if writeChunksResp == Success - writeIndicesResp <- writeIndices(ref, partBytes, chunkset, diskTimeToLiveSeconds) - if writeIndicesResp == Success - } yield { - span.finish() - sinkStats.chunksetWrite() - writeIndicesResp - } - Task.fromFuture(future) - } - .countL.runAsync - .map { chunksWritten => - if (chunksWritten > 0) Success else NotApplied + val start = System.currentTimeMillis() + val partBytes = BinaryRegionLarge.asNewByteArray(chunkset.partition) + val future = + for { writeChunksResp <- writeChunks(ref, partBytes, chunkset, diskTimeToLiveSeconds) + if writeChunksResp == Success + writeIndicesResp <- writeIndices(ref, partBytes, chunkset, diskTimeToLiveSeconds) + if writeIndicesResp == Success + } yield { + writeChunksetLatency.record(System.currentTimeMillis() - start) + sinkStats.chunksetWrite() + writeIndicesResp } + Task.fromFuture(future) + } + .countL.runAsync + .map { chunksWritten => + if (chunksWritten > 0) Success else NotApplied + } } private def writeChunks(ref: DatasetRef, partition: Array[Byte], chunkset: ChunkSet, diskTimeToLiveSeconds: Int): Future[Response] = { - asyncSubtrace("write-chunks", "ingestion") { - val chunkTable = getOrCreateChunkTable(ref) - chunkTable.writeChunks(partition, chunkset.info, chunkset.chunks, sinkStats, diskTimeToLiveSeconds) - .collect { - case Success => chunkset.invokeFlushListener(); Success - } - } + val chunkTable = getOrCreateChunkTable(ref) + chunkTable.writeChunks(partition, chunkset.info, chunkset.chunks, sinkStats, diskTimeToLiveSeconds) + .collect { + case Success => chunkset.invokeFlushListener(); Success + } } private def writeIndices(ref: DatasetRef, partition: Array[Byte], chunkset: ChunkSet, diskTimeToLiveSeconds: Int): Future[Response] = { - asyncSubtrace("write-index", "ingestion") { - val indexTable = getOrCreateIngestionTimeIndexTable(ref) - val info = chunkset.info - val infos = Seq((info.ingestionTime, info.startTime, ChunkSetInfo.toBytes(info))) - indexTable.writeIndices(partition, infos, sinkStats, diskTimeToLiveSeconds) - } + val indexTable = getOrCreateIngestionTimeIndexTable(ref) + val info = chunkset.info + val infos = Seq((info.ingestionTime, info.startTime, ChunkSetInfo.toBytes(info))) + indexTable.writeIndices(partition, infos, sinkStats, diskTimeToLiveSeconds) } /** @@ -212,11 +215,11 @@ extends ColumnStore with CassandraChunkSource with StrictLogging { s"endTimeExclusive=$endTimeExclusive maxChunkTime=$maxChunkTime") // This could be more parallel, but decision was made to control parallelism at one place: In spark (via its // parallelism configuration. Revisit if needed later. - val batchReadSpan = Kamon.spanBuilder("cassandra-per-batch-data-read-latency").start() + val start = System.currentTimeMillis() try { chunksTable.readRawPartitionRangeBBNoAsync(parts, userTimeStart - maxChunkTime, endTimeExclusive) } finally { - batchReadSpan.finish() + readChunksBatchLatency.record(System.currentTimeMillis() - start) } } } @@ -376,7 +379,7 @@ extends ColumnStore with CassandraChunkSource with StrictLogging { writeToPkUTTable: Boolean = true): Future[Response] = { val pkTable = getOrCreatePartitionKeysTable(ref, shard) val pkByUTTable = getOrCreatePartitionKeysByUpdateTimeTable(ref) - val span = Kamon.spanBuilder("write-part-keys").asChildOf(Kamon.currentSpan()).start() + val start = System.currentTimeMillis() val ret = partKeys.mapAsync(writeParallelism) { pk => val ttl = if (pk.endTime == Long.MaxValue) -1 else diskTTLSeconds // caller needs to supply hash for partKey - cannot be None @@ -393,7 +396,9 @@ extends ColumnStore with CassandraChunkSource with StrictLogging { resp } }.findL(_.isInstanceOf[ErrorResponse]).map(_.getOrElse(Success)).runAsync - ret.onComplete(_ => span.finish()) + ret.onComplete { _ => + writePksLatency.record(System.currentTimeMillis() - start) + } ret } diff --git a/cassandra/src/test/scala/filodb.cassandra/columnstore/OdpSpec.scala b/cassandra/src/test/scala/filodb.cassandra/columnstore/OdpSpec.scala index f78c307fa1..cc682fe197 100644 --- a/cassandra/src/test/scala/filodb.cassandra/columnstore/OdpSpec.scala +++ b/cassandra/src/test/scala/filodb.cassandra/columnstore/OdpSpec.scala @@ -1,14 +1,12 @@ package filodb.cassandra.columnstore import scala.concurrent.Future - import com.typesafe.config.ConfigFactory import monix.execution.Scheduler import monix.reactive.Observable import org.scalatest.BeforeAndAfterAll import org.scalatest.concurrent.ScalaFutures import org.scalatest.time.{Millis, Seconds, Span} - import filodb.cassandra.DefaultFiloSessionProvider import filodb.core.{MachineMetricsData, TestData} import filodb.core.binaryrecord2.{BinaryRecordRowReader, RecordBuilder} @@ -16,7 +14,7 @@ import filodb.core.downsample.OffHeapMemory import filodb.core.memstore._ import filodb.core.memstore.FiloSchedulers.QuerySchedName import filodb.core.metadata.{Dataset, Schemas} -import filodb.core.query.{ColumnFilter, QueryConfig, QueryContext, QuerySession} +import filodb.core.query.{ColumnFilter, PlannerParams, QueryConfig, QueryContext, QuerySession} import filodb.core.query.Filter.Equals import filodb.core.store.{InMemoryMetaStore, PartKeyRecord, StoreConfig, TimeRangeChunkScan} import filodb.memory.format.ZeroCopyUTF8String._ @@ -84,7 +82,7 @@ class OdpSpec extends AnyFunSpec with Matchers with BeforeAndAfterAll with Scala MachineMetricsData.records(dataset, rawSamples).records.foreach { case (base, offset) => val rr = new BinaryRecordRowReader(Schemas.gauge.ingestionSchema, base, offset) - part.ingest( System.currentTimeMillis(), rr, offheapMem.blockMemFactory) + part.ingest( System.currentTimeMillis(), rr, offheapMem.blockMemFactory, false, Option.empty) part.switchBuffers(offheapMem.blockMemFactory, true) } val chunks = part.makeFlushChunks(offheapMem.blockMemFactory) @@ -187,8 +185,9 @@ class OdpSpec extends AnyFunSpec with Matchers with BeforeAndAfterAll with Scala def query(memStore: TimeSeriesMemStore): Future[QueryResponse] = { val colFilters = seriesTags.map { case (t, v) => ColumnFilter(t.toString, Equals(v.toString)) }.toSeq val queryFilters = colFilters :+ ColumnFilter("_metric_", Equals(gaugeName)) - val exec = MultiSchemaPartitionsExec(QueryContext(sampleLimit = numSamples * 2), InProcessPlanDispatcher, - dataset.ref, 0, queryFilters, TimeRangeChunkScan(firstSampleTime, firstSampleTime + 2 * numSamples)) + val exec = MultiSchemaPartitionsExec(QueryContext(plannerParams = PlannerParams(sampleLimit = numSamples * 2)), + InProcessPlanDispatcher, dataset.ref, 0, queryFilters, TimeRangeChunkScan(firstSampleTime, firstSampleTime + 2 * + numSamples)) val queryConfig = new QueryConfig(config.getConfig("query")) val querySession = QuerySession(QueryContext(), queryConfig) exec.execute(memStore, querySession)(queryScheduler).runAsync(queryScheduler) diff --git a/cli/src/main/scala/filodb.cli/CliMain.scala b/cli/src/main/scala/filodb.cli/CliMain.scala index 0a7774ad2e..77e8e70ad7 100644 --- a/cli/src/main/scala/filodb.cli/CliMain.scala +++ b/cli/src/main/scala/filodb.cli/CliMain.scala @@ -30,7 +30,6 @@ import filodb.query._ // scalastyle:off class Arguments(args: Seq[String]) extends ScallopConf(args) { - val dataset = opt[String]() val database = opt[String]() val command = opt[String]() @@ -64,6 +63,9 @@ class Arguments(args: Seq[String]) extends ScallopConf(args) { val everynseconds = opt[String]() val shards = opt[List[String]]() val spread = opt[Int]() + val k = opt[Int]() + val shardkeyprefix = opt[List[String]](default = Some(List())) + verify() override def onError(e: Throwable): Unit = e match { @@ -101,6 +103,7 @@ object CliMain extends FilodbClusterNode { println(" --host [--port ...] --command list") println(" --host [--port ...] --command status --dataset ") println(" --host [--port ...] --command labelvalues --labelName --labelfilter --dataset ") + println(" --host [--port ...] --command topkcard --dataset prometheus --k 2 --shardkeyprefix demo App-0") println(""" --command promFilterToPartKeyBR --promql "myMetricName{_ws_='myWs',_ns_='myNs'}" --schema prom-counter""") println(""" --command partKeyBrAsString --hexpk 0x2C0000000F1712000000200000004B8B36940C006D794D65747269634E616D650E00C104006D794E73C004006D795773""") println(""" --command decodeChunkInfo --hexchunkinfo 0x12e8253a267ea2db060000005046fc896e0100005046fc896e010000""") @@ -132,8 +135,8 @@ object CliMain extends FilodbClusterNode { } def main(rawArgs: Array[String]): Unit = { - val args = new Arguments(rawArgs) try { + val args = new Arguments(rawArgs) val timeout = args.timeoutseconds().seconds args.command.toOption match { case Some("init") => @@ -165,6 +168,23 @@ object CliMain extends FilodbClusterNode { val values = remote.getIndexValues(ref, args.indexname(), args.shards().head.toInt, args.limit()) values.foreach { case (term, freq) => println(f"$term%40s\t$freq") } + case Some("topkcard") => + require(args.host.isDefined && args.dataset.isDefined && args.k.isDefined, + "--host, --dataset, --k must be defined") + val (remote, ref) = getClientAndRef(args) + val res = remote.getTopkCardinality(ref, args.shards.getOrElse(Nil).map(_.toInt), + args.shardkeyprefix(), args.k()) + println(s"ShardKeyPrefix: ${args.shardkeyprefix}") + res.groupBy(_.shard).foreach { crs => + println(s"Shard: ${crs._1}") + printf("%40s %12s %10s %10s\n", "Child", "TimeSeries", "Children", "Children") + printf("%40s %12s %10s %10s\n", "Name", "Count", "Count", "Quota") + println("===================================================================================") + crs._2.foreach { cr => + printf("%40s %12d %10d %10d\n", cr.childName, cr.timeSeriesCount, cr.childrenCount, cr.childrenQuota) + } + } + case Some("status") => val (remote, ref) = getClientAndRef(args) dumpShardStatus(remote, ref) @@ -349,9 +369,11 @@ object CliMain extends FilodbClusterNode { options: QOptions, tsdbQueryParams: TsdbQueryParams): Unit = { val ref = DatasetRef(dataset) val spreadProvider: Option[SpreadProvider] = options.spread.map(s => StaticSpreadProvider(SpreadChange(0, s))) - val qOpts = QueryContext(tsdbQueryParams, spreadProvider, options.sampleLimit) - .copy(queryTimeoutMillis = options.timeout.toMillis.toInt, - shardOverrides = options.shardOverrides) + + val qOpts = QueryContext(origQueryParams = tsdbQueryParams, + plannerParams = PlannerParams(applicationId = "filodb-cli", spreadOverride = spreadProvider, + sampleLimit = options.sampleLimit, queryTimeoutMillis = options.timeout.toMillis.toInt, + shardOverrides = options.shardOverrides)) println(s"Sending query command to server for $ref with options $qOpts...") println(s"Query Plan:\n$plan") options.everyN match { diff --git a/conf/timeseries-dev-source.conf b/conf/timeseries-dev-source.conf index 246e2b9245..5240c576ba 100644 --- a/conf/timeseries-dev-source.conf +++ b/conf/timeseries-dev-source.conf @@ -97,6 +97,9 @@ # Limits maximum amount of data a single leaf query can scan max-data-per-shard-query = 50 MB + + # Set to true to enable metering of time series. Used for rate-limiting + metering-enabled = true } downsample { # Resolutions for downsampled data ** in ascending order ** diff --git a/conf/timeseries-filodb-server.conf b/conf/timeseries-filodb-server.conf index 2800835194..5915f323ec 100644 --- a/conf/timeseries-filodb-server.conf +++ b/conf/timeseries-filodb-server.conf @@ -11,6 +11,22 @@ filodb { "conf/timeseries-dev-source.conf" ] + quotas { + prometheus { + defaults = [100, 500, 10000, 100000] + custom = [ + { + shardKeyPrefix = ["demo", "App-0", "heap_usage"] + quota = 100 + }, + { + shardKeyPrefix = ["demo"] + quota = 10 + } + ] + } + } + spread-default = 1 # Override default spread for application using override block which will have non metric shard keys and spread. diff --git a/coordinator/src/main/scala/filodb.coordinator/IngestionActor.scala b/coordinator/src/main/scala/filodb.coordinator/IngestionActor.scala index 7a9f21a412..ee4186673d 100644 --- a/coordinator/src/main/scala/filodb.coordinator/IngestionActor.scala +++ b/coordinator/src/main/scala/filodb.coordinator/IngestionActor.scala @@ -11,6 +11,7 @@ import scala.util.control.NonFatal import akka.actor.{ActorRef, Props} import akka.event.LoggingReceive import kamon.Kamon +import kamon.metric.MeasurementUnit import monix.eval.Task import monix.execution.{CancelableFuture, Scheduler, UncaughtExceptionReporter} import monix.reactive.Observable @@ -293,10 +294,12 @@ private[filodb] final class IngestionActor(ref: DatasetRef, private def doRecovery(shard: Int, startOffset: Long, endOffset: Long, interval: Long, checkpoints: Map[Int, Long]): Future[Option[Long]] = { val futTry = create(shard, Some(startOffset)) map { ingestionStream => - val recoveryTrace = Kamon.spanBuilder("ingestion-recovery-trace") - .asChildOf(Kamon.currentSpan()) - .tag("shard", shard.toString) - .tag("dataset", ref.toString).start() + + val ingestionRecoveryLatency = Kamon.histogram("ingestion-recovery-latency", MeasurementUnit.time.milliseconds) + .withTag("dataset", ref.dataset) + .withTag("shard", shard) + + val recoveryStart = System.currentTimeMillis() val stream = ingestionStream.get statusActor ! RecoveryInProgress(ref, shard, nodeCoord, 0) @@ -317,12 +320,11 @@ private[filodb] final class IngestionActor(ref: DatasetRef, logger.info(s"Finished recovery for dataset=$ref shard=$shard") ingestionStream.teardown() streams.remove(shard) - recoveryTrace.finish() + ingestionRecoveryLatency.record(System.currentTimeMillis() - recoveryStart) case Failure(ex) => - recoveryTrace.fail(s"Recovery failed for dataset=$ref shard=$shard", ex) logger.error(s"Recovery failed for dataset=$ref shard=$shard", ex) handleError(ref, shard, ex) - recoveryTrace.finish() + ingestionRecoveryLatency.record(System.currentTimeMillis() - recoveryStart) }(actorDispatcher) fut } diff --git a/coordinator/src/main/scala/filodb.coordinator/NodeCoordinatorActor.scala b/coordinator/src/main/scala/filodb.coordinator/NodeCoordinatorActor.scala index fd5e211a5d..2dc8a3380f 100644 --- a/coordinator/src/main/scala/filodb.coordinator/NodeCoordinatorActor.scala +++ b/coordinator/src/main/scala/filodb.coordinator/NodeCoordinatorActor.scala @@ -8,6 +8,7 @@ import scala.concurrent.duration._ import akka.actor.{ActorRef, OneForOneStrategy, PoisonPill, Props, Terminated} import akka.actor.SupervisorStrategy.{Restart, Stop} import akka.event.LoggingReceive +import kamon.Kamon import net.ceedubs.ficus.Ficus._ import filodb.coordinator.client.MiscCommands @@ -195,6 +196,7 @@ private[filodb] final class NodeCoordinatorActor(metaStore: MetaStore, def queryHandlers: Receive = LoggingReceive { case q: QueryCommand => val originator = sender() + Kamon.currentSpan().mark("NodeCoordinatorActor received query") withQueryActor(originator, q.dataset) { _.tell(q, originator) } case QueryActor.ThrowException(dataset) => val originator = sender() diff --git a/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala b/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala index 5fb1e366b8..54d282efb4 100644 --- a/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala +++ b/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala @@ -5,9 +5,8 @@ import java.util.concurrent.{ForkJoinPool, ForkJoinWorkerThread} import scala.util.control.NonFatal -import akka.actor.{ActorRef, ActorSystem, Props} -import akka.dispatch.{Envelope, UnboundedStablePriorityMailbox} -import com.typesafe.config.Config +import akka.actor.{ActorRef, Props} +import akka.pattern.AskTimeoutException import kamon.Kamon import kamon.instrumentation.executor.ExecutorInstrumentation import kamon.tag.TagSet @@ -25,20 +24,6 @@ import filodb.core.store.CorruptVectorException import filodb.query._ import filodb.query.exec.ExecPlan -object QueryCommandPriority extends java.util.Comparator[Envelope] { - override def compare(o1: Envelope, o2: Envelope): Int = { - (o1.message, o2.message) match { - case (q1: QueryCommand, q2: QueryCommand) => q1.submitTime.compareTo(q2.submitTime) - case (_, _: QueryCommand) => -1 // non-query commands are admin and have higher priority - case (_: QueryCommand, _) => 1 // non-query commands are admin and have higher priority - case _ => 0 - } - } -} - -class QueryActorMailbox(settings: ActorSystem.Settings, config: Config) - extends UnboundedStablePriorityMailbox(QueryCommandPriority) - object QueryActor { final case class ThrowException(dataset: DatasetRef) @@ -46,7 +31,7 @@ object QueryActor { schemas: Schemas, shardMapFunc: => ShardMapper, earliestRawTimestampFn: => Long): Props = Props(new QueryActor(memStore, dsRef, schemas, - shardMapFunc, earliestRawTimestampFn)).withMailbox("query-actor-mailbox") + shardMapFunc, earliestRawTimestampFn)) } /** @@ -133,30 +118,52 @@ final class QueryActor(memStore: MemStore, def execPhysicalPlan2(q: ExecPlan, replyTo: ActorRef): Unit = { if (checkTimeout(q.queryContext, replyTo)) { epRequests.increment() - Kamon.currentSpan().tag("query", q.getClass.getSimpleName) - Kamon.currentSpan().tag("query-id", q.queryContext.queryId) - val querySession = QuerySession(q.queryContext, queryConfig) - q.execute(memStore, querySession)(queryScheduler) - .foreach { res => - FiloSchedulers.assertThreadName(QuerySchedName) - querySession.close() - replyTo ! res - res match { - case QueryResult(_, _, vectors) => resultVectors.record(vectors.length) - case e: QueryError => - queryErrors.increment() - logger.debug(s"queryId ${q.queryContext.queryId} Normal QueryError returned from query execution: $e") - e.t match { - case cve: CorruptVectorException => memStore.analyzeAndLogCorruptPtr(dsRef, cve) - case t: Throwable => - } - } - }(queryScheduler).recover { case ex => - querySession.close() - // Unhandled exception in query, should be rare - logger.error(s"queryId ${q.queryContext.queryId} Unhandled Query Error: ", ex) - replyTo ! QueryError(q.queryContext.queryId, ex) - }(queryScheduler) + val queryExecuteSpan = Kamon.spanBuilder(s"query-actor-exec-plan-execute-${q.getClass.getSimpleName}") + .asChildOf(Kamon.currentSpan()) + .start() + // Dont finish span since we finish it asynchronously when response is received + Kamon.runWithSpan(queryExecuteSpan, false) { + queryExecuteSpan.tag("query", q.getClass.getSimpleName) + queryExecuteSpan.tag("query-id", q.queryContext.queryId) + val querySession = QuerySession(q.queryContext, queryConfig) + queryExecuteSpan.mark("query-actor-received-execute-start") + q.execute(memStore, querySession)(queryScheduler) + .foreach { res => + FiloSchedulers.assertThreadName(QuerySchedName) + querySession.close() + replyTo ! res + res match { + case QueryResult(_, _, vectors) => resultVectors.record(vectors.length) + case e: QueryError => + queryErrors.increment() + queryExecuteSpan.fail(e.t.getMessage) + // error logging + e.t match { + case _: BadQueryException => // dont log user errors + case _: AskTimeoutException => // dont log ask timeouts. useless - let it simply flow up + case e: QueryTimeoutException => // log just message, no need for stacktrace + logger.error(s"queryId: ${q.queryContext.queryId} QueryTimeoutException: " + + s"${q.queryContext.origQueryParams} ${e.getMessage}") + case e: Throwable => + logger.error(s"queryId: ${q.queryContext.queryId} Query Error: " + + s"${q.queryContext.origQueryParams}", e) + } + // debug logging + e.t match { + case cve: CorruptVectorException => memStore.analyzeAndLogCorruptPtr(dsRef, cve) + case t: Throwable => + } + } + queryExecuteSpan.finish() + }(queryScheduler).recover { case ex => + querySession.close() + // Unhandled exception in query, should be rare + logger.error(s"queryId ${q.queryContext.queryId} Unhandled Query Error," + + s" query was ${q.queryContext.origQueryParams}", ex) + queryExecuteSpan.finish() + replyTo ! QueryError(q.queryContext.queryId, ex) + }(queryScheduler) + } } } @@ -202,11 +209,20 @@ final class QueryActor(memStore: MemStore, } } + private def execTopkCardinalityQuery(q: GetTopkCardinality, sender: ActorRef): Unit = { + try { + val ret = memStore.topKCardinality(q.dataset, q.shards, q.shardKeyPrefix, q.k) + sender ! ret + } catch { case e: Exception => + sender ! QueryError(s"Error Occurred", e) + } + } + def checkTimeout(queryContext: QueryContext, replyTo: ActorRef): Boolean = { // timeout can occur here if there is a build up in actor mailbox queue and delayed delivery val queryTimeElapsed = System.currentTimeMillis() - queryContext.submitTime - if (queryTimeElapsed >= queryContext.queryTimeoutMillis) { - replyTo ! QueryError(s"Query timeout, $queryTimeElapsed ms > ${queryContext.queryTimeoutMillis}", + if (queryTimeElapsed >= queryContext.plannerParams.queryTimeoutMillis) { + replyTo ! QueryError(s"Query timeout, $queryTimeElapsed ms > ${queryContext.plannerParams.queryTimeoutMillis}", QueryTimeoutException(queryTimeElapsed, this.getClass.getName)) false } else true @@ -218,6 +234,7 @@ final class QueryActor(memStore: MemStore, case q: ExplainPlan2Query => val replyTo = sender() processExplainPlanQuery(q, replyTo) case q: ExecPlan => execPhysicalPlan2(q, sender()) + case q: GetTopkCardinality => execTopkCardinalityQuery(q, sender()) case GetIndexNames(ref, limit, _) => sender() ! memStore.indexNames(ref, limit).map(_._1).toBuffer diff --git a/coordinator/src/main/scala/filodb.coordinator/client/QueryCommands.scala b/coordinator/src/main/scala/filodb.coordinator/client/QueryCommands.scala index 3504af2e5e..02750b4ca1 100644 --- a/coordinator/src/main/scala/filodb.coordinator/client/QueryCommands.scala +++ b/coordinator/src/main/scala/filodb.coordinator/client/QueryCommands.scala @@ -32,9 +32,11 @@ object QueryCommands { limit: Int = 100, submitTime: Long = System.currentTimeMillis()) extends QueryCommand - - - + final case class GetTopkCardinality(dataset: DatasetRef, + shards: Seq[Int], + shardKeyPrefix: Seq[String], + k: Int, + submitTime: Long = System.currentTimeMillis()) extends QueryCommand final case class StaticSpreadProvider(spreadChange: SpreadChange = SpreadChange()) extends SpreadProvider { def spreadFunc(filter: Seq[ColumnFilter]): Seq[SpreadChange] = { diff --git a/coordinator/src/main/scala/filodb.coordinator/client/QueryOps.scala b/coordinator/src/main/scala/filodb.coordinator/client/QueryOps.scala index cd29d6d54e..ae7a1d1125 100644 --- a/coordinator/src/main/scala/filodb.coordinator/client/QueryOps.scala +++ b/coordinator/src/main/scala/filodb.coordinator/client/QueryOps.scala @@ -5,8 +5,9 @@ import scala.concurrent.duration._ import com.typesafe.scalalogging.StrictLogging import filodb.core._ +import filodb.core.memstore.ratelimit.CardinalityRecord import filodb.core.query.QueryContext -import filodb.query.{LogicalPlan => LogicalPlan2, QueryResponse => QueryResponse2} +import filodb.query.{LogicalPlan => LogicalPlan2, QueryError, QueryResponse => QueryResponse2} trait QueryOps extends ClientBase with StrictLogging { import QueryCommands._ @@ -44,6 +45,16 @@ trait QueryOps extends ClientBase with StrictLogging { case s: Seq[(String, Int)] @unchecked => s } + def getTopkCardinality(dataset: DatasetRef, + shards: Seq[Int], + shardKeyPrefix: Seq[String], + k: Int, + timeout: FiniteDuration = 15.seconds): Seq[CardinalityRecord] = + askCoordinator(GetTopkCardinality(dataset, shards, shardKeyPrefix, k), timeout) { + case s: Seq[CardinalityRecord] @unchecked => s + case e: QueryError => throw e.t + } + /** * Asks the FiloDB node to perform a query using a LogicalPlan. * @param dataset the Dataset (and Database) to query @@ -58,7 +69,7 @@ trait QueryOps extends ClientBase with StrictLogging { // NOTE: It's very important to extend the query timeout for the ask itself, because the queryTimeoutMillis is // the internal FiloDB scatter-gather timeout. We need additional time for the proper error to get transmitted // back in case of internal timeouts. - askCoordinator(qCmd, (qContext.queryTimeoutMillis + 10000).millis) { case r: QueryResponse2 => r } + askCoordinator(qCmd, (qContext.plannerParams.queryTimeoutMillis + 10000).millis) { case r: QueryResponse2 => r } } } \ No newline at end of file diff --git a/coordinator/src/main/scala/filodb.coordinator/queryengine/Utils.scala b/coordinator/src/main/scala/filodb.coordinator/queryengine/Utils.scala index 082607322e..5530b510d0 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryengine/Utils.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryengine/Utils.scala @@ -60,7 +60,7 @@ object Utils extends StrictLogging { case FilteredPartitionQuery(filters) => // get limited # of shards if shard key available, otherwise query all shards // TODO: monitor ratio of queries using shardKeyHash to queries that go to all shards - val shards = options.shardOverrides.getOrElse { + val shards = options.plannerParams.shardOverrides.getOrElse { val shardCols = dataset.options.shardKeyColumns if (shardCols.length > 0) { shardHashFromFilters(filters, shardCols, dataset) match { diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/HighAvailabilityPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/HighAvailabilityPlanner.scala index ba4a7c7108..ff7b64709f 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/HighAvailabilityPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/HighAvailabilityPlanner.scala @@ -21,7 +21,9 @@ import filodb.query.exec._ class HighAvailabilityPlanner(dsRef: DatasetRef, localPlanner: QueryPlanner, failureProvider: FailureProvider, - queryConfig: QueryConfig) extends QueryPlanner with StrictLogging { + queryConfig: QueryConfig, + remoteExecHttpClient: RemoteExecHttpClient = RemoteHttpClient.defaultClient) + extends QueryPlanner with StrictLogging { import net.ceedubs.ficus.Ficus._ import LogicalPlanUtils._ @@ -71,19 +73,21 @@ class HighAvailabilityPlanner(dsRef: DatasetRef, val queryParams = qContext.origQueryParams.asInstanceOf[PromQlQueryParams] // Divide by 1000 to convert millis to seconds. PromQL params are in seconds. val promQlParams = PromQlQueryParams(queryParams.promQl, - (timeRange.startMs + offsetMs) / 1000, queryParams.stepSecs, (timeRange.endMs + offsetMs) / 1000, - queryParams.spread, processFailure = false) + (timeRange.startMs + offsetMs) / 1000, queryParams.stepSecs, (timeRange.endMs + offsetMs) / 1000) + val newQueryContext = qContext.copy(origQueryParams = promQlParams, plannerParams = qContext.plannerParams. + copy(processFailure = false) ) logger.debug("PromQlExec params:" + promQlParams) val httpEndpoint = remoteHttpEndpoint + queryParams.remoteQueryPath.getOrElse("") rootLogicalPlan match { case lp: LabelValues => MetadataRemoteExec(httpEndpoint, remoteHttpTimeoutMs, - PlannerUtil.getLabelValuesUrlParams(lp, queryParams), qContext, - InProcessPlanDispatcher, dsRef, promQlParams) + PlannerUtil.getLabelValuesUrlParams(lp, queryParams), newQueryContext, + InProcessPlanDispatcher, dsRef, remoteExecHttpClient) case lp: SeriesKeysByFilters => val urlParams = Map("match[]" -> queryParams.promQl) MetadataRemoteExec(httpEndpoint, remoteHttpTimeoutMs, - urlParams, qContext, InProcessPlanDispatcher, dsRef, promQlParams) + urlParams, newQueryContext, InProcessPlanDispatcher, + dsRef, remoteExecHttpClient) case _ => PromQlRemoteExec(httpEndpoint, remoteHttpTimeoutMs, - qContext, InProcessPlanDispatcher, dsRef, promQlParams) + newQueryContext, InProcessPlanDispatcher, dsRef, remoteExecHttpClient) } } @@ -111,7 +115,7 @@ class HighAvailabilityPlanner(dsRef: DatasetRef, if (!logicalPlan.isRoutable || !tsdbQueryParams.isInstanceOf[PromQlQueryParams] || // We don't know the promql issued (unusual) (tsdbQueryParams.isInstanceOf[PromQlQueryParams] - && !tsdbQueryParams.asInstanceOf[PromQlQueryParams].processFailure) || // This is a query that was + && !qContext.plannerParams.processFailure) || // This is a query that was // part of failure routing !hasSingleTimeRange(logicalPlan) || // Sub queries have different time ranges (unusual) failures.isEmpty) { // no failures in query time range diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala index 2b1cbd5392..2241ae349b 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala @@ -5,7 +5,7 @@ import com.typesafe.scalalogging.StrictLogging import filodb.coordinator.queryplanner.LogicalPlanUtils._ import filodb.core.metadata.Dataset import filodb.core.query.{PromQlQueryParams, QueryConfig, QueryContext} -import filodb.query.{BinaryJoin, LabelValues, LogicalPlan, SeriesKeysByFilters} +import filodb.query.{BinaryJoin, LabelValues, LogicalPlan, SeriesKeysByFilters, SetOperator} import filodb.query.exec._ case class PartitionAssignment(partitionName: String, endPoint: String, timeRange: TimeRange) @@ -20,20 +20,24 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider localPartitionPlanner: QueryPlanner, localPartitionName: String, dataset: Dataset, - queryConfig: QueryConfig) extends QueryPlanner with StrictLogging { + queryConfig: QueryConfig, + remoteExecHttpClient: RemoteExecHttpClient = RemoteHttpClient.defaultClient) + extends QueryPlanner with StrictLogging { import net.ceedubs.ficus.Ficus._ val remoteHttpTimeoutMs: Long = queryConfig.routingConfig.config.as[Option[Long]]("remote.http.timeout").getOrElse(60000) + val datasetMetricColumn: String = dataset.options.metricColumn + override def materialize(logicalPlan: LogicalPlan, qContext: QueryContext): ExecPlan = { val tsdbQueryParams = qContext.origQueryParams if(!tsdbQueryParams.isInstanceOf[PromQlQueryParams] || // We don't know the promql issued (unusual) (tsdbQueryParams.isInstanceOf[PromQlQueryParams] - && !tsdbQueryParams.asInstanceOf[PromQlQueryParams].processMultiPartition)) // Query was part of routing + && !qContext.plannerParams.processMultiPartition)) // Query was part of routing localPartitionPlanner.materialize(logicalPlan, qContext) else logicalPlan match { @@ -41,30 +45,32 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider case lp: LabelValues => materializeLabelValues(lp, qContext) case lp: SeriesKeysByFilters => materializeSeriesKeysFilters(lp, qContext) case _ => materializeSimpleQuery(logicalPlan, qContext) - } } private def getRoutingKeys(logicalPlan: LogicalPlan) = { val columnFilterGroup = LogicalPlan.getColumnFilterGroup(logicalPlan) - dataset.options.nonMetricShardColumns + val routingKeys = dataset.options.nonMetricShardColumns .map(x => (x, LogicalPlan.getColumnValues(columnFilterGroup, x))) + if (routingKeys.flatMap(_._2).isEmpty) Seq.empty else routingKeys } - private def generateRemoteExecParams(queryParams: PromQlQueryParams, startMs: Long, endMs: Long) = { - PromQlQueryParams(queryParams.promQl, startMs / 1000, queryParams.stepSecs, endMs / 1000, queryParams.spread, - queryParams.remoteQueryPath, queryParams.processFailure, processMultiPartition = false, queryParams.verbose) + private def generateRemoteExecParams(queryContext: QueryContext, startMs: Long, endMs: Long) = { + val queryParams = queryContext.origQueryParams.asInstanceOf[PromQlQueryParams] + queryContext.copy(origQueryParams = queryParams.copy(startSecs = startMs/1000, endSecs = endMs / 1000), + plannerParams = queryContext.plannerParams.copy(processMultiPartition = false)) } /** - * - * @param routingKeys Non Metric ShardColumns of dataset and value in logicalPlan - * @param queryParams PromQlQueryParams having query details - * @param logicalPlan Logical plan - */ - private def partitionUtil(routingKeys: Seq[(String, Set[String])], queryParams: PromQlQueryParams, - logicalPlan: LogicalPlan) = { - val routingKeyMap = routingKeys.map(x => (x._1, x._2.head)).toMap + * + * @param logicalPlan Logical plan + * @param queryParams PromQlQueryParams having query details + * @return Returns PartitionAssignment, lookback, offset and routing keys + */ + private def partitionUtilNonBinaryJoin(logicalPlan: LogicalPlan, queryParams: PromQlQueryParams) = { + + val routingKeys = getRoutingKeys(logicalPlan) + val offsetMs = LogicalPlanUtils.getOffsetMillis(logicalPlan) val periodicSeriesTimeWithOffset = TimeRange((queryParams.startSecs * 1000) - offsetMs, (queryParams.endSecs * 1000) - offsetMs) @@ -75,26 +81,53 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider val queryTimeRange = TimeRange(periodicSeriesTimeWithOffset.startMs - lookBackMs, periodicSeriesTimeWithOffset.endMs) - val partitions = partitionLocationProvider.getPartitions(routingKeyMap, queryTimeRange). - sortBy(_.timeRange.startMs) - if (partitions.isEmpty) new UnsupportedOperationException("No partitions found for routing keys: " + routingKeyMap) + val partitions = if (routingKeys.isEmpty) List.empty + else { + val routingKeyMap = routingKeys.map(x => (x._1, x._2.head)).toMap + partitionLocationProvider.getPartitions(routingKeyMap, queryTimeRange). + sortBy(_.timeRange.startMs) + } + if (partitions.isEmpty && !routingKeys.isEmpty) + new UnsupportedOperationException("No partitions found for routing keys: " + routingKeys) + + (partitions, lookBackMs, offsetMs, routingKeys) + } + /** + * @param queryParams PromQlQueryParams having query details + * @param logicalPlan Logical plan + * @return Returns PartitionAssignment and routing keys + */ + private def partitionUtil(queryParams: PromQlQueryParams, + logicalPlan: BinaryJoin): (List[PartitionAssignment], Seq[(String, Set[String])]) = { + + val lhsPartitionsAndRoutingKeys = logicalPlan.lhs match { + case b: BinaryJoin => partitionUtil(queryParams, b) + case _ => val p = partitionUtilNonBinaryJoin(logicalPlan.lhs, queryParams) + (p._1, p._4) + } - (partitions, lookBackMs, offsetMs) + val rhsPartitionsAndRoutingKeys = logicalPlan.rhs match { + case b: BinaryJoin => partitionUtil(queryParams, b) + case _ => val p = partitionUtilNonBinaryJoin(logicalPlan.rhs, queryParams) + (p._1, p._4) + } + (lhsPartitionsAndRoutingKeys._1 ++ rhsPartitionsAndRoutingKeys._1, + lhsPartitionsAndRoutingKeys._2 ++ rhsPartitionsAndRoutingKeys._2) } + /** * Materialize all queries except Binary Join and Metadata */ def materializeSimpleQuery(logicalPlan: LogicalPlan, qContext: QueryContext): ExecPlan = { - val routingKeys = getRoutingKeys(logicalPlan) + val queryParams = qContext.origQueryParams.asInstanceOf[PromQlQueryParams] + val (partitions, lookBackMs, offsetMs, routingKeys) = partitionUtilNonBinaryJoin(logicalPlan, queryParams) if (routingKeys.forall(_._2.isEmpty)) localPartitionPlanner.materialize(logicalPlan, qContext) else { - val queryParams = qContext.origQueryParams.asInstanceOf[PromQlQueryParams] val stepMs = queryParams.stepSecs * 1000 val isInstantQuery: Boolean = if (queryParams.startSecs == queryParams.endSecs) true else false - val (partitions, lookBackMs, offsetMs) = partitionUtil(routingKeys, queryParams, logicalPlan) var prevPartitionStart = queryParams.startSecs * 1000 val execPlans = partitions.zipWithIndex.map { case (p, i) => // First partition should start from query start time @@ -116,8 +149,8 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider copyLogicalPlanWithUpdatedTimeRange(logicalPlan, TimeRange(startMs, endMs)), qContext) else { val httpEndpoint = p.endPoint + queryParams.remoteQueryPath.getOrElse("") - PromQlRemoteExec(httpEndpoint, remoteHttpTimeoutMs, qContext, InProcessPlanDispatcher, dataset.ref, - generateRemoteExecParams(queryParams, startMs, endMs)) + PromQlRemoteExec(httpEndpoint, remoteHttpTimeoutMs, generateRemoteExecParams(qContext, startMs, endMs), + InProcessPlanDispatcher, dataset.ref, remoteExecHttpClient) } } if (execPlans.size == 1) execPlans.head @@ -128,28 +161,58 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider } } - def materializeBinaryJoin(logicalPlan: LogicalPlan, qContext: QueryContext): ExecPlan = { + def materializeMultiPartitionBinaryJoin(logicalPlan: BinaryJoin, qContext: QueryContext): ExecPlan = { + val lhsQueryContext = qContext.copy(origQueryParams = qContext.origQueryParams.asInstanceOf[PromQlQueryParams]. + copy(promQl = LogicalPlanParser.convertToQuery(logicalPlan.lhs))) + val rhsQueryContext = qContext.copy(origQueryParams = qContext.origQueryParams.asInstanceOf[PromQlQueryParams]. + copy(promQl = LogicalPlanParser.convertToQuery(logicalPlan.rhs))) - val routingKeys = getRoutingKeys(logicalPlan) + val lhsExec = logicalPlan.lhs match { + case b: BinaryJoin => materializeBinaryJoin(b, lhsQueryContext) + case _ => materializeSimpleQuery(logicalPlan.lhs, lhsQueryContext) + } + + val rhsExec = logicalPlan.rhs match { + case b: BinaryJoin => materializeBinaryJoin(b, rhsQueryContext) + case _ => materializeSimpleQuery(logicalPlan.rhs, rhsQueryContext) + } + + val onKeysReal = ExtraOnByKeysUtil.getRealOnLabels(logicalPlan, queryConfig.addExtraOnByKeysTimeRanges) + + if (logicalPlan.operator.isInstanceOf[SetOperator]) + SetOperatorExec(qContext, InProcessPlanDispatcher, Seq(lhsExec), Seq(rhsExec), logicalPlan.operator, + LogicalPlanUtils.renameLabels(onKeysReal, datasetMetricColumn), + LogicalPlanUtils.renameLabels(logicalPlan.ignoring, datasetMetricColumn), datasetMetricColumn) + else + BinaryJoinExec(qContext, InProcessPlanDispatcher, Seq(lhsExec), Seq(rhsExec), logicalPlan.operator, + logicalPlan.cardinality, LogicalPlanUtils.renameLabels(onKeysReal, datasetMetricColumn), + LogicalPlanUtils.renameLabels(logicalPlan.ignoring, datasetMetricColumn), + LogicalPlanUtils.renameLabels(logicalPlan.include, datasetMetricColumn), datasetMetricColumn) + + } + + def materializeBinaryJoin(logicalPlan: BinaryJoin, qContext: QueryContext): ExecPlan = { + + val queryParams = qContext.origQueryParams.asInstanceOf[PromQlQueryParams] + val (partitions, routingKeys) = partitionUtil(queryParams, logicalPlan) if (routingKeys.forall(_._2.isEmpty)) localPartitionPlanner.materialize(logicalPlan, qContext) else { - val queryParams = qContext.origQueryParams.asInstanceOf[PromQlQueryParams] - val partitions = partitionUtil(routingKeys, queryParams, logicalPlan)._1 val partitionName = partitions.head.partitionName - - // Binary Join supported only for single partition now + // Binary Join for single partition if (partitions.forall(_.partitionName.equals((partitionName)))) { if (partitionName.equals(localPartitionName)) localPartitionPlanner.materialize(logicalPlan, qContext) else { val httpEndpoint = partitions.head.endPoint + queryParams.remoteQueryPath.getOrElse("") - PromQlRemoteExec(httpEndpoint, remoteHttpTimeoutMs, qContext, InProcessPlanDispatcher, dataset.ref, - generateRemoteExecParams(queryParams, queryParams.startSecs * 1000, queryParams.endSecs * 1000)) + PromQlRemoteExec(httpEndpoint, remoteHttpTimeoutMs, generateRemoteExecParams(qContext, + queryParams.startSecs * 1000, queryParams.endSecs * 1000), + InProcessPlanDispatcher, dataset.ref, remoteExecHttpClient) } } - else throw new UnsupportedOperationException("Binary Join across multiple partitions not supported") + else materializeMultiPartitionBinaryJoin(logicalPlan, qContext) } } + def materializeSeriesKeysFilters(lp: SeriesKeysByFilters, qContext: QueryContext): ExecPlan = { val queryParams = qContext.origQueryParams.asInstanceOf[PromQlQueryParams] val partitions = partitionLocationProvider.getAuthorizedPartitions( @@ -184,10 +247,11 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider private def createMetadataRemoteExec(qContext: QueryContext, queryParams: PromQlQueryParams, partitionAssignment: PartitionAssignment, urlParams: Map[String, String]) = { - val finalQueryParams = generateRemoteExecParams( - queryParams, partitionAssignment.timeRange.startMs, partitionAssignment.timeRange.endMs) - val httpEndpoint = partitionAssignment.endPoint + finalQueryParams.remoteQueryPath.getOrElse("") + val finalQueryContext = generateRemoteExecParams( + qContext, partitionAssignment.timeRange.startMs, partitionAssignment.timeRange.endMs) + val httpEndpoint = partitionAssignment.endPoint + finalQueryContext.origQueryParams.asInstanceOf[PromQlQueryParams]. + remoteQueryPath.getOrElse("") MetadataRemoteExec(httpEndpoint, remoteHttpTimeoutMs, - urlParams, qContext, InProcessPlanDispatcher, dataset.ref, finalQueryParams) + urlParams, finalQueryContext, InProcessPlanDispatcher, dataset.ref, remoteExecHttpClient) } } diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/PlannerMaterializer.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/PlannerMaterializer.scala index b5ed10a9ca..a0d0a0548c 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/PlannerMaterializer.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/PlannerMaterializer.scala @@ -2,6 +2,8 @@ package filodb.coordinator.queryplanner import java.util.concurrent.ThreadLocalRandom +import com.typesafe.scalalogging.StrictLogging + import filodb.core.metadata.{DatasetOptions, Schemas} import filodb.core.query.{PromQlQueryParams, QueryContext, RangeParams} import filodb.prometheus.ast.Vectors.PromMetricLabel @@ -141,7 +143,8 @@ trait PlannerMaterializer { } } -object PlannerUtil { +object PlannerUtil extends StrictLogging { + /** * Returns URL params for label values which is used to create Metadata remote exec plan */ @@ -152,4 +155,5 @@ object PlannerUtil { head}$quote"""}.mkString(",") Map("filter" -> filters, "labels" -> lp.labelNames.mkString(",")) } + } diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/QueryPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/QueryPlanner.scala index 3182059c01..4e3f03b892 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/QueryPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/QueryPlanner.scala @@ -33,6 +33,7 @@ trait QueryPlanner { // Please note that the following needs to be wrapped inside `runWithSpan` so that the context will be propagated // across threads. Note that task/observable will not run on the thread where span is present since // kamon uses thread-locals. + // Dont finish span since this code didnt create it Kamon.runWithSpan(parentSpan, false) { execPlan.dispatcher.dispatch(execPlan) } diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlanner.scala index b1ccf0ecb6..8a97355633 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlanner.scala @@ -1,7 +1,7 @@ package filodb.coordinator.queryplanner import filodb.core.metadata.{Dataset, Schemas} -import filodb.core.query.{ColumnFilter, PromQlQueryParams, QueryContext} +import filodb.core.query.{ColumnFilter, PromQlQueryParams, QueryConfig, QueryContext, RangeParams} import filodb.query._ import filodb.query.exec._ @@ -26,10 +26,13 @@ case class ShardKeyMatcher(columnFilters: Seq[ColumnFilter], query: String) class ShardKeyRegexPlanner(dataset: Dataset, queryPlanner: QueryPlanner, - shardKeyMatcher: Seq[ColumnFilter] => Seq[Seq[ColumnFilter]]) + shardKeyMatcher: Seq[ColumnFilter] => Seq[Seq[ColumnFilter]], + queryConfig: QueryConfig) extends QueryPlanner with PlannerMaterializer { + val datasetMetricColumn = dataset.options.metricColumn override val schemas = Schemas(dataset.schema) + /** * Converts a logical plan to execution plan. * @@ -65,24 +68,45 @@ class ShardKeyRegexPlanner(dataset: Dataset, private def generateExecWithoutRegex(logicalPlan: LogicalPlan, nonMetricShardKeyFilters: Seq[ColumnFilter], qContext: QueryContext): Seq[ExecPlan] = { val queryParams = qContext.origQueryParams.asInstanceOf[PromQlQueryParams] - shardKeyMatcher(nonMetricShardKeyFilters).map { result => + val shardKeyMatches = shardKeyMatcher(nonMetricShardKeyFilters) + val skipAggregatePresentValue = if (shardKeyMatches.length == 1) false else true + shardKeyMatches.map { result => val newLogicalPlan = logicalPlan.replaceFilters(result) // Querycontext should just have the part of query which has regex // For example for exp(sum(test{_ws_ = "demo", _ns_ =~ "App.*"})), sub queries should be // sum(test{_ws_ = "demo", _ns_ = "App-1"}), sum(test{_ws_ = "demo", _ns_ = "App-2"}) etc val newQueryParams = queryParams.copy(promQl = LogicalPlanParser.convertToQuery(newLogicalPlan)) - val newQueryContext = qContext.copy(origQueryParams = newQueryParams) + val newQueryContext = qContext.copy(origQueryParams = newQueryParams, plannerParams = qContext.plannerParams. + copy(skipAggregatePresent = skipAggregatePresentValue)) queryPlanner.materialize(logicalPlan.replaceFilters(result), newQueryContext) } } /** * For binary join queries like test1{_ws_ = "demo", _ns_ =~ "App.*"} + test2{_ws_ = "demo", _ns_ =~ "App.*"}) + * LHS and RHS could be across multiple partitions */ - private def materializeBinaryJoin(binaryJoin: BinaryJoin, qContext: QueryContext): PlanResult = { - if (LogicalPlan.hasShardKeyEqualsOnly(binaryJoin, dataset.options.nonMetricShardColumns)) - PlanResult(Seq(queryPlanner.materialize(binaryJoin, qContext))) - else throw new UnsupportedOperationException("Regex not supported for Binary Join") + private def materializeBinaryJoin(logicalPlan: BinaryJoin, qContext: QueryContext): PlanResult = { + val lhsQueryContext = qContext.copy(origQueryParams = qContext.origQueryParams.asInstanceOf[PromQlQueryParams]. + copy(promQl = LogicalPlanParser.convertToQuery(logicalPlan.lhs))) + val rhsQueryContext = qContext.copy(origQueryParams = qContext.origQueryParams.asInstanceOf[PromQlQueryParams]. + copy(promQl = LogicalPlanParser.convertToQuery(logicalPlan.rhs))) + + val lhsExec = materialize(logicalPlan.lhs, lhsQueryContext) + val rhsExec = materialize(logicalPlan.rhs, rhsQueryContext) + + val onKeysReal = ExtraOnByKeysUtil.getRealOnLabels(logicalPlan, queryConfig.addExtraOnByKeysTimeRanges) + + val execPlan = if (logicalPlan.operator.isInstanceOf[SetOperator]) + SetOperatorExec(qContext, InProcessPlanDispatcher, Seq(lhsExec), Seq(rhsExec), logicalPlan.operator, + LogicalPlanUtils.renameLabels(onKeysReal, datasetMetricColumn), + LogicalPlanUtils.renameLabels(logicalPlan.ignoring, datasetMetricColumn), datasetMetricColumn) + else + BinaryJoinExec(qContext, InProcessPlanDispatcher, Seq(lhsExec), Seq(rhsExec), logicalPlan.operator, + logicalPlan.cardinality, LogicalPlanUtils.renameLabels(onKeysReal, datasetMetricColumn), + LogicalPlanUtils.renameLabels(logicalPlan.ignoring, datasetMetricColumn), + LogicalPlanUtils.renameLabels(logicalPlan.include, datasetMetricColumn), datasetMetricColumn) + PlanResult(Seq(execPlan)) } /*** @@ -95,9 +119,14 @@ class ShardKeyRegexPlanner(dataset: Dataset, LogicalPlan.getNonMetricShardKeyFilters(aggregate, dataset.options.nonMetricShardColumns).head, queryContext) val exec = if (execPlans.size == 1) execPlans.head else { + if (aggregate.operator.equals(AggregationOperator.TopK) || aggregate.operator.equals(AggregationOperator.BottomK) + || aggregate.operator.equals(AggregationOperator.CountValues)) + throw new UnsupportedOperationException(s"Shard Key regex not supported for ${aggregate.operator}") val reducer = MultiPartitionReduceAggregateExec(queryContext, InProcessPlanDispatcher, execPlans.sortWith((x, y) => !x.isInstanceOf[PromQlRemoteExec]), aggregate.operator, aggregate.params) - reducer.addRangeVectorTransformer(AggregatePresenter(aggregate.operator, aggregate.params)) + val promQlQueryParams = queryContext.origQueryParams.asInstanceOf[PromQlQueryParams] + reducer.addRangeVectorTransformer(AggregatePresenter(aggregate.operator, aggregate.params, + RangeParams(promQlQueryParams.startSecs, promQlQueryParams.stepSecs, promQlQueryParams.endSecs))) reducer } PlanResult(Seq(exec)) diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala index 09e4fc0739..590e037dbe 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala @@ -11,10 +11,10 @@ import filodb.coordinator.client.QueryCommands.StaticSpreadProvider import filodb.core.{DatasetRef, SpreadProvider} import filodb.core.binaryrecord2.RecordBuilder import filodb.core.metadata.Schemas -import filodb.core.query.{ColumnFilter, Filter, QueryConfig, QueryContext} +import filodb.core.query.{ColumnFilter, Filter, PromQlQueryParams, QueryConfig, QueryContext, RangeParams} import filodb.core.store.{AllChunkScan, ChunkScanMethod, InMemoryChunkScan, TimeRangeChunkScan, WriteBufferChunkScan} import filodb.prometheus.ast.Vectors.{PromMetricLabel, TypeLabel} -import filodb.prometheus.ast.WindowConstants +import filodb.prometheus.ast. WindowConstants import filodb.query.{exec, _} import filodb.query.exec.{LocalPartitionDistConcatExec, _} @@ -106,12 +106,12 @@ class SingleClusterPlanner(dsRef: DatasetRef, private def shardsFromFilters(filters: Seq[ColumnFilter], qContext: QueryContext): Seq[Int] = { - val spreadProvToUse = qContext.spreadOverride.getOrElse(spreadProvider) + val spreadProvToUse = qContext.plannerParams.spreadOverride.getOrElse(spreadProvider) - require(shardColumns.nonEmpty || qContext.shardOverrides.nonEmpty, + require(shardColumns.nonEmpty || qContext.plannerParams.shardOverrides.nonEmpty, s"Dataset $dsRef does not have shard columns defined, and shard overrides were not mentioned") - qContext.shardOverrides.getOrElse { + qContext.plannerParams.shardOverrides.getOrElse { val shardVals = shardColumns.map { shardCol => // So to compute the shard hash we need shardCol == value filter (exact equals) for each shardColumn filters.find(f => f.column == shardCol) match { @@ -258,7 +258,11 @@ class SingleClusterPlanner(dsRef: DatasetRef, val reduceDispatcher = pickDispatcher(toReduceLevel2) val reducer = LocalPartitionReduceAggregateExec(qContext, reduceDispatcher, toReduceLevel2, lp.operator, lp.params) - reducer.addRangeVectorTransformer(AggregatePresenter(lp.operator, lp.params)) + val promQlQueryParams = qContext.origQueryParams.asInstanceOf[PromQlQueryParams] + + if (!qContext.plannerParams.skipAggregatePresent) + reducer.addRangeVectorTransformer(AggregatePresenter(lp.operator, lp.params, RangeParams( + promQlQueryParams.startSecs, promQlQueryParams.stepSecs, promQlQueryParams.endSecs))) PlanResult(Seq(reducer), false) // since we have aggregated, no stitching } @@ -342,7 +346,7 @@ class SingleClusterPlanner(dsRef: DatasetRef, private def materializeRawSeries(qContext: QueryContext, lp: RawSeries): PlanResult = { - val spreadProvToUse = qContext.spreadOverride.getOrElse(spreadProvider) + val spreadProvToUse = qContext.plannerParams.spreadOverride.getOrElse(spreadProvider) val offsetMillis: Long = lp.offsetMs.getOrElse(0) val colName = lp.columns.headOption val (renamedFilters, schemaOpt) = extractSchemaFilter(renameMetricFilter(lp.filters)) diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SinglePartitionPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SinglePartitionPlanner.scala index 7c27d89a1f..3be9cfbf28 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SinglePartitionPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SinglePartitionPlanner.scala @@ -39,38 +39,61 @@ class SinglePartitionPlanner(planners: Map[String, QueryPlanner], if(planner.isEmpty) planners.values.head else planner.head } - private def materializeSimpleQuery(logicalPlan: LogicalPlan, qContext: QueryContext): ExecPlan = { - getPlanner(logicalPlan).materialize(logicalPlan, qContext) - } + /** + * Returns lhs and rhs planners of BinaryJoin + */ + private def getBinaryJoinPlanners(binaryJoin: BinaryJoin) : Seq[QueryPlanner] = { + val lhsPlanners = binaryJoin.lhs match { + case b: BinaryJoin => getBinaryJoinPlanners(b) + case _ => Seq(getPlanner(binaryJoin.lhs)) - private def materializeBinaryJoin(logicalPlan: BinaryJoin, qContext: QueryContext): ExecPlan = { + } - val lhsQueryContext = qContext.copy(origQueryParams = qContext.origQueryParams.asInstanceOf[PromQlQueryParams]. - copy(promQl = LogicalPlanParser.convertToQuery(logicalPlan.lhs))) - val rhsQueryContext = qContext.copy(origQueryParams = qContext.origQueryParams.asInstanceOf[PromQlQueryParams]. - copy(promQl = LogicalPlanParser.convertToQuery(logicalPlan.rhs))) + val rhsPlanners = binaryJoin.rhs match { + case b: BinaryJoin => getBinaryJoinPlanners(b) + case _ => Seq(getPlanner(binaryJoin.rhs)) - val lhsExec = logicalPlan.lhs match { - case b: BinaryJoin => materializeBinaryJoin(b, lhsQueryContext) - case _ => getPlanner(logicalPlan.lhs).materialize(logicalPlan.lhs, lhsQueryContext) } + lhsPlanners ++ rhsPlanners + } - val rhsExec = logicalPlan.rhs match { - case b: BinaryJoin => materializeBinaryJoin(b, rhsQueryContext) - case _ => getPlanner(logicalPlan.rhs).materialize(logicalPlan.rhs, rhsQueryContext) - } + private def materializeSimpleQuery(logicalPlan: LogicalPlan, qContext: QueryContext): ExecPlan = { + getPlanner(logicalPlan).materialize(logicalPlan, qContext) + } - val onKeysReal = ExtraOnByKeysUtil.getRealOnLabels(logicalPlan, queryConfig.addExtraOnByKeysTimeRanges) - - if (logicalPlan.operator.isInstanceOf[SetOperator]) - SetOperatorExec(qContext, InProcessPlanDispatcher, Seq(lhsExec), Seq(rhsExec), logicalPlan.operator, - LogicalPlanUtils.renameLabels(onKeysReal, datasetMetricColumn), - LogicalPlanUtils.renameLabels(logicalPlan.ignoring, datasetMetricColumn), datasetMetricColumn) - else - BinaryJoinExec(qContext, InProcessPlanDispatcher, Seq(lhsExec), Seq(rhsExec), logicalPlan.operator, - logicalPlan.cardinality, LogicalPlanUtils.renameLabels(onKeysReal, datasetMetricColumn), - LogicalPlanUtils.renameLabels(logicalPlan.ignoring, datasetMetricColumn), - LogicalPlanUtils.renameLabels(logicalPlan.include, datasetMetricColumn), datasetMetricColumn) + private def materializeBinaryJoin(logicalPlan: BinaryJoin, qContext: QueryContext): ExecPlan = { + val allPlanners = getBinaryJoinPlanners(logicalPlan) + + if (allPlanners.forall(_.equals(allPlanners.head))) allPlanners.head.materialize(logicalPlan, qContext) + else { + + val lhsQueryContext = qContext.copy(origQueryParams = qContext.origQueryParams.asInstanceOf[PromQlQueryParams]. + copy(promQl = LogicalPlanParser.convertToQuery(logicalPlan.lhs))) + val rhsQueryContext = qContext.copy(origQueryParams = qContext.origQueryParams.asInstanceOf[PromQlQueryParams]. + copy(promQl = LogicalPlanParser.convertToQuery(logicalPlan.rhs))) + + val lhsExec = logicalPlan.lhs match { + case b: BinaryJoin => materializeBinaryJoin(b, lhsQueryContext) + case _ => getPlanner(logicalPlan.lhs).materialize(logicalPlan.lhs, lhsQueryContext) + } + + val rhsExec = logicalPlan.rhs match { + case b: BinaryJoin => materializeBinaryJoin(b, rhsQueryContext) + case _ => getPlanner(logicalPlan.rhs).materialize(logicalPlan.rhs, rhsQueryContext) + } + + val onKeysReal = ExtraOnByKeysUtil.getRealOnLabels(logicalPlan, queryConfig.addExtraOnByKeysTimeRanges) + + if (logicalPlan.operator.isInstanceOf[SetOperator]) + SetOperatorExec(qContext, InProcessPlanDispatcher, Seq(lhsExec), Seq(rhsExec), logicalPlan.operator, + LogicalPlanUtils.renameLabels(onKeysReal, datasetMetricColumn), + LogicalPlanUtils.renameLabels(logicalPlan.ignoring, datasetMetricColumn), datasetMetricColumn) + else + BinaryJoinExec(qContext, InProcessPlanDispatcher, Seq(lhsExec), Seq(rhsExec), logicalPlan.operator, + logicalPlan.cardinality, LogicalPlanUtils.renameLabels(onKeysReal, datasetMetricColumn), + LogicalPlanUtils.renameLabels(logicalPlan.ignoring, datasetMetricColumn), + LogicalPlanUtils.renameLabels(logicalPlan.include, datasetMetricColumn), datasetMetricColumn) + } } private def materializeLabelValues(logicalPlan: LogicalPlan, qContext: QueryContext) = { diff --git a/coordinator/src/multi-jvm/scala/filodb.coordinator/ClusterRecoverySpec.scala b/coordinator/src/multi-jvm/scala/filodb.coordinator/ClusterRecoverySpec.scala index 31ff578857..212c80edc2 100644 --- a/coordinator/src/multi-jvm/scala/filodb.coordinator/ClusterRecoverySpec.scala +++ b/coordinator/src/multi-jvm/scala/filodb.coordinator/ClusterRecoverySpec.scala @@ -10,7 +10,7 @@ import com.typesafe.config.ConfigFactory import org.scalatest.time.{Millis, Seconds, Span} import filodb.core._ import filodb.core.metadata.Column.ColumnType -import filodb.core.query.{ColumnInfo, QueryContext} +import filodb.core.query.{ColumnInfo, PlannerParams, QueryContext} object ClusterRecoverySpecConfig extends MultiNodeConfig { // register the named roles (nodes) of the test @@ -145,7 +145,7 @@ abstract class ClusterRecoverySpec extends ClusterSpec(ClusterRecoverySpecConfig // val query = LogicalPlanQuery(dataset6.ref, // simpleAgg("count", childPlan=PartitionsRange.all(FilteredPartitionQuery(Nil), Seq("MonthYear")))) - val qOpt = QueryContext(shardOverrides = Some(Seq(0, 1))) + val qOpt = QueryContext(plannerParams = PlannerParams(shardOverrides = Some(Seq(0, 1)))) val q2 = LogicalPlan2Query(dataset6.ref, PeriodicSeriesWithWindowing( RawSeries(AllChunksSelector, Nil, Seq("AvgTone"), Some(300000), None), diff --git a/coordinator/src/test/scala/filodb.coordinator/NodeCoordinatorActorSpec.scala b/coordinator/src/test/scala/filodb.coordinator/NodeCoordinatorActorSpec.scala index 48b70cf1ae..d3a74a002e 100644 --- a/coordinator/src/test/scala/filodb.coordinator/NodeCoordinatorActorSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/NodeCoordinatorActorSpec.scala @@ -122,7 +122,8 @@ class NodeCoordinatorActorSpec extends ActorTest(NodeCoordinatorActorSpec.getNew val timeMinSchema = ResultSchema(Seq(ColumnInfo("timestamp", TimestampColumn), ColumnInfo("min", DoubleColumn)), 1) val countSchema = ResultSchema(Seq(ColumnInfo("timestamp", TimestampColumn), ColumnInfo("count", DoubleColumn)), 1) val valueSchema = ResultSchema(Seq(ColumnInfo("timestamp", TimestampColumn), ColumnInfo("value", DoubleColumn)), 1) - val qOpt = QueryContext(shardOverrides = Some(Seq(0))) + val qOpt = QueryContext(plannerParams = PlannerParams(shardOverrides = Some(Seq(0))), origQueryParams = + PromQlQueryParams("", 1000, 1, 1000)) describe("QueryActor commands and responses") { import MachineMetricsData._ @@ -275,7 +276,8 @@ class NodeCoordinatorActorSpec extends ActorTest(NodeCoordinatorActorSpec.getNew // Should return results from both shards // shard 1 - timestamps 110000 -< 130000; shard 2 - timestamps 130000 <- 1400000 - val queryOpt = QueryContext(shardOverrides = Some(Seq(0, 1))) + val queryOpt = QueryContext(plannerParams = PlannerParams(shardOverrides = Some(Seq(0, 1))), + origQueryParams = PromQlQueryParams("", 1000, 1, 1000)) val series2 = (2 to 4).map(n => s"Series $n").toSet.asInstanceOf[Set[Any]] val multiFilter = Seq(ColumnFilter("series", Filter.In(series2))) val q2 = LogicalPlan2Query(ref, @@ -302,7 +304,7 @@ class NodeCoordinatorActorSpec extends ActorTest(NodeCoordinatorActorSpec.getNew memStore.refreshIndexForTesting(dataset1.ref) - val queryOpt = QueryContext(shardOverrides = Some(Seq(0, 1))) + val queryOpt = QueryContext(plannerParams = PlannerParams(shardOverrides = Some(Seq(0, 1)))) val series2 = (2 to 4).map(n => s"Series $n") val multiFilter = Seq(ColumnFilter("series", Filter.In(series2.toSet.asInstanceOf[Set[Any]]))) val q2 = LogicalPlan2Query(ref, RawSeries(AllChunksSelector, multiFilter, Seq("min"), Some(300000), None), diff --git a/coordinator/src/test/scala/filodb.coordinator/client/SerializationSpec.scala b/coordinator/src/test/scala/filodb.coordinator/client/SerializationSpec.scala index 876183ce46..3aeb63553f 100644 --- a/coordinator/src/test/scala/filodb.coordinator/client/SerializationSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/client/SerializationSpec.scala @@ -204,8 +204,8 @@ class SerializationSpec extends ActorTest(SerializationSpecConfig.getNewSystem) val windowed2 = PeriodicSeriesWithWindowing(raw2, from, 1000, to, 5000, RangeFunctionId.Rate) val summed2 = Aggregate(AggregationOperator.Sum, windowed2, Nil, Seq("job")) val logicalPlan = BinaryJoin(summed1, BinaryOperator.DIV, Cardinality.OneToOne, summed2) - val execPlan = engine.materialize(logicalPlan, QueryContext(Some(StaticSpreadProvider(SpreadChange(0, 0))), - 100)) + val execPlan = engine.materialize(logicalPlan, QueryContext(plannerParams = PlannerParams(Some(StaticSpreadProvider + (SpreadChange(0, 0))), 100), origQueryParams = PromQlQueryParams("", from/1000, 1000, to/1000))) roundTrip(execPlan) shouldEqual execPlan } @@ -224,8 +224,10 @@ class SerializationSpec extends ActorTest(SerializationSpecConfig.getNewSystem) val logicalPlan1 = Parser.queryRangeToLogicalPlan( s"""sum(rate(http_request_duration_seconds_bucket{job="prometheus",$shardKeyStr}[20s])) by (handler)""", qParams) - val execPlan1 = engine.materialize(logicalPlan1, QueryContext(Some(new StaticSpreadProvider(SpreadChange(0, 0))), - 100)) + val execPlan1 = engine.materialize(logicalPlan1, QueryContext(origQueryParams = PromQlQueryParams( + s"""sum(rate(http_request_duration_seconds_bucket{job="prometheus",$shardKeyStr}[20s])) by (handler)""", + from, 10, to), plannerParams = PlannerParams(Some(new StaticSpreadProvider(SpreadChange(0, 0))), + 100))) roundTrip(execPlan1) shouldEqual execPlan1 // scalastyle:off @@ -233,7 +235,11 @@ class SerializationSpec extends ActorTest(SerializationSpecConfig.getNewSystem) s"""sum(rate(http_request_duration_microseconds_sum{job="prometheus",$shardKeyStr}[5m])) by (handler) / sum(rate(http_request_duration_microseconds_count{job="prometheus",$shardKeyStr}[5m])) by (handler)""", qParams) // scalastyle:on - val execPlan2 = engine.materialize(logicalPlan2, QueryContext(Some(new StaticSpreadProvider(SpreadChange(0, 0))), 100)) + val execPlan2 = engine.materialize(logicalPlan2, QueryContext(origQueryParams = PromQlQueryParams( + s"""sum(rate(http_request_duration_microseconds_sum{job="prometheus",$shardKeyStr}[5m])) by (handler) / + |sum(rate(http_request_duration_microseconds_count{job="prometheus",$shardKeyStr}[5m])) by (handler)""".stripMargin, + from, 10, to), plannerParams = PlannerParams(Some(new StaticSpreadProvider(SpreadChange(0, 0))), + 100))) roundTrip(execPlan2) shouldEqual execPlan2 } @@ -337,7 +343,7 @@ class SerializationSpec extends ActorTest(SerializationSpecConfig.getNewSystem) qParams) val param = PromQlQueryParams("test", 1000, 200, 5000) val execPlan = engine.materialize(logicalPlan, QueryContext(origQueryParams = param, - spreadOverride = Some(new StaticSpreadProvider(SpreadChange(0, 0))))) + plannerParams = PlannerParams(spreadOverride = Some(new StaticSpreadProvider(SpreadChange(0, 0)))))) roundTrip(execPlan) shouldEqual execPlan } } diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/HighAvailabilityPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/HighAvailabilityPlannerSpec.scala index 602b43ca79..a2689a31f8 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/HighAvailabilityPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/HighAvailabilityPlannerSpec.scala @@ -31,8 +31,8 @@ class HighAvailabilityPlannerSpec extends AnyFunSpec with Matchers { private val dsRef = dataset.ref private val schemas = Schemas(dataset.schema) - private val routingConfigString = "routing {\n remote {\n " + - " http {\n endpoint = localhost\n timeout = 10000\n }\n }\n}" + private val routingConfigString = "routing {\n remote {\n http {\n" + + " endpoint = localhost\n timeout = 10000\n }\n }\n}" private val routingConfig = ConfigFactory.parseString(routingConfigString) @@ -113,8 +113,9 @@ class HighAvailabilityPlannerSpec extends AnyFunSpec with Matchers { val execPlan = engine.materialize(summed, QueryContext(origQueryParams = promQlQueryParams)) execPlan.isInstanceOf[PromQlRemoteExec] shouldEqual (true) - execPlan.asInstanceOf[PromQlRemoteExec].params.startSecs shouldEqual(from/1000) - execPlan.asInstanceOf[PromQlRemoteExec].params.endSecs shouldEqual(to/1000) + val queryParams = execPlan.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams] + queryParams.startSecs shouldEqual(from/1000) + queryParams.endSecs shouldEqual(to/1000) } it("should generate RemoteExecPlan with RawSeries time according to lookBack") { @@ -162,10 +163,12 @@ class HighAvailabilityPlannerSpec extends AnyFunSpec with Matchers { l1.rangeVectorTransformers(0).asInstanceOf[PeriodicSamplesMapper].end shouldEqual (2000000) l1.rangeVectorTransformers(1).isInstanceOf[AggregateMapReduce] shouldEqual true } + val queryParams = child2.queryContext.origQueryParams. + asInstanceOf[PromQlQueryParams] - child2.params.startSecs shouldEqual from/1000 - child2.params.endSecs shouldEqual (1060000-1)/1000 - child2.params.processFailure shouldEqual(false) + queryParams.startSecs shouldEqual from/1000 + queryParams.endSecs shouldEqual (1060000-1)/1000 + child2.queryContext.plannerParams.processFailure shouldEqual(false) } it("should generate only PromQlExec when local failure starts before query time") { @@ -188,8 +191,10 @@ class HighAvailabilityPlannerSpec extends AnyFunSpec with Matchers { val execPlan = engine.materialize(summed, QueryContext(origQueryParams = promQlQueryParams)) execPlan.isInstanceOf[PromQlRemoteExec] shouldEqual (true) - execPlan.asInstanceOf[PromQlRemoteExec].params.startSecs shouldEqual(from/1000) - execPlan.asInstanceOf[PromQlRemoteExec].params.endSecs shouldEqual(to/1000) + val queryParams = execPlan.asInstanceOf[PromQlRemoteExec].queryContext.origQueryParams. + asInstanceOf[PromQlQueryParams] + queryParams.startSecs shouldEqual(from/1000) + queryParams.endSecs shouldEqual(to/1000) } it("should generate only PromQlExec when local failure timerange coincide with query time range") { @@ -209,11 +214,13 @@ class HighAvailabilityPlannerSpec extends AnyFunSpec with Matchers { val engine = new HighAvailabilityPlanner(dsRef, localPlanner, failureProvider, queryConfig) - val execPlan = engine.materialize(summed, QueryContext(origQueryParams = promQlQueryParams)) + val execPlan = engine.materialize(summed, QueryContext(origQueryParams = promQlQueryParams)) execPlan.isInstanceOf[PromQlRemoteExec] shouldEqual (true) - execPlan.asInstanceOf[PromQlRemoteExec].params.startSecs shouldEqual(from/1000) - execPlan.asInstanceOf[PromQlRemoteExec].params.endSecs shouldEqual(to/1000) + val queryParams = execPlan.asInstanceOf[PromQlRemoteExec].queryContext.origQueryParams. + asInstanceOf[PromQlQueryParams] + queryParams.startSecs shouldEqual(from/1000) + queryParams.endSecs shouldEqual(to/1000) } it("should generate only PromQlExec when local failure starts before query end time and ends after query end time") { @@ -236,8 +243,10 @@ class HighAvailabilityPlannerSpec extends AnyFunSpec with Matchers { val execPlan = engine.materialize(summed, QueryContext(origQueryParams = promQlQueryParams)) execPlan.isInstanceOf[PromQlRemoteExec] shouldEqual (true) - execPlan.asInstanceOf[PromQlRemoteExec].params.startSecs shouldEqual(from/1000) - execPlan.asInstanceOf[PromQlRemoteExec].params.endSecs shouldEqual(to/1000) + val queryParams =execPlan.asInstanceOf[PromQlRemoteExec].queryContext.origQueryParams. + asInstanceOf[PromQlQueryParams] + queryParams.startSecs shouldEqual(from/1000) + queryParams.endSecs shouldEqual(to/1000) } it("should generate PromQlExecPlan and LocalPlan with RawSeries time according to lookBack and step") { @@ -289,10 +298,12 @@ class HighAvailabilityPlannerSpec extends AnyFunSpec with Matchers { l1.rangeVectorTransformers(1).isInstanceOf[AggregateMapReduce] shouldEqual true } - child2.params.startSecs shouldEqual 900 - child2.params.endSecs shouldEqual 1020 - child2.params.stepSecs shouldEqual 60 - child2.params.processFailure shouldEqual(false) + val queryParams = child2.asInstanceOf[PromQlRemoteExec].queryContext.origQueryParams. + asInstanceOf[PromQlQueryParams] + queryParams.startSecs shouldEqual 900 + queryParams.endSecs shouldEqual 1020 + queryParams.stepSecs shouldEqual 60 + child2.asInstanceOf[PromQlRemoteExec].queryContext.plannerParams.processFailure shouldEqual(false) } it("should generate only PromQlExecPlan when second remote ends after query end time") { @@ -321,11 +332,12 @@ class HighAvailabilityPlannerSpec extends AnyFunSpec with Matchers { execPlan.isInstanceOf[PromQlRemoteExec] shouldEqual true val child = execPlan.asInstanceOf[PromQlRemoteExec] - - child.params.startSecs shouldEqual 900 - child.params.endSecs shouldEqual 1980 - child.params.stepSecs shouldEqual 60 - child.params.processFailure shouldEqual false + val queryParams = child.asInstanceOf[PromQlRemoteExec].queryContext.origQueryParams. + asInstanceOf[PromQlQueryParams] + queryParams.startSecs shouldEqual 900 + queryParams.endSecs shouldEqual 1980 + queryParams.stepSecs shouldEqual 60 + child.asInstanceOf[PromQlRemoteExec].queryContext.plannerParams.processFailure shouldEqual false } it("should not do routing for InstantQueries when there are local and remote failures") { @@ -393,10 +405,12 @@ class HighAvailabilityPlannerSpec extends AnyFunSpec with Matchers { execPlan.isInstanceOf[PromQlRemoteExec] shouldEqual true val child = execPlan.asInstanceOf[PromQlRemoteExec] - child.params.startSecs shouldEqual from - child.params.endSecs shouldEqual to - child.params.stepSecs shouldEqual step - child.params.processFailure shouldEqual false + val queryParams = child.asInstanceOf[PromQlRemoteExec].queryContext.origQueryParams. + asInstanceOf[PromQlQueryParams] + queryParams.startSecs shouldEqual from + queryParams.endSecs shouldEqual to + queryParams.stepSecs shouldEqual step + child.asInstanceOf[PromQlRemoteExec].queryContext.plannerParams.processFailure shouldEqual false } it("should work with offset") { @@ -420,8 +434,10 @@ class HighAvailabilityPlannerSpec extends AnyFunSpec with Matchers { // So PromQlExec is generated instead of local LocalPartitionDistConcatExec. PromQlExec will have original query and start time // Start time with offset will be calculated by buddy pod execPlan2.isInstanceOf[PromQlRemoteExec] shouldEqual (true) - execPlan2.asInstanceOf[PromQlRemoteExec].params.startSecs shouldEqual(700) - execPlan2.asInstanceOf[PromQlRemoteExec].params.endSecs shouldEqual(10000) + val queryParams = execPlan2.asInstanceOf[PromQlRemoteExec].queryContext.origQueryParams. + asInstanceOf[PromQlQueryParams] + queryParams.startSecs shouldEqual(700) + queryParams.endSecs shouldEqual(10000) } it("should generate PromQlExec for metadata queries") { @@ -443,8 +459,9 @@ class HighAvailabilityPlannerSpec extends AnyFunSpec with Matchers { val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) execPlan.isInstanceOf[MetadataRemoteExec] shouldEqual (true) - execPlan.asInstanceOf[MetadataRemoteExec].params.startSecs shouldEqual (from) - execPlan.asInstanceOf[MetadataRemoteExec].params.endSecs shouldEqual (to) + val queryParams = execPlan.asInstanceOf[MetadataRemoteExec].queryContext.origQueryParams.asInstanceOf[PromQlQueryParams] + queryParams.startSecs shouldEqual (from) + queryParams.endSecs shouldEqual (to) } } diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala index 66ddffc688..9cf3767ef7 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala @@ -5,11 +5,11 @@ import akka.testkit.TestProbe import com.typesafe.config.ConfigFactory import org.scalatest.funspec.AnyFunSpec import org.scalatest.matchers.should.Matchers - import filodb.coordinator.ShardMapper import filodb.core.MetricsTestData import filodb.core.metadata.Schemas -import filodb.core.query.{PromQlQueryParams, QueryConfig, QueryContext} +import filodb.core.query.Filter.Equals +import filodb.core.query.{ColumnFilter, PlannerParams, PromQlQueryParams, QueryConfig, QueryContext} import filodb.prometheus.ast.TimeStepParams import filodb.prometheus.parse.Parser import filodb.query.LogicalPlan @@ -58,9 +58,10 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers { val engine = new MultiPartitionPlanner(partitionLocationProvider, localPlanner, "local", dataset, queryConfig) val lp = Parser.queryRangeToLogicalPlan("test{job = \"app\"}", TimeStepParams(1000, 100, 2000)) - val promQlQueryParams = PromQlQueryParams("test{job = \"app\"}", 1000, 100, 2000, processMultiPartition = true) + val promQlQueryParams = PromQlQueryParams("test{job = \"app\"}", 1000, 100, 2000) - val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams, plannerParams = + PlannerParams(processMultiPartition = true))) execPlan.isInstanceOf[LocalPartitionDistConcatExec] shouldEqual (true) execPlan.children.length shouldEqual 2 @@ -87,23 +88,23 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers { val engine = new MultiPartitionPlanner(partitionLocationProvider, localPlanner, "local", dataset, queryConfig) val lp = Parser.queryRangeToLogicalPlan("test{job = \"app\"}", TimeStepParams(startSeconds, step, endSeconds)) - val promQlQueryParams = PromQlQueryParams("test{job = \"app\"}", startSeconds, step, endSeconds, - processMultiPartition = true) + val promQlQueryParams = PromQlQueryParams("test{job = \"app\"}", startSeconds, step, endSeconds) - val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams, plannerParams = + PlannerParams(processMultiPartition = true))) val stitchRvsExec = execPlan.asInstanceOf[StitchRvsExec] stitchRvsExec.children.size shouldEqual (2) stitchRvsExec.children(0).isInstanceOf[PromQlRemoteExec] shouldEqual true stitchRvsExec.children(1).isInstanceOf[PromQlRemoteExec] shouldEqual true - val remoteExec1 = stitchRvsExec.children(0).asInstanceOf[PromQlRemoteExec] - remoteExec1.params.startSecs shouldEqual startSeconds - remoteExec1.params.endSecs shouldEqual (localPartitionStart - 1) - remoteExec1.params.stepSecs shouldEqual step - remoteExec1.params.processFailure shouldEqual true - remoteExec1.params.processMultiPartition shouldEqual false + val queryParams1 = remoteExec1.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams] + queryParams1.startSecs shouldEqual startSeconds + queryParams1.endSecs shouldEqual (localPartitionStart - 1) + queryParams1.stepSecs shouldEqual step + remoteExec1.queryContext.plannerParams.processFailure shouldEqual true + remoteExec1.queryContext.plannerParams.processMultiPartition shouldEqual false remoteExec1.queryEndpoint shouldEqual "remote-url" val expectedStartMs = ((startSeconds*1000) to (endSeconds*1000) by (step*1000)).find { instant => @@ -111,11 +112,12 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers { }.get val remoteExec2 = stitchRvsExec.children(1).asInstanceOf[PromQlRemoteExec] - remoteExec2.params.startSecs shouldEqual (expectedStartMs / 1000) - remoteExec2.params.endSecs shouldEqual endSeconds - remoteExec2.params.stepSecs shouldEqual step - remoteExec2.params.processFailure shouldEqual true - remoteExec2.params.processMultiPartition shouldEqual false + val queryParams2 = remoteExec2.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams] + queryParams2.startSecs shouldEqual (expectedStartMs / 1000) + queryParams2.endSecs shouldEqual endSeconds + queryParams2.stepSecs shouldEqual step + remoteExec2.queryContext.plannerParams.processFailure shouldEqual true + remoteExec2.queryContext.plannerParams.processMultiPartition shouldEqual false remoteExec2.queryEndpoint shouldEqual "remote-url2" } @@ -132,9 +134,10 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers { val engine = new MultiPartitionPlanner(partitionLocationProvider, localPlanner, "local", dataset, queryConfig) val lp = Parser.queryRangeToLogicalPlan("time()", TimeStepParams(1000, 100, 2000)) - val promQlQueryParams = PromQlQueryParams("time()", 1000, 100, 2000, processMultiPartition = true) + val promQlQueryParams = PromQlQueryParams("time()", 1000, 100, 2000) - val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams, plannerParams = + PlannerParams(processMultiPartition = true))) execPlan.isInstanceOf[TimeScalarGeneratorExec] shouldEqual (true) } @@ -152,10 +155,10 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers { val lp = Parser.queryRangeToLogicalPlan("test1{job = \"app\"} + test2{job = \"app\"}", TimeStepParams(1000, 100, 2000)) - val promQlQueryParams = PromQlQueryParams("test1{job = \"app\"} + test2{job = \"app\"}", 1000, 100, 2000, - processMultiPartition = true) + val promQlQueryParams = PromQlQueryParams("test1{job = \"app\"} + test2{job = \"app\"}", 1000, 100, 2000) - val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams, plannerParams = + PlannerParams(processMultiPartition = true))) execPlan.isInstanceOf[BinaryJoinExec] shouldEqual (true) } @@ -196,14 +199,16 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers { val lp = Parser.queryRangeToLogicalPlan("test1{job = \"app\"} + test2{job = \"app\"}", TimeStepParams(1000, 100, 10000)) - val promQlQueryParams = PromQlQueryParams("test1{job = \"app\"} + test2{job = \"app\"}", 1000, 100, 10000, - processMultiPartition = true) + val promQlQueryParams = PromQlQueryParams("test1{job = \"app\"} + test2{job = \"app\"}", 1000, 100, 10000) - val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams, plannerParams = + PlannerParams(processMultiPartition = true))) execPlan.isInstanceOf[PromQlRemoteExec] shouldEqual (true) - execPlan.asInstanceOf[PromQlRemoteExec].params.startSecs shouldEqual 1000 - execPlan.asInstanceOf[PromQlRemoteExec].params.endSecs shouldEqual 10000 + val queryParams = execPlan.asInstanceOf[PromQlRemoteExec].queryContext.origQueryParams. + asInstanceOf[PromQlQueryParams] + queryParams.startSecs shouldEqual 1000 + queryParams.endSecs shouldEqual 10000 } it ("should generate Exec plan for Metadata query") { @@ -225,17 +230,20 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers { TimeStepParams(startSeconds, step, endSeconds)) val promQlQueryParams = PromQlQueryParams( - "http_requests_total{job=\"prometheus\", method=\"GET\"}", startSeconds, step, endSeconds, - processMultiPartition = true) + "http_requests_total{job=\"prometheus\", method=\"GET\"}", startSeconds, step, endSeconds) - val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams, plannerParams = + PlannerParams(processMultiPartition = true))) execPlan.isInstanceOf[PartKeysDistConcatExec] shouldEqual (true) execPlan.children(0).isInstanceOf[PartKeysDistConcatExec] shouldEqual(true) execPlan.children(1).isInstanceOf[MetadataRemoteExec] shouldEqual(true) - execPlan.children(1).asInstanceOf[MetadataRemoteExec].params.startSecs shouldEqual(startSeconds) - execPlan.children(1).asInstanceOf[MetadataRemoteExec].params.endSecs shouldEqual(localPartitionStart - 1) + val queryParams = execPlan.children(1).asInstanceOf[MetadataRemoteExec].queryContext.origQueryParams. + asInstanceOf[PromQlQueryParams] + + queryParams.startSecs shouldEqual(startSeconds) + queryParams.endSecs shouldEqual(localPartitionStart - 1) execPlan.children(0).asInstanceOf[PartKeysDistConcatExec].children(0).asInstanceOf[PartKeysExec].start shouldEqual (localPartitionStart * 1000) execPlan.children(0).asInstanceOf[PartKeysDistConcatExec].children(0).asInstanceOf[PartKeysExec].end shouldEqual @@ -268,20 +276,21 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers { } val engine = new MultiPartitionPlanner(partitionLocationProvider, localPlanner, "local", dataset, queryConfig) val lp = Parser.queryRangeToLogicalPlan("test{job = \"app\"}", TimeStepParams(startSeconds, step, endSeconds)) - val promQlQueryParams = PromQlQueryParams("test{job = \"app\"}", startSeconds, step, endSeconds, - processMultiPartition = true) - val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + val promQlQueryParams = PromQlQueryParams("test{job = \"app\"}", startSeconds, step, endSeconds) + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams, plannerParams = + PlannerParams(processMultiPartition = true))) val stitchRvsExec = execPlan.asInstanceOf[StitchRvsExec] stitchRvsExec.children.size shouldEqual (3) stitchRvsExec.children(0).isInstanceOf[PromQlRemoteExec] shouldEqual (true) stitchRvsExec.children(1).isInstanceOf[PromQlRemoteExec] shouldEqual (true) stitchRvsExec.children(2).isInstanceOf[PromQlRemoteExec] shouldEqual (true) val remoteExec1 = stitchRvsExec.children(0).asInstanceOf[PromQlRemoteExec] - remoteExec1.params.startSecs shouldEqual startSeconds - remoteExec1.params.endSecs shouldEqual 3999 - remoteExec1.params.stepSecs shouldEqual step - remoteExec1.params.processFailure shouldEqual true - remoteExec1.params.processMultiPartition shouldEqual false + val queryParams1 = remoteExec1.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams] + queryParams1.startSecs shouldEqual startSeconds + queryParams1.endSecs shouldEqual 3999 + queryParams1.stepSecs shouldEqual step + remoteExec1.queryContext.plannerParams.processFailure shouldEqual true + remoteExec1.queryContext.plannerParams.processMultiPartition shouldEqual false remoteExec1.queryEndpoint shouldEqual "remote-url1" val remoteExec2 = stitchRvsExec.children(1).asInstanceOf[PromQlRemoteExec] @@ -293,19 +302,21 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers { instant - lookbackMs > (thirdPartitionStart * 1000) }.get - remoteExec2.params.startSecs shouldEqual expectedStartMs1 / 1000 - remoteExec2.params.endSecs shouldEqual 6999 - remoteExec2.params.stepSecs shouldEqual step - remoteExec2.params.processFailure shouldEqual true - remoteExec2.params.processMultiPartition shouldEqual false + val queryParams2 = remoteExec2.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams] + queryParams2.startSecs shouldEqual expectedStartMs1 / 1000 + queryParams2.endSecs shouldEqual 6999 + queryParams2.stepSecs shouldEqual step + remoteExec2.queryContext.plannerParams.processFailure shouldEqual true + remoteExec2.queryContext.plannerParams.processMultiPartition shouldEqual false remoteExec2.queryEndpoint shouldEqual "remote-url2" val remoteExec3 = stitchRvsExec.children(2).asInstanceOf[PromQlRemoteExec] - remoteExec3.params.startSecs shouldEqual expectedStartMs2 / 1000 - remoteExec3.params.endSecs shouldEqual endSeconds - remoteExec3.params.stepSecs shouldEqual step - remoteExec3.params.processFailure shouldEqual true - remoteExec3.params.processMultiPartition shouldEqual false + val queryParams3 = remoteExec3.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams] + queryParams3.startSecs shouldEqual expectedStartMs2 / 1000 + queryParams3.endSecs shouldEqual endSeconds + queryParams3.stepSecs shouldEqual step + remoteExec3.queryContext.plannerParams.processFailure shouldEqual true + remoteExec3.queryContext.plannerParams.processMultiPartition shouldEqual false remoteExec3.queryEndpoint shouldEqual "remote-url3" } @@ -334,10 +345,10 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers { val engine = new MultiPartitionPlanner(partitionLocationProvider, localPlanner, "local", dataset, queryConfig) val lp = Parser.queryRangeToLogicalPlan("test{job = \"app\"}[100s]", TimeStepParams(startSeconds, step, endSeconds)) - val promQlQueryParams = PromQlQueryParams("test{job = \"app\"}", startSeconds, step, endSeconds, - processMultiPartition = true) + val promQlQueryParams = PromQlQueryParams("test{job = \"app\"}", startSeconds, step, endSeconds) - val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams, + plannerParams = PlannerParams(processMultiPartition = true))) val stitchRvsExec = execPlan.asInstanceOf[StitchRvsExec] stitchRvsExec.children.size shouldEqual (2) @@ -347,19 +358,21 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers { // Instant/Raw queries will have same start and end point in all partitions as we want to fetch raw data val remoteExec1 = stitchRvsExec.children(0).asInstanceOf[PromQlRemoteExec] - remoteExec1.params.startSecs shouldEqual startSeconds - remoteExec1.params.endSecs shouldEqual endSeconds - remoteExec1.params.stepSecs shouldEqual step - remoteExec1.params.processFailure shouldEqual true - remoteExec1.params.processMultiPartition shouldEqual false + val queryParams1 = remoteExec1.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams] + queryParams1.startSecs shouldEqual startSeconds + queryParams1.endSecs shouldEqual endSeconds + queryParams1.stepSecs shouldEqual step + remoteExec1.queryContext.plannerParams.processFailure shouldEqual true + remoteExec1.queryContext.plannerParams.processMultiPartition shouldEqual false remoteExec1.queryEndpoint shouldEqual "remote-url1" val remoteExec2 = stitchRvsExec.children(1).asInstanceOf[PromQlRemoteExec] - remoteExec2.params.startSecs shouldEqual startSeconds - remoteExec2.params.endSecs shouldEqual endSeconds - remoteExec2.params.stepSecs shouldEqual step - remoteExec2.params.processFailure shouldEqual true - remoteExec2.params.processMultiPartition shouldEqual false + val queryParams2 = remoteExec1.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams] + queryParams2.startSecs shouldEqual startSeconds + queryParams2.endSecs shouldEqual endSeconds + queryParams2.stepSecs shouldEqual step + remoteExec2.queryContext.plannerParams.processFailure shouldEqual true + remoteExec2.queryContext.plannerParams.processMultiPartition shouldEqual false remoteExec2.queryEndpoint shouldEqual "remote-url2" } @@ -390,10 +403,10 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers { val engine = new MultiPartitionPlanner(partitionLocationProvider, localPlanner, "local", dataset, queryConfig) val lp = Parser.queryRangeToLogicalPlan("test{job = \"app\"}", TimeStepParams(startSeconds, step, endSeconds)) - val promQlQueryParams = PromQlQueryParams("test{job = \"app\"}", startSeconds, step, endSeconds, - processMultiPartition = true) + val promQlQueryParams = PromQlQueryParams("test{job = \"app\"}", startSeconds, step, endSeconds) - val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams, plannerParams = + PlannerParams(processMultiPartition = true))) val stitchRvsExec = execPlan.asInstanceOf[StitchRvsExec] stitchRvsExec.children.size shouldEqual (2) stitchRvsExec.children(0).isInstanceOf[PromQlRemoteExec] shouldEqual (true) @@ -401,19 +414,21 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers { val remoteExec = stitchRvsExec.children(0).asInstanceOf[PromQlRemoteExec] - remoteExec.params.startSecs shouldEqual startSeconds - remoteExec.params.endSecs shouldEqual (localPartitionStartMs - 1) / 1000 - remoteExec.params.stepSecs shouldEqual step - remoteExec.params.processFailure shouldEqual true - remoteExec.params.processMultiPartition shouldEqual false + val queryParams = remoteExec.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams] + queryParams.startSecs shouldEqual startSeconds + queryParams.endSecs shouldEqual (localPartitionStartMs - 1) / 1000 + queryParams.stepSecs shouldEqual step + remoteExec.queryContext.plannerParams.processFailure shouldEqual true + remoteExec.queryContext.plannerParams.processMultiPartition shouldEqual false remoteExec.queryEndpoint shouldEqual "remote-url" val remoteExec2 = stitchRvsExec.children(1).asInstanceOf[PromQlRemoteExec] - remoteExec2.params.startSecs shouldEqual endSeconds - remoteExec2.params.endSecs shouldEqual endSeconds - remoteExec2.params.stepSecs shouldEqual step - remoteExec2.params.processFailure shouldEqual true - remoteExec2.params.processMultiPartition shouldEqual false + val queryParams2 = remoteExec2.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams] + queryParams2.startSecs shouldEqual endSeconds + queryParams2.endSecs shouldEqual endSeconds + queryParams2.stepSecs shouldEqual step + remoteExec2.queryContext.plannerParams.processFailure shouldEqual true + remoteExec2.queryContext.plannerParams.processMultiPartition shouldEqual false remoteExec2.queryEndpoint shouldEqual "remote-url" } @@ -433,13 +448,13 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers { } val engine = new MultiPartitionPlanner(partitionLocationProvider, localPlanner, "local", dataset, queryConfig) + val lp = Parser.labelValuesQueryToLogicalPlan(Seq("""__metric__"""), Some("""_ws_="demo""""), TimeStepParams(startSeconds, step, endSeconds) ) - val promQlQueryParams = PromQlQueryParams( - "", startSeconds, step, endSeconds, None, Some("/api/v2/label/values"), - processMultiPartition = true) + val promQlQueryParams = PromQlQueryParams("", startSeconds, step, endSeconds, Some("/api/v2/label/values")) - val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams, plannerParams = + PlannerParams(processMultiPartition = true))) execPlan.isInstanceOf[LabelValuesDistConcatExec] shouldEqual (true) execPlan.children(0).isInstanceOf[LabelValuesDistConcatExec] shouldEqual(true) @@ -448,11 +463,50 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers { val expectedUrlParams = Map("filter" -> """_ws_="demo"""", "labels" -> "__metric__") execPlan.children(1).asInstanceOf[MetadataRemoteExec].urlParams shouldEqual(expectedUrlParams) // Filter values // should have quotes - execPlan.children(1).asInstanceOf[MetadataRemoteExec].params.endSecs shouldEqual(localPartitionStart - 1) + execPlan.children(1).asInstanceOf[MetadataRemoteExec].queryContext.origQueryParams.asInstanceOf[PromQlQueryParams]. + endSecs shouldEqual(localPartitionStart - 1) execPlan.children(0).asInstanceOf[LabelValuesDistConcatExec].children(0).asInstanceOf[LabelValuesExec].startMs shouldEqual (localPartitionStart * 1000) execPlan.children(0).asInstanceOf[LabelValuesDistConcatExec].children(0).asInstanceOf[LabelValuesExec].endMs shouldEqual (endSeconds * 1000) } + + it ("should generate multipartition BinaryJoin") { + def partitions(timeRange: TimeRange): List[PartitionAssignment] = List(PartitionAssignment("remote", "remote-url", + TimeRange(timeRange.startMs, timeRange.endMs))) + + val partitionLocationProvider = new PartitionLocationProvider { + override def getPartitions(routingKey: Map[String, String], timeRange: TimeRange): List[PartitionAssignment] = { + if (routingKey.equals(Map("job" -> "app1"))) List( + PartitionAssignment("remote", "remote-url", TimeRange(timeRange.startMs, + timeRange.endMs))) + else List( + PartitionAssignment("local", "local-url", TimeRange(timeRange.startMs, + timeRange.endMs))) + } + + override def getAuthorizedPartitions(timeRange: TimeRange): List[PartitionAssignment] = + partitions(timeRange) + } + + val engine = new MultiPartitionPlanner(partitionLocationProvider, localPlanner, "local", dataset, queryConfig) + val lp = Parser.queryRangeToLogicalPlan("""test1{job = "app1"} + test2{job = "app2"}""", + TimeStepParams(1000, 100, 10000)) + + val promQlQueryParams = PromQlQueryParams("""test1{job = "app1"} + test2{job = "app2"}""", 1000, 100, 10000) + + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams, plannerParams = + PlannerParams(processMultiPartition = true))) + execPlan.isInstanceOf[BinaryJoinExec] shouldEqual (true) + execPlan.asInstanceOf[BinaryJoinExec].lhs.head.isInstanceOf[PromQlRemoteExec] shouldEqual(true) + execPlan.asInstanceOf[BinaryJoinExec].rhs.head.isInstanceOf[LocalPartitionDistConcatExec] shouldEqual(true) + + execPlan.asInstanceOf[BinaryJoinExec].lhs.head.asInstanceOf[PromQlRemoteExec].queryContext.origQueryParams.asInstanceOf + [PromQlQueryParams].promQl shouldEqual("""test1{job="app1"}""") + + execPlan.asInstanceOf[BinaryJoinExec].rhs.head.asInstanceOf[LocalPartitionDistConcatExec].children.head. + asInstanceOf[MultiSchemaPartitionsExec].filters.contains(ColumnFilter("job", Equals("app2"))) shouldEqual(true) + + } } diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlannerSpec.scala index a03acc976d..d7ababf217 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlannerSpec.scala @@ -9,7 +9,7 @@ import org.scalatest.matchers.should.Matchers import filodb.coordinator.ShardMapper import filodb.core.MetricsTestData import filodb.core.metadata.Schemas -import filodb.core.query.{ColumnFilter, PromQlQueryParams, QueryConfig, QueryContext} +import filodb.core.query.{ColumnFilter, PlannerParams, PromQlQueryParams, QueryConfig, QueryContext} import filodb.core.query.Filter.Equals import filodb.prometheus.ast.TimeStepParams import filodb.prometheus.parse.Parser @@ -30,7 +30,7 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture withFallback(routingConfig) private val queryConfig = new QueryConfig(config) - private val promQlQueryParams = PromQlQueryParams("sum(heap_usage)", 100, 1, 1000, None) + private val promQlQueryParams = PromQlQueryParams("sum(heap_usage)", 100, 1, 1000) private val localMapper = new ShardMapper(32) for {i <- 0 until 32} localMapper.registerNode(Seq(i), node) @@ -43,7 +43,7 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-2"))))} - val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn) + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) execPlan.isInstanceOf[MultiPartitionDistConcatExec] shouldEqual(true) execPlan.children(0).children.head.isInstanceOf[MultiSchemaPartitionsExec] @@ -59,9 +59,9 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-2"))))} - val engine = new ShardKeyRegexPlanner( dataset, localPlanner, shardKeyMatcherFn) + val engine = new ShardKeyRegexPlanner( dataset, localPlanner, shardKeyMatcherFn, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = PromQlQueryParams("sum(heap_usage)", 100, 1, - 1000, None))) + 1000))) execPlan.isInstanceOf[MultiPartitionReduceAggregateExec] shouldEqual(true) execPlan.children(0).children.head.isInstanceOf[MultiSchemaPartitionsExec] execPlan.children(1).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. @@ -73,7 +73,7 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture it("should generate Exec plan for time()") { val lp = Parser.queryToLogicalPlan("time()", 1000, 1000) val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq((Seq.empty)) } - val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn) + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) execPlan.isInstanceOf[TimeScalarGeneratorExec] shouldEqual(true) } @@ -84,9 +84,9 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-2"))))} - val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn) + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = PromQlQueryParams("1 + test{_ws_ = \"demo\"," + - " _ns_ =~ \"App.*\", instance = \"Inst-1\" }", 100, 1, 1000, None))) + " _ns_ =~ \"App.*\", instance = \"Inst-1\" }", 100, 1, 1000))) execPlan.isInstanceOf[MultiPartitionDistConcatExec] shouldEqual(true) execPlan.rangeVectorTransformers(0).isInstanceOf[ScalarOperationMapper] shouldEqual true execPlan.children(0).children.head.isInstanceOf[MultiSchemaPartitionsExec] @@ -109,7 +109,7 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-2"))))} - val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn) + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) execPlan.isInstanceOf[BinaryJoinExec] shouldEqual(true) } @@ -120,7 +120,7 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-2"))))} - val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn) + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) execPlan.isInstanceOf[PartKeysDistConcatExec] shouldEqual (true) } @@ -131,7 +131,7 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-2"))))} - val engine = new ShardKeyRegexPlanner( dataset, localPlanner, shardKeyMatcherFn) + val engine = new ShardKeyRegexPlanner( dataset, localPlanner, shardKeyMatcherFn, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) execPlan.isInstanceOf[MultiPartitionReduceAggregateExec] shouldEqual(true) execPlan.asInstanceOf[MultiPartitionReduceAggregateExec].rangeVectorTransformers(0). @@ -165,7 +165,7 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-2"))))} - val engine = new ShardKeyRegexPlanner( dataset, localPlanner, shardKeyMatcherFn) + val engine = new ShardKeyRegexPlanner( dataset, localPlanner, shardKeyMatcherFn, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) execPlan.isInstanceOf[MultiPartitionReduceAggregateExec] shouldEqual(true) execPlan.asInstanceOf[MultiPartitionReduceAggregateExec].rangeVectorTransformers(0). @@ -198,7 +198,7 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-2"))))} - val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn) + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) execPlan.isInstanceOf[LocalPartitionDistConcatExec] shouldEqual(true) execPlan.children(0).isInstanceOf[MultiSchemaPartitionsExec] @@ -209,11 +209,11 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture it("should generate Exec plan for scalar - time()") { val lp = Parser.queryToLogicalPlan("""scalar(test{_ws_ = "demo", _ns_ =~ "App.*"}) - time()""", 1000, 1000) - val promQlQueryParams = PromQlQueryParams("""scalar(test{_ws_ = "demo", _ns_ =~ "App.*"}) - time()""", 100, 1, 1000, None) + val promQlQueryParams = PromQlQueryParams("""scalar(test{_ws_ = "demo", _ns_ =~ "App.*"}) - time()""", 100, 1, 1000) val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-2"))))} - val engine = new ShardKeyRegexPlanner( dataset, localPlanner, shardKeyMatcherFn) + val engine = new ShardKeyRegexPlanner( dataset, localPlanner, shardKeyMatcherFn, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) execPlan.isInstanceOf[TimeScalarGeneratorExec] shouldEqual(true) execPlan.rangeVectorTransformers.head.isInstanceOf[ScalarOperationMapper] shouldEqual true @@ -237,15 +237,91 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture it ("should generate Exec plan for Metadata Label values query") { val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => Seq.empty - val engine = new ShardKeyRegexPlanner( dataset, localPlanner, shardKeyMatcherFn) + val engine = new ShardKeyRegexPlanner( dataset, localPlanner, shardKeyMatcherFn, queryConfig) val lp = Parser.labelValuesQueryToLogicalPlan(Seq("""__metric__"""), Some("""_ws_="demo", _ns_=~".*" """), TimeStepParams(1000, 20, 5000) ) val promQlQueryParams = PromQlQueryParams( - "", 1000, 20, 5000, None, Some("/api/v2/label/values"), - processMultiPartition = true) + "", 1000, 20, 5000, Some("/api/v2/label/values")) + + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams, plannerParams = + PlannerParams(processMultiPartition = true))) - val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) execPlan.isInstanceOf[LabelValuesDistConcatExec] shouldEqual (true) } + + it("should generate Exec plan for Binary join with regex") { + val lp = Parser.queryToLogicalPlan("test1{_ws_ = \"demo\", _ns_ =~ \"App.*\"} + " + + "test2{_ws_ = \"demo\", _ns_ =~ \"App.*\"}", 1000, 1000) + val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), + ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), + ColumnFilter("_ns_", Equals("App-2"))))} + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + execPlan.isInstanceOf[BinaryJoinExec] shouldEqual(true) + execPlan.children(0).isInstanceOf[MultiPartitionDistConcatExec] shouldEqual(true) + val lhs = execPlan.children(0).asInstanceOf[MultiPartitionDistConcatExec] + lhs.children.length shouldEqual 2 + lhs.children(0).children.head.isInstanceOf[MultiSchemaPartitionsExec] shouldEqual true + lhs.children(1).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", Equals("App-1"))) shouldEqual(true) + lhs.children(0).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", Equals("App-2"))) shouldEqual(true) + val rhs = execPlan.children(1).asInstanceOf[MultiPartitionDistConcatExec] + rhs.children(0).children.head.isInstanceOf[MultiSchemaPartitionsExec] shouldEqual true + rhs.children(1).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", Equals("App-1"))) shouldEqual(true) + rhs.children(0).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", Equals("App-2"))) shouldEqual(true) + } + + it("should generate Exec plan for Binary join with regex only on one side") { + val lp = Parser.queryToLogicalPlan("test1{_ws_ = \"demo\", _ns_ = \"App-0\"} + " + + "test2{_ws_ = \"demo\", _ns_ =~ \"App.*\"}", 1000, 1000) + val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { + Seq(Seq(ColumnFilter("_ws_", Equals("demo")), + ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), + ColumnFilter("_ns_", Equals("App-2")))) + } + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + execPlan.isInstanceOf[BinaryJoinExec] shouldEqual (true) + execPlan.children(0).isInstanceOf[LocalPartitionDistConcatExec] shouldEqual (true) + val lhs = execPlan.children(0).asInstanceOf[LocalPartitionDistConcatExec] + lhs.children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", Equals("App-0"))) shouldEqual (true) + val rhs = execPlan.children(1).asInstanceOf[MultiPartitionDistConcatExec] + rhs.children(0).children.head.isInstanceOf[MultiSchemaPartitionsExec] shouldEqual true + rhs.children(1).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", Equals("App-1"))) shouldEqual (true) + rhs.children(0).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", Equals("App-2"))) shouldEqual (true) + } + + it("should generate Exec plan for topk query with single matching value for regex") { + val lp = Parser.queryToLogicalPlan(s"""topk(2, test{_ws_ = "demo", _ns_ =~ "App-1"})""", + 1000, 1000) + val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { + Seq(Seq(ColumnFilter("_ws_", Equals("demo")), + ColumnFilter("_ns_", Equals("App-1")))) + } + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + execPlan.isInstanceOf[LocalPartitionReduceAggregateExec] shouldEqual (true) + } + + it("should throw UnsupportedOperationException for topk query with multiple matching values for regex") { + val lp = Parser.queryToLogicalPlan(s"""topk(2, test{_ws_ = "demo", _ns_ =~ "App.*"})""", + 1000, 1000) + val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { + Seq(Seq(ColumnFilter("_ws_", Equals("demo")), + ColumnFilter("_ns_", Equals("App-1"))), + Seq(ColumnFilter("_ws_", Equals("demo")), + ColumnFilter("_ns_", Equals("App-2")))) + } + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) + the[UnsupportedOperationException] thrownBy { + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + } should have message "Shard Key regex not supported for TopK" + } } diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/SingleClusterPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/SingleClusterPlannerSpec.scala index e9b2ae0199..94b8478da0 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/SingleClusterPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/SingleClusterPlannerSpec.scala @@ -1,17 +1,15 @@ package filodb.coordinator.queryplanner import scala.concurrent.duration._ - import akka.actor.ActorSystem import akka.testkit.TestProbe import com.typesafe.config.ConfigFactory import org.scalatest.concurrent.ScalaFutures - import filodb.coordinator.ShardMapper import filodb.coordinator.client.QueryCommands.{FunctionalSpreadProvider, StaticSpreadProvider} import filodb.core.{GlobalScheduler, MetricsTestData, SpreadChange} import filodb.core.metadata.Schemas -import filodb.core.query.{ColumnFilter, Filter, PromQlQueryParams, QueryConfig, QueryContext} +import filodb.core.query.{ColumnFilter, Filter, PlannerParams, PromQlQueryParams, QueryConfig, QueryContext} import filodb.core.store.TimeRangeChunkScan import filodb.prometheus.ast.{TimeStepParams, WindowConstants} import filodb.prometheus.parse.Parser @@ -114,7 +112,7 @@ class SingleClusterPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture // materialized exec plan val execPlan = engine.materialize(logicalPlan, - QueryContext(promQlQueryParams, Some(StaticSpreadProvider(SpreadChange(0, 4))), 1000000)) + QueryContext(promQlQueryParams, plannerParams = PlannerParams(spreadOverride = Some(StaticSpreadProvider(SpreadChange(0, 4))), queryTimeoutMillis =1000000))) execPlan.isInstanceOf[BinaryJoinExec] shouldEqual true // Now there should be multiple levels of reduce because we have 16 shards @@ -142,9 +140,8 @@ class SingleClusterPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture case _ => throw new IllegalArgumentException(s"Unexpected LP $lp") } - val execPlan = engine.materialize(lp, - QueryContext(promQlQueryParams, Some(StaticSpreadProvider(SpreadChange(0, 4))), 1000000)) - + val execPlan = engine.materialize(lp, QueryContext(promQlQueryParams, plannerParams = PlannerParams(spreadOverride = + Some(StaticSpreadProvider(SpreadChange(0, 4))), queryTimeoutMillis =1000000))) info(s"First child plan: ${execPlan.children.head.printTree()}") execPlan.isInstanceOf[LocalPartitionDistConcatExec] shouldEqual true execPlan.children.foreach { l1 => @@ -185,7 +182,8 @@ class SingleClusterPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture val logicalPlan = BinaryJoin(summed1, BinaryOperator.DIV, Cardinality.OneToOne, summed2) // materialized exec plan - val execPlan = engine.materialize(logicalPlan, QueryContext(promQlQueryParams, Some(FunctionalSpreadProvider(spreadFunc)), 1000000)) + val execPlan = engine.materialize(logicalPlan, QueryContext(promQlQueryParams, plannerParams = PlannerParams + (spreadOverride = Some(FunctionalSpreadProvider(spreadFunc)), queryTimeoutMillis =1000000))) execPlan.printTree() execPlan.isInstanceOf[BinaryJoinExec] shouldEqual true @@ -201,8 +199,8 @@ class SingleClusterPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture def spread(filter: Seq[ColumnFilter]): Seq[SpreadChange] = { Seq(SpreadChange(0, 1), SpreadChange(25000000, 2)) // spread change time is in ms } - val execPlan = engine.materialize(lp, QueryContext(promQlQueryParams, - Some(FunctionalSpreadProvider(spread)), 1000000)) + val execPlan = engine.materialize(lp,QueryContext(promQlQueryParams, plannerParams = PlannerParams + (spreadOverride = Some(FunctionalSpreadProvider(spread)), queryTimeoutMillis = 1000000))) execPlan.rangeVectorTransformers.head.isInstanceOf[StitchRvsMapper] shouldEqual true } @@ -211,8 +209,8 @@ class SingleClusterPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture def spread(filter: Seq[ColumnFilter]): Seq[SpreadChange] = { Seq(SpreadChange(0, 1), SpreadChange(35000000, 2)) } - val execPlan = engine.materialize(lp, QueryContext(promQlQueryParams, - Some(FunctionalSpreadProvider(spread)), 1000000)) + val execPlan = engine.materialize(lp, QueryContext(promQlQueryParams, plannerParams = PlannerParams + (spreadOverride = Some(FunctionalSpreadProvider(spread)), queryTimeoutMillis = 1000000))) execPlan.rangeVectorTransformers.isEmpty shouldEqual true } @@ -222,8 +220,8 @@ class SingleClusterPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture def spread(filter: Seq[ColumnFilter]): Seq[SpreadChange] = { Seq(SpreadChange(0, 1), SpreadChange(25000000, 2)) } - val execPlan = engine.materialize(lp, QueryContext(promQlQueryParams, Some(FunctionalSpreadProvider(spread)), - 1000000)) + val execPlan = engine.materialize(lp, QueryContext(promQlQueryParams, plannerParams = PlannerParams + (spreadOverride = Some(FunctionalSpreadProvider(spread)), queryTimeoutMillis = 1000000))) val binaryJoinNode = execPlan.children(0) binaryJoinNode.isInstanceOf[BinaryJoinExec] shouldEqual true binaryJoinNode.children.size shouldEqual 2 @@ -236,8 +234,8 @@ class SingleClusterPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture def spread(filter: Seq[ColumnFilter]): Seq[SpreadChange] = { Seq(SpreadChange(0, 1), SpreadChange(35000000, 2)) } - val execPlan = engine.materialize(lp, QueryContext(promQlQueryParams, Some(FunctionalSpreadProvider(spread)), - 1000000)) + val execPlan = engine.materialize(lp, QueryContext(promQlQueryParams, plannerParams = PlannerParams + (spreadOverride = Some(FunctionalSpreadProvider(spread)), queryTimeoutMillis = 1000000))) val binaryJoinNode = execPlan.children(0) binaryJoinNode.isInstanceOf[BinaryJoinExec] shouldEqual true binaryJoinNode.children.foreach(_.isInstanceOf[StitchRvsExec] should not equal true) @@ -324,7 +322,8 @@ class SingleClusterPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture val logicalPlan = Parser.queryRangeToLogicalPlan("""sum(rate(foo{job="bar"}[3d]))""", TimeStepParams(nowSeconds, 1.minute.toSeconds, nowSeconds)) - val ep = planner.materialize(logicalPlan, QueryContext()).asInstanceOf[LocalPartitionReduceAggregateExec] + val ep = planner.materialize(logicalPlan, QueryContext(origQueryParams = PromQlQueryParams + ("""sum(rate(foo{job="bar"}[3d]))""",1000, 100, 1000))).asInstanceOf[LocalPartitionReduceAggregateExec] val psm = ep.children.head.asInstanceOf[MultiSchemaPartitionsExec] .rangeVectorTransformers.head.asInstanceOf[PeriodicSamplesMapper] psm.start shouldEqual (nowSeconds * 1000) diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/SingleClusterPlannerSplitSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/SingleClusterPlannerSplitSpec.scala index 00c8f10c16..c6ca2bf9d8 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/SingleClusterPlannerSplitSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/SingleClusterPlannerSplitSpec.scala @@ -155,7 +155,8 @@ class SingleClusterPlannerSplitSpec extends AnyFunSpec with Matchers with ScalaF // materialized exec plan val parentExecPlan = engine.materialize(logicalPlan, - QueryContext(promQlQueryParams, Some(StaticSpreadProvider(SpreadChange(0, 4))), 1000000)) + QueryContext(promQlQueryParams, plannerParams = PlannerParams(spreadOverride = + Some(StaticSpreadProvider(SpreadChange(0, 4))), queryTimeoutMillis = 1000000))) parentExecPlan.isInstanceOf[SplitLocalPartitionDistConcatExec] shouldEqual true parentExecPlan.children.foreach { execPlan => @@ -189,7 +190,8 @@ class SingleClusterPlannerSplitSpec extends AnyFunSpec with Matchers with ScalaF } val parentExecPlan = engine.materialize(lp, - QueryContext(promQlQueryParams, Some(StaticSpreadProvider(SpreadChange(0, 4))), 1000000)) + QueryContext(promQlQueryParams, plannerParams = PlannerParams(spreadOverride = + Some(StaticSpreadProvider(SpreadChange(0, 4))), queryTimeoutMillis = 1000000))) info(s"First inner child plan: ${parentExecPlan.children.head.children.head.printTree()}") parentExecPlan.isInstanceOf[SplitLocalPartitionDistConcatExec] shouldEqual true @@ -220,7 +222,9 @@ class SingleClusterPlannerSplitSpec extends AnyFunSpec with Matchers with ScalaF val logicalPlan = BinaryJoin(summed1, BinaryOperator.DIV, Cardinality.OneToOne, summed2) // materialized exec plan - val parentExecPlan = engine.materialize(logicalPlan, QueryContext(promQlQueryParams, Some(FunctionalSpreadProvider(spreadFunc)), 1000000)) + val parentExecPlan = engine.materialize(logicalPlan, QueryContext(promQlQueryParams, plannerParams = + PlannerParams(spreadOverride = Some(FunctionalSpreadProvider(spreadFunc)), queryTimeoutMillis = 1000000))) + parentExecPlan.printTree() parentExecPlan.isInstanceOf[SplitLocalPartitionDistConcatExec] shouldEqual true @@ -253,8 +257,8 @@ class SingleClusterPlannerSplitSpec extends AnyFunSpec with Matchers with ScalaF def spread(filter: Seq[ColumnFilter]): Seq[SpreadChange] = { Seq(SpreadChange(0, 1), SpreadChange(350000, 2)) // spread change time is in ms } - val execPlan = engine.materialize(lp, QueryContext(promQlQueryParams, - Some(FunctionalSpreadProvider(spread)), 1000000)) + val execPlan = engine.materialize(lp, QueryContext(promQlQueryParams, plannerParams = PlannerParams(spreadOverride = + Some(FunctionalSpreadProvider(spread)), queryTimeoutMillis = 1000000))) execPlan.isInstanceOf[SplitLocalPartitionDistConcatExec] shouldEqual true execPlan.rangeVectorTransformers.head.isInstanceOf[StitchRvsMapper] shouldEqual true // Stitch split plans execPlan.children should have length (5) @@ -272,8 +276,8 @@ class SingleClusterPlannerSplitSpec extends AnyFunSpec with Matchers with ScalaF def spread(filter: Seq[ColumnFilter]): Seq[SpreadChange] = { Seq(SpreadChange(0, 1), SpreadChange(450000, 2)) // spread change time is in ms } - val execPlan = engine.materialize(lp, QueryContext(promQlQueryParams, - Some(FunctionalSpreadProvider(spread)), 1000000)) + val execPlan = engine.materialize(lp, QueryContext(promQlQueryParams, plannerParams = PlannerParams(spreadOverride = + Some(FunctionalSpreadProvider(spread)), queryTimeoutMillis = 1000000))) execPlan.isInstanceOf[SplitLocalPartitionDistConcatExec] shouldEqual true execPlan.rangeVectorTransformers.head.isInstanceOf[StitchRvsMapper] shouldEqual true // Stitch split plans execPlan.children should have length (5) @@ -293,8 +297,8 @@ class SingleClusterPlannerSplitSpec extends AnyFunSpec with Matchers with ScalaF def spread(filter: Seq[ColumnFilter]): Seq[SpreadChange] = { Seq(SpreadChange(0, 1), SpreadChange(350000, 2)) } - val execPlan = engine.materialize(lp, QueryContext(promQlQueryParams, Some(FunctionalSpreadProvider(spread)), - 1000000)) + val execPlan = engine.materialize(lp, QueryContext(promQlQueryParams, plannerParams = PlannerParams(spreadOverride = + Some(FunctionalSpreadProvider(spread)), queryTimeoutMillis = 1000000))) execPlan.isInstanceOf[SplitLocalPartitionDistConcatExec] shouldEqual true execPlan.rangeVectorTransformers.head.isInstanceOf[StitchRvsMapper] shouldEqual true // Stitch split plans @@ -329,8 +333,8 @@ class SingleClusterPlannerSplitSpec extends AnyFunSpec with Matchers with ScalaF def spread(filter: Seq[ColumnFilter]): Seq[SpreadChange] = { Seq(SpreadChange(0, 1), SpreadChange(450000, 2)) } - val execPlan = engine.materialize(lp, QueryContext(promQlQueryParams, Some(FunctionalSpreadProvider(spread)), - 1000000)) + val execPlan = engine.materialize(lp, QueryContext(promQlQueryParams, plannerParams = PlannerParams(spreadOverride = + Some(FunctionalSpreadProvider(spread)), queryTimeoutMillis = 1000000))) execPlan.isInstanceOf[SplitLocalPartitionDistConcatExec] shouldEqual true execPlan.rangeVectorTransformers.head.isInstanceOf[StitchRvsMapper] shouldEqual true // Stitch split plans @@ -467,7 +471,8 @@ class SingleClusterPlannerSplitSpec extends AnyFunSpec with Matchers with ScalaF val logicalPlan = Parser.queryRangeToLogicalPlan("""sum(rate(foo{job="bar"}[3d]))""", TimeStepParams(nowSeconds, 1.minute.toSeconds, nowSeconds)) - val ep = planner.materialize(logicalPlan, QueryContext()).asInstanceOf[LocalPartitionReduceAggregateExec] + val ep = planner.materialize(logicalPlan, QueryContext(origQueryParams = PromQlQueryParams + ("""sum(rate(foo{job="bar"}[3d]))""",1000, 100, 1000))).asInstanceOf[LocalPartitionReduceAggregateExec] val psm = ep.children.head.asInstanceOf[MultiSchemaPartitionsExec] .rangeVectorTransformers.head.asInstanceOf[PeriodicSamplesMapper] psm.start shouldEqual (nowSeconds * 1000) diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/SinglePartitionPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/SinglePartitionPlannerSpec.scala index 60a7b98382..d6bf4e3732 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/SinglePartitionPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/SinglePartitionPlannerSpec.scala @@ -97,32 +97,32 @@ class SinglePartitionPlannerSpec extends AnyFunSpec with Matchers { val lp = Parser.queryToLogicalPlan("test1{job = \"app\"} + test2{job = \"app\"}", 1000, 1000) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) - execPlan.printTree() + execPlan.dispatcher.isInstanceOf[ActorPlanDispatcher] shouldEqual true // Since all metrics belong to same cluster execPlan.isInstanceOf[BinaryJoinExec] shouldEqual (true) execPlan.children.foreach { l1 => - l1.isInstanceOf[LocalPartitionDistConcatExec] shouldEqual true - l1.children.foreach { l2 => - l2.isInstanceOf[MultiSchemaPartitionsExec] shouldEqual true - l2.rangeVectorTransformers.size shouldEqual 1 - l2.rangeVectorTransformers(0).isInstanceOf[PeriodicSamplesMapper] shouldEqual true + l1.isInstanceOf[MultiSchemaPartitionsExec] shouldEqual true + l1.rangeVectorTransformers.size shouldEqual 1 + l1.rangeVectorTransformers(0).isInstanceOf[PeriodicSamplesMapper] shouldEqual true } } - } + it("should generate exec plan for nested Binary Join query") { val lp = Parser.queryToLogicalPlan("test1{job = \"app\"} + test2{job = \"app\"} + test3{job = \"app\"}", 1000, 1000) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + + execPlan.dispatcher.isInstanceOf[ActorPlanDispatcher] shouldEqual true // Since all metrics belong to same cluster execPlan.isInstanceOf[BinaryJoinExec] shouldEqual (true) execPlan.asInstanceOf[BinaryJoinExec].lhs.head.isInstanceOf[BinaryJoinExec] shouldEqual true - execPlan.asInstanceOf[BinaryJoinExec].rhs.head.isInstanceOf[LocalPartitionDistConcatExec] shouldEqual true + execPlan.asInstanceOf[BinaryJoinExec].rhs.head.isInstanceOf[MultiSchemaPartitionsExec] shouldEqual true } it("should generate BinaryJoin Exec plan with remote and local cluster metrics") { val lp = Parser.queryToLogicalPlan("test{job = \"app\"} + rr1{job = \"app\"}", 1000, 1000) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) - execPlan.printTree() + execPlan.dispatcher shouldEqual (InProcessPlanDispatcher) //rr1 and test belong to different clusters execPlan.isInstanceOf[BinaryJoinExec] shouldEqual (true) execPlan.asInstanceOf[BinaryJoinExec].rhs.head.asInstanceOf[MockExecPlan].name shouldEqual ("rules1") execPlan.asInstanceOf[BinaryJoinExec].lhs.head.isInstanceOf[LocalPartitionDistConcatExec] shouldEqual true @@ -132,6 +132,7 @@ class SinglePartitionPlannerSpec extends AnyFunSpec with Matchers { val lp = Parser.queryToLogicalPlan("rr1{job = \"app\"} + rr2{job = \"app\"}", 1000, 1000) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + execPlan.dispatcher shouldEqual (InProcessPlanDispatcher) //rr1 and rr2 belong to different clusters execPlan.isInstanceOf[BinaryJoinExec] shouldEqual (true) execPlan.asInstanceOf[BinaryJoinExec].lhs.head.asInstanceOf[MockExecPlan].name shouldEqual ("rules1") execPlan.asInstanceOf[BinaryJoinExec].rhs.head.asInstanceOf[MockExecPlan].name shouldEqual ("rules2") @@ -160,19 +161,13 @@ class SinglePartitionPlannerSpec extends AnyFunSpec with Matchers { execPlan.isInstanceOf[TimeScalarGeneratorExec] shouldEqual true } - it("should generate BinaryJoin Exec with remote exec's having lhs or rhs query") { + it("should generate BinaryJoin Exec with remote exec's having lhs and rhs query") { val lp = Parser.queryRangeToLogicalPlan("""test1{job = "app"} + test2{job = "app"}""", TimeStepParams(300, 20, 500)) - val promQlQueryParams = PromQlQueryParams("test1{job = \"app\"} + test2{job = \"app\"}", 300, 20, 500) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) - println(execPlan.printTree()) - execPlan.isInstanceOf[BinaryJoinExec] shouldEqual (true) - execPlan.children(0).isInstanceOf[PromQlRemoteExec] shouldEqual(true) - execPlan.children(1).isInstanceOf[PromQlRemoteExec] shouldEqual(true) - - // LHS should have only LHS query and RHS should have oly RHS query - execPlan.children(0).asInstanceOf[PromQlRemoteExec].params.promQl shouldEqual("""test1{job="app"}""") - execPlan.children(1).asInstanceOf[PromQlRemoteExec].params.promQl shouldEqual("""test2{job="app"}""") + execPlan.isInstanceOf[PromQlRemoteExec] shouldEqual (true) + execPlan.asInstanceOf[PromQlRemoteExec].queryContext.origQueryParams.asInstanceOf[PromQlQueryParams]. + promQl shouldEqual("""test1{job = "app"} + test2{job = "app"}""") } } diff --git a/core/src/main/resources/filodb-defaults.conf b/core/src/main/resources/filodb-defaults.conf index a5b0ec168e..021a10ca43 100644 --- a/core/src/main/resources/filodb-defaults.conf +++ b/core/src/main/resources/filodb-defaults.conf @@ -44,7 +44,8 @@ filodb { ignoreShardKeyColumnSuffixes = { "_metric_" = ["_bucket", "_count", "_sum"] } ignoreTagsOnPartitionKeyHash = ["le"] metricColumn = "_metric_" - shardKeyColumns = [ "_metric_", "_ws_", "_ns_" ] + # shard key columns should be in hierarchical order + shardKeyColumns = ["_ws_", "_ns_", "_metric_"] } } @@ -109,6 +110,20 @@ filodb { } } + quotas { + # if one is not defined for data source, this number is used for all limits + default = 1000000 + # prometheus { + # defaults = [100, 500, 10000, 100000] + # custom = [ + # { + # shardKeyPrefix = ["myWs", "myNs", "myMetricName"] + # quota = 10000 + # } + # ] + # } + } + tasks { # Frequency with which new shard maps are published shardmap-publish-frequency = 5s @@ -405,10 +420,6 @@ filodb { } } -query-actor-mailbox { - mailbox-type = "filodb.coordinator.QueryActorMailbox" -} - # Configuration for the open-source ingestion gateway gateway { # TCP Port for Influx Line Protocol incoming data diff --git a/core/src/main/scala/filodb.core/binaryrecord2/RecordSchema.scala b/core/src/main/scala/filodb.core/binaryrecord2/RecordSchema.scala index d71b391a1f..e302ceb4d3 100644 --- a/core/src/main/scala/filodb.core/binaryrecord2/RecordSchema.scala +++ b/core/src/main/scala/filodb.core/binaryrecord2/RecordSchema.scala @@ -260,7 +260,7 @@ final class RecordSchema(val columns: Seq[ColumnInfo], def toStringPairs(base: Any, offset: Long): Seq[(String, String)] = { import Column.ColumnType._ val result = new collection.mutable.ArrayBuffer[(String, String)]() - columnTypes.zipWithIndex.map { + columnTypes.zipWithIndex.foreach { case (IntColumn, i) => result += ((colNames(i), getInt(base, offset, i).toString)) case (LongColumn, i) => result += ((colNames(i), getLong(base, offset, i).toString)) case (DoubleColumn, i) => result += ((colNames(i), getDouble(base, offset, i).toString)) @@ -281,6 +281,30 @@ final class RecordSchema(val columns: Seq[ColumnInfo], result } + def colValues(base: Any, offset: Long, cols: Seq[String]): Seq[String] = { + import Column.ColumnType._ + val res = collection.mutable.ArrayBuffer.fill[String](cols.size)(UnsafeUtils.ZeroPointer.asInstanceOf[String]) + columnTypes.zipWithIndex.foreach { + case (IntColumn, i) if cols.contains(colNames(i)) => + res(cols.indexOf(colNames(i))) = getInt(base, offset, i).toString + case (LongColumn, i) if cols.contains(colNames(i)) => + res(cols.indexOf(colNames(i))) = getLong(base, offset, i).toString + case (DoubleColumn, i) if cols.contains(colNames(i)) => + res(cols.indexOf(colNames(i))) = getDouble(base, offset, i).toString + case (StringColumn, i) if cols.contains(colNames(i)) => + res(cols.indexOf(colNames(i))) = asJavaString(base, offset, i).toString + case (TimestampColumn, i) if cols.contains(colNames(i)) => + res(cols.indexOf(colNames(i))) = getLong(base, offset, i).toString + case (MapColumn, i) => val consumer = new SelectColsMapItemConsumer(cols, res) + consumeMapItems(base, offset, i, consumer) + case (BinaryRecordColumn, i) => ??? + case (HistogramColumn, i) => ??? + case _ => // column not selected + } + res + + } + /** * Iterates through each key/value pair of a MapColumn field without any object allocations. * How is this done? By calling the consumer for each pair and directly passing the base and offset. @@ -415,6 +439,18 @@ class DebugStringMapItemConsumer(baseOffset: Long) extends MapItemConsumer { } } + +/** + * A MapItemConsumer which selects col values from map + */ +class SelectColsMapItemConsumer(cols: Seq[String], buf: ArrayBuffer[String]) extends MapItemConsumer { + def consume(keyBase: Any, keyOffset: Long, valueBase: Any, valueOffset: Long, index: Int): Unit = { + val key = UTF8StringShort.toString(keyBase, keyOffset) + if (cols.contains(key)) buf(cols.indexOf(key)) = UTF8StringMedium.toString(valueBase, valueOffset) + } +} + + object RecordSchema { import Column.ColumnType._ diff --git a/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesShard.scala b/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesShard.scala index 237b358679..95cb95dbbe 100644 --- a/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesShard.scala @@ -9,6 +9,7 @@ import scala.concurrent.duration._ import com.typesafe.config.Config import com.typesafe.scalalogging.StrictLogging import kamon.Kamon +import kamon.metric.MeasurementUnit import kamon.tag.TagSet import monix.eval.Task import monix.execution.{CancelableFuture, Scheduler, UncaughtExceptionReporter} @@ -35,6 +36,10 @@ class DownsampledTimeSeriesShardStats(dataset: DatasetRef, shardNum: Int) { val indexPurgeFailed = Kamon.counter("index-purge-failed").withTags(TagSet.from(tags)) val indexEntries = Kamon.gauge("downsample-store-index-entries").withTags(TagSet.from(tags)) val indexRamBytes = Kamon.gauge("downsample-store-index-ram-bytes").withTags(TagSet.from(tags)) + val singlePartCassFetchLatency = Kamon.histogram("single-partition-cassandra-latency", + MeasurementUnit.time.milliseconds).withTags(TagSet.from(tags)) + val purgeIndexEntriesLatency = Kamon.histogram("downsample-store-purge-index-entries-latency", + MeasurementUnit.time.milliseconds).withTags(TagSet.from(tags)) } class DownsampledTimeSeriesShard(rawDatasetRef: DatasetRef, @@ -138,10 +143,7 @@ class DownsampledTimeSeriesShard(rawDatasetRef: DatasetRef, } private def purgeExpiredIndexEntries(): Unit = { - val tracer = Kamon.spanBuilder("downsample-store-purge-index-entries-latency") - .asChildOf(Kamon.currentSpan()) - .tag("dataset", rawDatasetRef.toString) - .tag("shard", shardNum).start() + val start = System.currentTimeMillis() try { val partsToPurge = partKeyIndex.partIdsEndedBefore(System.currentTimeMillis() - downsampleTtls.last.toMillis) partKeyIndex.removePartKeys(partsToPurge) @@ -152,7 +154,7 @@ class DownsampledTimeSeriesShard(rawDatasetRef: DatasetRef, logger.error(s"Error occurred when purging index entries dataset=$rawDatasetRef shard=$shardNum", e) stats.indexPurgeFailed.increment() } finally { - tracer.finish() + stats.purgeIndexEntriesLatency.record(System.currentTimeMillis() - start) } } @@ -251,11 +253,7 @@ class DownsampledTimeSeriesShard(rawDatasetRef: DatasetRef, // when query is complete. Observable.fromIterable(lookup.pkRecords) .mapAsync(downsampleStoreConfig.demandPagingParallelism) { partRec => - val partLoadSpan = Kamon.spanBuilder(s"single-partition-cassandra-latency") - .asChildOf(Kamon.currentSpan()) - .tag("dataset", rawDatasetRef.toString) - .tag("shard", shardNum) - .start() + val startExecute = System.currentTimeMillis() // TODO test multi-partition scan if latencies are high store.readRawPartitions(downsampledDataset, downsampleStoreConfig.maxChunkTime.toMillis, @@ -265,7 +263,7 @@ class DownsampledTimeSeriesShard(rawDatasetRef: DatasetRef, val part = makePagedPartition(pd, lookup.firstSchemaId.get, Some(resolution), colIds) stats.partitionsQueried.increment() stats.chunksQueried.increment(part.numChunks) - partLoadSpan.finish() + stats.singlePartCassFetchLatency.record(Math.max(0, System.currentTimeMillis - startExecute)) part } .defaultIfEmpty(makePagedPartition(RawPartData(partRec.partKey, Seq.empty), diff --git a/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesStore.scala b/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesStore.scala index 0673265623..f526cefb34 100644 --- a/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesStore.scala +++ b/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesStore.scala @@ -13,6 +13,7 @@ import org.jctools.maps.NonBlockingHashMapLong import filodb.core.{DatasetRef, Response, Types} import filodb.core.memstore._ +import filodb.core.memstore.ratelimit.CardinalityRecord import filodb.core.metadata.Schemas import filodb.core.query.{ColumnFilter, QuerySession} import filodb.core.store._ @@ -169,4 +170,10 @@ extends MemStore with StrictLogging { override def readRawPartitions(ref: DatasetRef, maxChunkTime: Long, partMethod: PartitionScanMethod, chunkMethod: ChunkScanMethod): Observable[RawPartData] = ??? + + // TODO we need breakdown for downsample store too, but in a less memory intensive way + override def topKCardinality(ref: DatasetRef, + shards: Seq[Int], + shardKeyPrefix: scala.Seq[String], + k: Int): scala.Seq[CardinalityRecord] = ??? } diff --git a/core/src/main/scala/filodb.core/downsample/OffHeapMemory.scala b/core/src/main/scala/filodb.core/downsample/OffHeapMemory.scala index 2d6992b915..cab49ba3a3 100644 --- a/core/src/main/scala/filodb.core/downsample/OffHeapMemory.scala +++ b/core/src/main/scala/filodb.core/downsample/OffHeapMemory.scala @@ -25,7 +25,7 @@ class OffHeapMemory(schemas: Seq[Schema], }, numPagesPerBlock = 50) - val blockMemFactory = new BlockMemFactory(blockStore, None, maxMetaSize, kamonTags, false) + val blockMemFactory = new BlockMemFactory(blockStore, maxMetaSize, kamonTags, false) val nativeMemoryManager = new NativeMemoryManager(nativeMemSize, kamonTags) diff --git a/core/src/main/scala/filodb.core/downsample/ShardDownsampler.scala b/core/src/main/scala/filodb.core/downsample/ShardDownsampler.scala index 1277f5bb35..81ac896123 100644 --- a/core/src/main/scala/filodb.core/downsample/ShardDownsampler.scala +++ b/core/src/main/scala/filodb.core/downsample/ShardDownsampler.scala @@ -1,7 +1,6 @@ package filodb.core.downsample import com.typesafe.scalalogging.StrictLogging -import kamon.Kamon import filodb.core.binaryrecord2.RecordBuilder import filodb.core.memstore.{TimeSeriesPartition, TimeSeriesShardStats} @@ -64,10 +63,6 @@ class ShardDownsampler(datasetName: String, chunksets: ChunkInfoIterator, records: Seq[DownsampleRecords]): Unit = { if (enabled) { - val downsampleTrace = Kamon.spanBuilder("memstore-downsample-records-trace") - .asChildOf(Kamon.currentSpan()) - .tag("dataset", datasetName) - .tag("shard", shardNum).start() while (chunksets.hasNext) { val chunkset = chunksets.nextInfoReader val startTime = chunkset.startTime @@ -103,7 +98,6 @@ class ShardDownsampler(datasetName: String, } } } - downsampleTrace.finish() } } } diff --git a/core/src/main/scala/filodb.core/memstore/DemandPagedChunkStore.scala b/core/src/main/scala/filodb.core/memstore/DemandPagedChunkStore.scala index 14048ba2ba..726bd2838c 100644 --- a/core/src/main/scala/filodb.core/memstore/DemandPagedChunkStore.scala +++ b/core/src/main/scala/filodb.core/memstore/DemandPagedChunkStore.scala @@ -3,11 +3,9 @@ package filodb.core.memstore import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer -import scala.concurrent.duration._ import com.typesafe.scalalogging.StrictLogging import monix.eval.Task -import org.jctools.maps.NonBlockingHashMapLong import spire.syntax.cfor._ import filodb.core.store._ @@ -30,39 +28,23 @@ import filodb.memory.format.UnsafeUtils * * @param tsShard the TimeSeriesShard containing the time series for the given shard * @param blockManager The block manager to be used for block allocation - * @param chunkRetentionHours number of hours chunks need to be retained for. Beyond this time, ODP blocks will be - * marked as reclaimable even if they are not full, so they can be reused for newer data. */ class DemandPagedChunkStore(tsShard: TimeSeriesShard, - val blockManager: BlockManager, - chunkRetentionHours: Int) + val blockManager: BlockManager) extends RawToPartitionMaker with StrictLogging { - val flushIntervalMillis = tsShard.storeConfig.flushInterval.toMillis - val retentionMillis = chunkRetentionHours * (1.hour.toMillis) - - // block factories for each time bucket - private val memFactories = new NonBlockingHashMapLong[BlockMemFactory](chunkRetentionHours, false) import TimeSeriesShard._ - import collection.JavaConverters._ private val baseContext = Map("dataset" -> tsShard.ref.toString, "shard" -> tsShard.shardNum.toString) - private def getMemFactory(bucket: Long): BlockMemFactory = { - val factory = memFactories.get(bucket) - if (factory == UnsafeUtils.ZeroPointer) { - val newFactory = new BlockMemFactory(blockManager, - Some(bucket), - tsShard.maxMetaSize, - baseContext ++ Map("bucket" -> bucket.toString), - markFullBlocksAsReclaimable = true) - memFactories.put(bucket, newFactory) - newFactory - } else { - factory - } - } + /* + * Only one BlockMemFactory for ODP per shard needed (pooling not needed) since all ODP + * allocations happen on a single thread + */ + val memFactory = new BlockMemFactory(blockManager, + tsShard.maxMetaSize, baseContext ++ Map("odp" -> "true"), + markFullBlocksAsReclaimable = true) /** * Stores raw chunks into offheap memory and populates chunks into partition @@ -84,18 +66,19 @@ extends RawToPartitionMaker with StrictLogging { // possible to guard against this by forcing an allocation, but it doesn't make sense // to allocate a block just for storing an unnecessary metadata entry. if (!rawVectors.isEmpty) { - val memFactory = getMemFactory(timeBucketForChunkSet(infoBytes)) val chunkID = ChunkSetInfo.getChunkID(infoBytes) if (!tsPart.chunkmapContains(chunkID)) { val chunkPtrs = new ArrayBuffer[BinaryVectorPtr](rawVectors.length) - memFactory.startMetaSpan() var metaAddr: Long = 0 - try { - copyToOffHeap(rawVectors, memFactory, chunkPtrs) - } finally { - metaAddr = memFactory.endMetaSpan(writeMeta(_, tsPart.partID, infoBytes, chunkPtrs), - tsPart.schema.data.blockMetaSize.toShort) + memFactory.synchronized { + memFactory.startMetaSpan() + try { + copyToOffHeap(rawVectors, memFactory, chunkPtrs) + } finally { + metaAddr = memFactory.endMetaSpan(writeMeta(_, tsPart.partID, infoBytes, chunkPtrs), + tsPart.schema.data.blockMetaSize.toShort) + } } require(metaAddr != 0) val infoAddr = metaAddr + 4 // Important: don't point at partID @@ -119,13 +102,6 @@ extends RawToPartitionMaker with StrictLogging { } //scalastyle:on - /** - * For a given chunkset, this method calculates the time bucket the chunks fall in. - * It is used in deciding which BlockMemFactory to use while allocating off-heap memory for this chunk. - */ - private def timeBucketForChunkSet(infoBytes: Array[Byte]): Long = - (ChunkSetInfo.getEndTime(infoBytes) / flushIntervalMillis) * flushIntervalMillis - /** * Copies the onHeap contents read from ColStore into off-heap using the given memFactory. * If an exception is thrown by this method, the tail of chunkPtrs sequence isn't filled in. @@ -144,15 +120,4 @@ extends RawToPartitionMaker with StrictLogging { } } - /** - * Ensures the oldest ODP time buckets, blocks, and BlockMemFactory's are reclaimable and cleaned up - * so we don't leak memory and blocks. Call this ideally every flushInterval. - */ - def cleanupOldestBuckets(): Unit = { - blockManager.markBucketedBlocksReclaimable(System.currentTimeMillis - retentionMillis) - // Now, iterate through memFactories and clean up ones with no blocks - memFactories.keySet.asScala.foreach { bucket => - if (!blockManager.hasTimeBucket(bucket)) memFactories.remove(bucket) - } - } } diff --git a/core/src/main/scala/filodb.core/memstore/IndexBootstrapper.scala b/core/src/main/scala/filodb.core/memstore/IndexBootstrapper.scala index 048b548821..67ced68b4d 100644 --- a/core/src/main/scala/filodb.core/memstore/IndexBootstrapper.scala +++ b/core/src/main/scala/filodb.core/memstore/IndexBootstrapper.scala @@ -1,6 +1,7 @@ package filodb.core.memstore import kamon.Kamon +import kamon.metric.MeasurementUnit import monix.eval.Task import monix.reactive.Observable @@ -28,11 +29,12 @@ class IndexBootstrapper(colStore: ColumnStore) { shardNum: Int, ref: DatasetRef) (assignPartId: PartKeyRecord => Int): Task[Long] = { - val tracer = Kamon.spanBuilder("memstore-recover-index-latency") - .asChildOf(Kamon.currentSpan()) - .tag("dataset", ref.dataset) - .tag("shard", shardNum).start() + val recoverIndexLatency = Kamon.histogram("shard-recover-index-latency", + MeasurementUnit.time.milliseconds) + .withTag("dataset", ref.dataset) + .withTag("shard", shardNum) + val start = System.currentTimeMillis() colStore.scanPartKeys(ref, shardNum) .map { pk => val partId = assignPartId(pk) @@ -41,7 +43,7 @@ class IndexBootstrapper(colStore: ColumnStore) { .countL .map { count => index.refreshReadersBlocking() - tracer.finish() + recoverIndexLatency.record(System.currentTimeMillis() - start) count } } @@ -64,11 +66,11 @@ class IndexBootstrapper(colStore: ColumnStore) { schemas: Schemas, parallelism: Int = Runtime.getRuntime.availableProcessors()) (lookUpOrAssignPartId: Array[Byte] => Int): Task[Long] = { - val tracer = Kamon.spanBuilder("downsample-store-refresh-index-latency") - .asChildOf(Kamon.currentSpan()) - .tag("dataset", ref.dataset) - .tag("shard", shardNum).start() - + val recoverIndexLatency = Kamon.histogram("downsample-store-refresh-index-latency", + MeasurementUnit.time.milliseconds) + .withTag("dataset", ref.dataset) + .withTag("shard", shardNum) + val start = System.currentTimeMillis() Observable.fromIterable(fromHour to toHour).flatMap { hour => colStore.getPartKeysByUpdateHour(ref, shardNum, hour) }.mapAsync(parallelism) { pk => @@ -83,7 +85,7 @@ class IndexBootstrapper(colStore: ColumnStore) { .countL .map { count => index.refreshReadersBlocking() - tracer.finish() + recoverIndexLatency.record(System.currentTimeMillis() - start) count } } diff --git a/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala b/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala index e036791fd7..c76bd49e0c 100644 --- a/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala +++ b/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala @@ -13,6 +13,7 @@ import monix.reactive.{Observable, OverflowStrategy} import filodb.core.{DatasetRef, Types} import filodb.core.binaryrecord2.RecordSchema +import filodb.core.memstore.ratelimit.QuotaSource import filodb.core.metadata.Schemas import filodb.core.query.QuerySession import filodb.core.store._ @@ -25,13 +26,15 @@ import filodb.memory.NativeMemoryManager class OnDemandPagingShard(ref: DatasetRef, schemas: Schemas, storeConfig: StoreConfig, + quotaSource: QuotaSource, shardNum: Int, bufferMemoryManager: NativeMemoryManager, rawStore: ColumnStore, metastore: MetaStore, evictionPolicy: PartitionEvictionPolicy) (implicit ec: ExecutionContext) extends -TimeSeriesShard(ref, schemas, storeConfig, shardNum, bufferMemoryManager, rawStore, metastore, evictionPolicy)(ec) { +TimeSeriesShard(ref, schemas, storeConfig, quotaSource, shardNum, bufferMemoryManager, rawStore, + metastore, evictionPolicy)(ec) { import TimeSeriesShard._ import FiloSchedulers._ diff --git a/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala b/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala index e064a1ae9e..6204a12ca1 100644 --- a/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala +++ b/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala @@ -12,6 +12,7 @@ import com.googlecode.javaewah.{EWAHCompressedBitmap, IntIterator} import com.typesafe.scalalogging.StrictLogging import java.util import kamon.Kamon +import kamon.metric.MeasurementUnit import org.apache.lucene.analysis.standard.StandardAnalyzer import org.apache.lucene.document._ import org.apache.lucene.document.Field.Store @@ -76,6 +77,21 @@ class PartKeyLuceneIndex(ref: DatasetRef, import PartKeyLuceneIndex._ + val startTimeLookupLatency = Kamon.histogram("index-startTimes-for-odp-lookup-latency", + MeasurementUnit.time.milliseconds) + .withTag("dataset", ref.dataset) + .withTag("shard", shardNum) + + val queryIndexLookupLatency = Kamon.histogram("index-partition-lookup-latency", + MeasurementUnit.time.milliseconds) + .withTag("dataset", ref.dataset) + .withTag("shard", shardNum) + + val partIdFromPartKeyLookupLatency = Kamon.histogram("index-ingestion-partId-lookup-latency", + MeasurementUnit.time.milliseconds) + .withTag("dataset", ref.dataset) + .withTag("shard", shardNum) + private val numPartColumns = schema.columns.length private val indexDiskLocation = diskLocation.getOrElse(createTempDir(ref, shardNum)).toPath private val mMapDirectory = new MMapDirectory(indexDiskLocation) @@ -210,7 +226,7 @@ class PartKeyLuceneIndex(ref: DatasetRef, /** * Fetch values/terms for a specific column/key/field, in order from most frequent on down. - * Note that it iterates through all docs up to a certain limit only, so if there are too many terms + * Note that it iterates through all docs up to a certain limit only, so if there are too many terms * it will not report an accurate top k in exchange for not running too long. * @param fieldName the name of the column/field/key to get terms for * @param topK the number of top k results to fetch @@ -360,11 +376,9 @@ class PartKeyLuceneIndex(ref: DatasetRef, * Called when a document is updated with new endTime */ def startTimeFromPartIds(partIds: Iterator[Int]): debox.Map[Int, Long] = { - val span = Kamon.spanBuilder("index-startTimes-for-odp-lookup-latency") - .asChildOf(Kamon.currentSpan()) - .tag("dataset", ref.dataset) - .tag("shard", shardNum) - .start() + + val startExecute = System.currentTimeMillis() + val span = Kamon.currentSpan() val collector = new PartIdStartTimeCollector() val terms = new util.ArrayList[BytesRef]() partIds.foreach { pId => @@ -374,7 +388,9 @@ class PartKeyLuceneIndex(ref: DatasetRef, // more efficient within Lucene withNewSearcher(s => s.search(new TermInSetQuery(PART_ID, terms), collector)) span.tag(s"num-partitions-to-page", terms.size()) - span.finish() + val latency = System.currentTimeMillis - startExecute + span.mark(s"index-startTimes-for-odp-lookup-latency=${latency}ms") + startTimeLookupLatency.record(latency) collector.startTimes } @@ -493,11 +509,9 @@ class PartKeyLuceneIndex(ref: DatasetRef, startTime: Long, endTime: Long, collector: Collector): Unit = { - val partKeySpan = Kamon.spanBuilder("index-partition-lookup-latency") - .tag("dataset", ref.dataset) - .tag("shard", shardNum) - .asChildOf(Kamon.currentSpan()) - .start() + + val startExecute = System.currentTimeMillis() + val span = Kamon.currentSpan() val booleanQuery = new BooleanQuery.Builder columnFilters.foreach { filter => val q = leafFilter(filter.column, filter.filter) @@ -508,7 +522,9 @@ class PartKeyLuceneIndex(ref: DatasetRef, val query = booleanQuery.build() logger.debug(s"Querying dataset=$ref shard=$shardNum partKeyIndex with: $query") withNewSearcher(s => s.search(query, collector)) - partKeySpan.finish() + val latency = System.currentTimeMillis - startExecute + span.mark(s"index-partition-lookup-latency=${latency}ms") + queryIndexLookupLatency.record(latency) } def partIdFromPartKeySlow(partKeyBase: Any, @@ -517,11 +533,7 @@ class PartKeyLuceneIndex(ref: DatasetRef, val columnFilters = schema.binSchema.toStringPairs(partKeyBase, partKeyOffset) .map { pair => ColumnFilter(pair._1, Filter.Equals(pair._2)) } - val partKeySpan = Kamon.spanBuilder("index-partition-lookup-latency") - .asChildOf(Kamon.currentSpan()) - .tag("dataset", ref.dataset) - .tag("shard", shardNum) - .start() + val startExecute = System.currentTimeMillis() val booleanQuery = new BooleanQuery.Builder columnFilters.foreach { filter => val q = leafFilter(filter.column, filter.filter) @@ -542,7 +554,7 @@ class PartKeyLuceneIndex(ref: DatasetRef, } val collector = new ActionCollector(handleMatch) withNewSearcher(s => s.search(query, collector)) - partKeySpan.finish() + partIdFromPartKeyLookupLatency.record(Math.max(0, System.currentTimeMillis - startExecute)) chosenPartId } } diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala index 4423f4ca92..7aaf49944e 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala @@ -13,6 +13,7 @@ import org.jctools.maps.NonBlockingHashMapLong import filodb.core.{DatasetRef, Response, Types} import filodb.core.downsample.DownsampleConfig +import filodb.core.memstore.ratelimit.{CardinalityRecord, ConfigQuotaSource} import filodb.core.metadata.Schemas import filodb.core.query.{ColumnFilter, QuerySession} import filodb.core.store._ @@ -30,6 +31,7 @@ extends MemStore 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 @@ -45,6 +47,8 @@ extends MemStore with StrictLogging { def setup(ref: DatasetRef, schemas: Schemas, shard: Int, storeConf: StoreConfig, downsample: DownsampleConfig = DownsampleConfig.disabled): Unit = synchronized { val shards = datasets.getOrElseUpdate(ref, new NonBlockingHashMapLong[TimeSeriesShard](32, false)) + val quotaSource = quotaSources.getOrElseUpdate(ref, + new ConfigQuotaSource(config, schemas.part.options.shardKeyColumns.length)) if (shards.containsKey(shard)) { throw ShardAlreadySetup(ref, shard) } else { @@ -55,12 +59,21 @@ extends MemStore with StrictLogging { new NativeMemoryManager(bufferMemorySize, tags) }) - val tsdb = new OnDemandPagingShard(ref, schemas, storeConf, shard, memFactory, store, metastore, + val tsdb = new OnDemandPagingShard(ref, schemas, storeConf, quotaSource, shard, memFactory, store, metastore, partEvictionPolicy) shards.put(shard, tsdb) } } + def topKCardinality(ref: DatasetRef, shards: Seq[Int], + shardKeyPrefix: Seq[String], k: Int): Seq[CardinalityRecord] = { + datasets.get(ref).toSeq + .flatMap { ts => + ts.values().asScala + .filter(s => shards.isEmpty || shards.contains(s.shardNum)) + .flatMap(_.topKCardinality(k, shardKeyPrefix)) + } + } /** * WARNING: use only for testing. Not performant */ @@ -242,6 +255,7 @@ extends MemStore with StrictLogging { def reset(): Unit = { datasets.clear() + quotaSources.clear() store.reset() } @@ -256,8 +270,8 @@ extends MemStore with StrictLogging { // Release memory etc. def shutdown(): Unit = { + quotaSources.clear() datasets.values.foreach(_.values.asScala.foreach(_.shutdown())) - datasets.values.foreach(_.values().asScala.foreach(_.closePartKeyIndex())) reset() } diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala index 7afbc4678c..fc1c429e09 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala @@ -121,10 +121,19 @@ extends ChunkMap(memFactory, initMapSize) with ReadablePartition { * If ingesting a new row causes WriteBuffers to overflow, then the current chunks are encoded, a new set * of appending chunks are obtained, and we re-ingest into the new chunks. * + * createChunkAtFlushBoundary - switch buffers and create chunk when current sample's timestamp crosses flush boundary + * e.g. for a flush-interval of 1hour, if new sample falls in different hour than last sample, then switch buffers + * and create chunk. This helps in aligning chunks across Active/Active HA clusters and facilitates chunk migration + * between the clusters during disaster recovery. + * Note: Enabling this might result into creation of smaller suboptimal chunks. + * * @param ingestionTime time (as milliseconds from 1970) at the data source * @param blockHolder the BlockMemFactory to use for encoding chunks in case of WriteBuffer overflow + * @param createChunkAtFlushBoundary create time bucketed (flush-interval) chunks + * @param flushIntervalMillis flush-interval in milliseconds */ def ingest(ingestionTime: Long, row: RowReader, blockHolder: BlockMemFactory, + createChunkAtFlushBoundary: Boolean, flushIntervalMillis: Option[Long], maxChunkTime: Long = Long.MaxValue): Unit = { // NOTE: lastTime is not persisted for recovery. Thus the first sample after recovery might still be out of order. val ts = schema.timestamp(row) @@ -136,18 +145,26 @@ extends ChunkMap(memFactory, initMapSize) with ReadablePartition { val newChunk = currentChunks == nullChunks if (newChunk) initNewChunk(ts, ingestionTime) - if (ts - currentInfo.startTime > maxChunkTime) { + if(!newChunk && createChunkAtFlushBoundary + && ts/flushIntervalMillis.get != timestampOfLatestSample/flushIntervalMillis.get) { + // we have reached maximum userTime in chunk. switch buffers, start a new chunk and ingest + switchBuffersAndIngest(ingestionTime, ts, row, blockHolder, + createChunkAtFlushBoundary, flushIntervalMillis, maxChunkTime) + } else if (ts - currentInfo.startTime > maxChunkTime) { // we have reached maximum userTime in chunk. switch buffers, start a new chunk and ingest - switchBuffersAndIngest(ingestionTime, ts, row, blockHolder, maxChunkTime) + switchBuffersAndIngest(ingestionTime, ts, row, blockHolder, + createChunkAtFlushBoundary, flushIntervalMillis, maxChunkTime) } else { cforRange { 0 until schema.numDataColumns } { col => currentChunks(col).addFromReaderNoNA(row, col) match { case r: VectorTooSmall => - switchBuffersAndIngest(ingestionTime, ts, row, blockHolder, maxChunkTime) + switchBuffersAndIngest(ingestionTime, ts, row, blockHolder, + createChunkAtFlushBoundary, flushIntervalMillis, maxChunkTime) return // Different histogram bucket schema: need a new vector here case BucketSchemaMismatch => - switchBuffersAndIngest(ingestionTime, ts, row, blockHolder, maxChunkTime) + switchBuffersAndIngest(ingestionTime, ts, row, blockHolder, + createChunkAtFlushBoundary, flushIntervalMillis, maxChunkTime) return case other: AddResponse => } @@ -168,12 +185,15 @@ extends ChunkMap(memFactory, initMapSize) with ReadablePartition { userTime: Long, row: RowReader, blockHolder: BlockMemFactory, - maxChunkTime: Long): Unit = { + createChunkAtFlushBoundary: Boolean, + flushIntervalMillis: Option[Long], + maxChunkTime: Long = Long.MaxValue): Unit = { // NOTE: a very bad infinite loop is possible if switching buffers fails (if the # rows is 0) but one of the // vectors fills up. This is possible if one vector fills up but the other one does not for some reason. // So we do not call ingest again unless switcing buffers succeeds. // re-ingest every element, allocating new WriteBuffers - if (switchBuffers(blockHolder, encode = true)) { ingest(ingestionTime, row, blockHolder, maxChunkTime) } + if (switchBuffers(blockHolder, encode = true)) { ingest(ingestionTime, row, blockHolder, + createChunkAtFlushBoundary, flushIntervalMillis, maxChunkTime) } else { _log.warn("EMPTY WRITEBUFFERS when switchBuffers called! Likely a severe bug!!! " + s"Part=$stringPartition userTime=$userTime numRows=${currentInfo.numRows}") } } @@ -220,39 +240,41 @@ extends ChunkMap(memFactory, initMapSize) with ReadablePartition { * Optimized chunks as well as chunk metadata are written into offheap block memory so they no longer consume */ private def encodeOneChunkset(info: ChunkSetInfo, appenders: AppenderArray, blockHolder: BlockMemFactory) = { - blockHolder.startMetaSpan() - val frozenVectors = try { - // optimize and compact chunks - appenders.zipWithIndex.map { case (appender, i) => - // This assumption cannot break. We should ensure one vector can be written - // to one block always atleast as per the current design. - // If this gets triggered, decrease the max writebuffer size so smaller chunks are encoded - require(blockHolder.blockAllocationSize() > appender.frozenSize) - val optimized = appender.optimize(blockHolder) - shardStats.encodedBytes.increment(BinaryVector.totalBytes(nativePtrReader, optimized)) - if (schema.data.columns(i).columnType == Column.ColumnType.HistogramColumn) - shardStats.encodedHistBytes.increment(BinaryVector.totalBytes(nativePtrReader, optimized)) - optimized + blockHolder.synchronized { + blockHolder.startMetaSpan() + val frozenVectors = try { + // optimize and compact chunks + appenders.zipWithIndex.map { case (appender, i) => + // This assumption cannot break. We should ensure one vector can be written + // to one block always atleast as per the current design. + // If this gets triggered, decrease the max writebuffer size so smaller chunks are encoded + require(blockHolder.blockAllocationSize() > appender.frozenSize) + val optimized = appender.optimize(blockHolder) + shardStats.encodedBytes.increment(BinaryVector.totalBytes(nativePtrReader, optimized)) + if (schema.data.columns(i).columnType == Column.ColumnType.HistogramColumn) + shardStats.encodedHistBytes.increment(BinaryVector.totalBytes(nativePtrReader, optimized)) + optimized + } + } catch { case e: Exception => + // Shutdown process right away! Reaching this state means that we could not reclaim + // a whole bunch of blocks possibly because they were not marked as reclaimable, + // because of some bug. Cleanup or rollback at this point is not viable. + Shutdown.haltAndCatchFire(new RuntimeException("Error occurred when encoding vectors", e)) + throw e } - } catch { case e: Exception => - // Shutdown process right away! Reaching this state means that we could not reclaim - // a whole bunch of blocks possibly because they were not marked as reclaimable, - // because of some bug. Cleanup or rollback at this point is not viable. - Shutdown.haltAndCatchFire(new RuntimeException("Error occurred when encoding vectors", e)) - throw e + shardStats.numSamplesEncoded.increment(info.numRows) + // Now, write metadata into offheap block metadata space and update infosChunks + val metaAddr = blockHolder.endMetaSpan(TimeSeriesShard.writeMeta(_, partID, info, frozenVectors), + schema.data.blockMetaSize.toShort) + + val newInfo = ChunkSetInfo(metaAddr + 4) + _log.trace(s"Adding new chunk ${newInfo.debugString} to part $stringPartition") + infoPut(newInfo) + + // release older write buffers back to pool. Nothing at this point should reference the older appenders. + bufferPool.release(info.infoAddr, appenders) + frozenVectors } - shardStats.numSamplesEncoded.increment(info.numRows) - // Now, write metadata into offheap block metadata space and update infosChunks - val metaAddr = blockHolder.endMetaSpan(TimeSeriesShard.writeMeta(_, partID, info, frozenVectors), - schema.data.blockMetaSize.toShort) - - val newInfo = ChunkSetInfo(metaAddr + 4) - _log.trace(s"Adding new chunk ${newInfo.debugString} to part $stringPartition") - infoPut(newInfo) - - // release older write buffers back to pool. Nothing at this point should reference the older appenders. - bufferPool.release(info.infoAddr, appenders) - frozenVectors } /** @@ -473,12 +495,14 @@ TimeSeriesPartition(partID, schema, partitionKey, shard, bufferPool, shardStats, _log.info(s"Creating TracingTimeSeriesPartition dataset=$ref schema=${schema.name} partId=$partID $stringPartition") - override def ingest(ingestionTime: Long, row: RowReader, blockHolder: BlockMemFactory, maxChunkTime: Long): Unit = { + override def ingest(ingestionTime: Long, row: RowReader, blockHolder: BlockMemFactory, + createChunkAtFlushBoundary: Boolean, flushIntervalMillis: Option[Long], + maxChunkTime: Long = Long.MaxValue): Unit = { val ts = row.getLong(0) _log.info(s"Ingesting dataset=$ref schema=${schema.name} shard=$shard partId=$partID $stringPartition " + s"ingestionTime=$ingestionTime ts=$ts " + (1 until schema.numDataColumns).map(row.getAny).mkString("[", ",", "]")) - super.ingest(ingestionTime, row, blockHolder, maxChunkTime) + super.ingest(ingestionTime, row, blockHolder, createChunkAtFlushBoundary, flushIntervalMillis, maxChunkTime) } override def switchBuffers(blockHolder: BlockMemFactory, encode: Boolean = false): Boolean = { diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala index baff77d164..be2e4cc3fc 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala @@ -25,6 +25,7 @@ import spire.syntax.cfor._ import filodb.core.{ErrorResponse, _} import filodb.core.binaryrecord2._ +import filodb.core.memstore.ratelimit.{CardinalityRecord, CardinalityTracker, QuotaSource, RocksDbCardinalityStore} import filodb.core.metadata.{Schema, Schemas} import filodb.core.query.{ColumnFilter, QuerySession} import filodb.core.store._ @@ -104,7 +105,10 @@ class TimeSeriesShardStats(dataset: DatasetRef, shardNum: Int) { * expected), then the delay reflects the delay between the generation of the samples and * receiving them, assuming that the clocks are in sync. */ - val ingestionClockDelay = Kamon.gauge("ingestion-clock-delay").withTags(TagSet.from(tags)) + val ingestionClockDelay = Kamon.gauge("ingestion-clock-delay", + MeasurementUnit.time.milliseconds).withTags(TagSet.from(tags)) + val chunkFlushTaskLatency = Kamon.histogram("chunk-flush-task-latency-after-retries", + MeasurementUnit.time.milliseconds).withTags(TagSet.from(tags)) } object TimeSeriesShard { @@ -220,6 +224,7 @@ object SchemaMismatch { class TimeSeriesShard(val ref: DatasetRef, val schemas: Schemas, val storeConfig: StoreConfig, + quotaSource: QuotaSource, val shardNum: Int, val bufferMemoryManager: NativeMemoryManager, colStore: ColumnStore, @@ -251,6 +256,19 @@ class TimeSeriesShard(val ref: DatasetRef, private[memstore] final val partKeyIndex = new PartKeyLuceneIndex(ref, schemas.part, shardNum, storeConfig.demandPagedRetentionPeriod) + private val cardTracker: CardinalityTracker = if (storeConfig.meteringEnabled) { + // FIXME switch this to some local-disk based store when we graduate out of POC mode + val cardStore = new RocksDbCardinalityStore(ref, shardNum) + + val defaultQuota = quotaSource.getDefaults(ref) + val tracker = new CardinalityTracker(ref, shardNum, schemas.part.options.shardKeyColumns.length, + defaultQuota, cardStore) + quotaSource.getQuotas(ref).foreach { q => + tracker.setQuota(q.shardKeyPrefix, q.quota) + } + tracker + } else UnsafeUtils.ZeroPointer.asInstanceOf[CardinalityTracker] + /** * Keeps track of count of rows ingested into memstore, not necessarily flushed. * This is generally used to report status and metrics. @@ -330,9 +348,9 @@ class TimeSeriesShard(val ref: DatasetRef, // Each shard has a single ingestion stream at a time. This BlockMemFactory is used for buffer overflow encoding // strictly during ingest() and switchBuffers(). - private[core] val overflowBlockFactory = new BlockMemFactory(blockStore, None, maxMetaSize, + private[core] val overflowBlockFactory = new BlockMemFactory(blockStore, maxMetaSize, shardTags ++ Map("overflow" -> "true"), true) - val partitionMaker = new DemandPagedChunkStore(this, blockStore, chunkRetentionHours) + val partitionMaker = new DemandPagedChunkStore(this, blockStore) private val partKeyBuilder = new RecordBuilder(MemFactory.onHeapFactory, reuseOneContainer = true) private val partKeyArray = partKeyBuilder.allContainers.head.base.asInstanceOf[Array[Byte]] @@ -392,12 +410,12 @@ class TimeSeriesShard(val ref: DatasetRef, // Flush groups when ingestion time is observed to cross a time boundary (typically an hour), // plus a group-specific offset. This simplifies disaster recovery -- chunks can be copied // without concern that they may overlap in time. - private val flushBoundaryMillis = storeConfig.flushInterval.toMillis + private val flushBoundaryMillis = Option(storeConfig.flushInterval.toMillis) // Defines the group-specific flush offset, to distribute the flushes around such they don't // all flush at the same time. With an hourly boundary and 60 flush groups, flushes are // scheduled once a minute. - private val flushOffsetMillis = flushBoundaryMillis / numGroups + private val flushOffsetMillis = flushBoundaryMillis.get / numGroups private[memstore] val evictedPartKeys = BloomFilter[PartKey](storeConfig.evictedPkBfCapacity, falsePositiveRate = 0.01)(new CanGenerateHashFrom[PartKey] { @@ -532,6 +550,11 @@ class TimeSeriesShard(val ref: DatasetRef, _offset } + def topKCardinality(k: Int, shardKeyPrefix: Seq[String]): Seq[CardinalityRecord] = { + if (storeConfig.meteringEnabled) cardTracker.topk(k, shardKeyPrefix) + else throw new IllegalArgumentException("Metering is not enabled") + } + def startFlushingIndex(): Unit = partKeyIndex.startFlushThread(storeConfig.partIndexFlushMinDelaySeconds, storeConfig.partIndexFlushMaxDelaySeconds) @@ -555,11 +578,11 @@ class TimeSeriesShard(val ref: DatasetRef, // scalastyle:off method.length private[memstore] def bootstrapPartKey(pk: PartKeyRecord): Int = { assertThreadName(IngestSchedName) + val schemaId = RecordSchema.schemaID(pk.partKey, UnsafeUtils.arayOffset) + val schema = schemas(schemaId) val partId = if (pk.endTime == Long.MaxValue) { // this is an actively ingesting partition val group = partKeyGroup(schemas.part.binSchema, pk.partKey, UnsafeUtils.arayOffset, numGroups) - val schemaId = RecordSchema.schemaID(pk.partKey, UnsafeUtils.arayOffset) - val schema = schemas(schemaId) if (schema != Schemas.UnknownSchema) { val part = createNewPartition(pk.partKey, UnsafeUtils.arayOffset, group, CREATE_NEW_PARTID, schema, 4) // In theory, we should not get an OutOfMemPartition here since @@ -600,6 +623,10 @@ class TimeSeriesShard(val ref: DatasetRef, else activelyIngesting -= partId } shardStats.indexRecoveryNumRecordsProcessed.increment() + if (storeConfig.meteringEnabled) { + val shardKey = schema.partKeySchema.colValues(pk.partKey, UnsafeUtils.arayOffset, schema.options.shardKeyColumns) + cardTracker.incrementCount(shardKey) + } partId } @@ -715,8 +742,6 @@ class TimeSeriesShard(val ref: DatasetRef, */ def refreshPartKeyIndexBlocking(): Unit = partKeyIndex.refreshReadersBlocking() - def closePartKeyIndex(): Unit = partKeyIndex.closeIndex() - def numRowsIngested: Long = ingested def numActivePartitions: Int = partSet.size @@ -767,6 +792,11 @@ class TimeSeriesShard(val ref: DatasetRef, if (!p.ingesting) { logger.debug(s"Purging partition with partId=${p.partID} ${p.stringPartition} from " + s"memory in dataset=$ref shard=$shardNum") + if (storeConfig.meteringEnabled) { + val schema = p.schema + val shardKey = schema.partKeySchema.colValues(p.partKeyBase, p.partKeyOffset, schema.options.shardKeyColumns) + cardTracker.decrementCount(shardKey) + } removePartition(p) removedParts += p.partID numDeleted += 1 @@ -810,7 +840,7 @@ class TimeSeriesShard(val ref: DatasetRef, As written the code the same thing but with fewer operations. It's also a bit shorter, but you also had to read this comment... */ - if (oldTimestamp / flushBoundaryMillis != newTimestamp / flushBoundaryMillis) { + if (oldTimestamp / flushBoundaryMillis.get != newTimestamp / flushBoundaryMillis.get) { // Flush out the group before ingesting records for a new hour (by group offset). tasks += createFlushTask(prepareFlushGroup(group)) } @@ -868,11 +898,7 @@ class TimeSeriesShard(val ref: DatasetRef, private def doFlushSteps(flushGroup: FlushGroup, partitionIt: Iterator[TimeSeriesPartition]): Task[Response] = { assertThreadName(IngestSchedName) - - val tracer = Kamon.spanBuilder("chunk-flush-task-latency-after-retries") - .asChildOf(Kamon.currentSpan()) - .tag("dataset", ref.dataset) - .tag("shard", shardNum).start() + val flushStart = System.currentTimeMillis() // Only allocate the blockHolder when we actually have chunks/partitions to flush val blockHolder = blockFactoryPool.checkout(Map("flushGroup" -> flushGroup.groupNum.toString)) @@ -931,7 +957,7 @@ class TimeSeriesShard(val ref: DatasetRef, blockHolder.markFullBlocksReclaimable() blockFactoryPool.release(blockHolder) flushDoneTasks(flushGroup, resp) - tracer.finish() + shardStats.chunkFlushTaskLatency.record(System.currentTimeMillis() - flushStart) } catch { case e: Throwable => logger.error(s"Error when wrapping up doFlushSteps in dataset=$ref shard=$shardNum", e) } @@ -947,7 +973,6 @@ class TimeSeriesShard(val ref: DatasetRef, logger.info(s"Flush of dataset=$ref shard=$shardNum group=${flushGroup.groupNum} " + s"flushWatermark=${flushGroup.flushWatermark} response=$resp offset=${_offset}") } - partitionMaker.cleanupOldestBuckets() // Some partitions might be evictable, see if need to free write buffer memory checkEnableAddPartitions() updateGauges() @@ -1099,11 +1124,11 @@ class TimeSeriesShard(val ref: DatasetRef, private def addPartitionForIngestion(recordBase: Any, recordOff: Long, schema: Schema, group: Int) = { assertThreadName(IngestSchedName) // TODO: remove when no longer needed - or figure out how to log only for tracing partitions - logger.debug(s"Adding ingestion record details: ${schema.ingestionSchema.debugString(recordBase, recordOff)}") + logger.trace(s"Adding ingestion record details: ${schema.ingestionSchema.debugString(recordBase, recordOff)}") val partKeyOffset = schema.comparator.buildPartKeyFromIngest(recordBase, recordOff, partKeyBuilder) val previousPartId = lookupPreviouslyAssignedPartId(partKeyArray, partKeyOffset) // TODO: remove when no longer needed - logger.debug(s"Adding part key details: ${schema.partKeySchema.debugString(partKeyArray, partKeyOffset)}") + logger.trace(s"Adding part key details: ${schema.partKeySchema.debugString(partKeyArray, partKeyOffset)}") val newPart = createNewPartition(partKeyArray, partKeyOffset, group, previousPartId, schema) if (newPart != OutOfMemPartition) { val partId = newPart.partID @@ -1112,6 +1137,11 @@ class TimeSeriesShard(val ref: DatasetRef, // add new lucene entry if this partKey was never seen before // causes endTime to be set to Long.MaxValue partKeyIndex.addPartKey(newPart.partKeyBytes, partId, startTime)() + if (storeConfig.meteringEnabled) { + val shardKey = schema.partKeySchema.colValues(newPart.partKeyBase, newPart.partKeyOffset, + schema.options.shardKeyColumns) + cardTracker.incrementCount(shardKey) + } } else { // newly created partition is re-ingesting now, so update endTime updatePartEndTimeInIndex(newPart, Long.MaxValue) @@ -1153,7 +1183,8 @@ class TimeSeriesShard(val ref: DatasetRef, val tsp = part.asInstanceOf[TimeSeriesPartition] brRowReader.schema = schema.ingestionSchema brRowReader.recordOffset = recordOff - tsp.ingest(ingestionTime, brRowReader, overflowBlockFactory, maxChunkTime) + tsp.ingest(ingestionTime, brRowReader, overflowBlockFactory, + storeConfig.timeAlignedChunksEnabled, flushBoundaryMillis, maxChunkTime) // Below is coded to work concurrently with logic in updateIndexWithEndTime // where we try to de-activate an active time series if (!tsp.ingesting) { @@ -1510,6 +1541,9 @@ class TimeSeriesShard(val ref: DatasetRef, } def shutdown(): Unit = { + if (storeConfig.meteringEnabled) { + cardTracker.close() + } evictedPartKeys.synchronized { if (!evictedPartKeysDisposed) { evictedPartKeysDisposed = true @@ -1517,6 +1551,7 @@ class TimeSeriesShard(val ref: DatasetRef, } } reset() // Not really needed, but clear everything just to be consistent + partKeyIndex.closeIndex() logger.info(s"Shutting down dataset=$ref shard=$shardNum") /* Don't explcitly free the memory just yet. These classes instead rely on a finalize method to ensure that no threads are accessing the memory before it's freed. diff --git a/core/src/main/scala/filodb.core/memstore/ratelimit/CardinalityStore.scala b/core/src/main/scala/filodb.core/memstore/ratelimit/CardinalityStore.scala new file mode 100644 index 0000000000..bc38b9e627 --- /dev/null +++ b/core/src/main/scala/filodb.core/memstore/ratelimit/CardinalityStore.scala @@ -0,0 +1,71 @@ +package filodb.core.memstore.ratelimit + +case class Cardinality(name: String, timeSeriesCount: Int, childrenCount: Int, childrenQuota: Int) + +/** + * + * Abstracts storage of cardinality for each shard prefix. + * + * Data model needs to represent a trie like data structure. + * For the Ws/Ns/Name shard key example, here is the trie structure: + * + *
+ * Root
+ * * myWs1
+ * ** myNs11
+ * *** myMetric111
+ * *** myMetric112
+ * ** myNs12
+ * *** myMetric121
+ * *** myMetric122
+ * * myWs2
+ * ** myNs21
+ * *** myMetric211
+ * *** myMetric212
+ * ** myNs22
+ * *** myMetric221
+ * *** myMetric222
+ * 
+ * + * The root to leaf path forms a full shard key. At each level in that path, we store + * the cardinality under that shard key prefix. + * + * The store also needs to be able to fetch immediate children of any node quickly. + * There can potentially be numerous nodes in the trie, so exhaustive tree-wide searches would + * be inefficient. So it is important that the store chosen is an implementation of some kind of tree + * like data structure and provides prefix search capability. + * + * Amount of memory used should be configurable. So it does not affect rest of the system. + * + * Implementations need to be local, fast and should not involve network I/O. + */ +trait CardinalityStore { + + /** + * This method will be called for each shard key prefix when a new time series is added + * to the index. + */ + def store(shardKeyPrefix: Seq[String], card: Cardinality): Unit + + /** + * Read existing cardinality value, if one does not exist return the zero value + * indicated + */ + def getOrZero(shardKeyPrefix: Seq[String], zero: Cardinality): Cardinality + + /** + * Remove entry from store. Need to call for each shard key prefix to fully remove shard key. + * Called when a time series is purged from the index. + */ + def remove(shardKeyPrefix: Seq[String]): Unit + + /** + * Fetch immediate children of the node for the given shard key prefix + */ + def scanChildren(shardKeyPrefix: Seq[String]): Seq[Cardinality] + + /** + * Close store. Data will be thrown away + */ + def close(): Unit +} diff --git a/core/src/main/scala/filodb.core/memstore/ratelimit/CardinalityTracker.scala b/core/src/main/scala/filodb.core/memstore/ratelimit/CardinalityTracker.scala new file mode 100644 index 0000000000..7524e64f25 --- /dev/null +++ b/core/src/main/scala/filodb.core/memstore/ratelimit/CardinalityTracker.scala @@ -0,0 +1,185 @@ +package filodb.core.memstore.ratelimit + +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer + +import com.typesafe.scalalogging.StrictLogging + +import filodb.core.DatasetRef + +case class QuotaReachedException(cannotSetShardKey: Seq[String], prefix: Seq[String], quota: Int) + extends RuntimeException + +case class CardinalityRecord(shard: Int, childName: String, timeSeriesCount: Int, + childrenCount: Int, childrenQuota: Int) + +/** + * Tracks cardinality (number of time series) under each shard key prefix. The shard key + * essentially comprises of the part key key/value pairs that determine which shard a time + * series goes into. + * + * For example if shard Key is Seq("myWs", "myNs", "myMetric") then cardinality of each prefix + * Seq(), Seq("myWs"), Seq("myWs, "myNs"), Seq("myWs", "myNs", "myMetric") would be tracked. + * + * Thus, the cardinality store can be represented as a Trie of shard key elements. For the above example, + * the trie would have 4 levels. Cardinality is tracked at each node of the trie. Both count of number + * of immediate children as well as number of time series under that level are tracked. + * + * This Cardinality Tracker can also enforce quotas. Quotas are set by invoking the setQuota method. + * While this tracker tracks both immediate children as well as total number of time series under each node, + * quota enforcement is for immediate children only. + * + * @param ref the dataset for which we track the cardinality + * @param shard the shard number + * @param shardKeyLen number of elements in the shard key + * @param defaultChildrenQuota the default quota at each level if no explicit quota is set + * @param store fast memory or disk based store where cardinality and quota can be read and written + * @param quotaExceededProtocol action to be taken when quota is breached + */ +class CardinalityTracker(ref: DatasetRef, + shard: Int, + shardKeyLen: Int, + defaultChildrenQuota: Seq[Int], + val store: CardinalityStore, + quotaExceededProtocol: QuotaExceededProtocol = NoActionQuotaProtocol) extends StrictLogging { + + require(defaultChildrenQuota.length == shardKeyLen + 1) + require(defaultChildrenQuota.forall(q => q > 0 && q < 2000000)) + logger.info(s"Initializing Cardinality Tracker for shard $shard with $defaultChildrenQuota") + + /** + * Call when a new time series with the given shard key has been added to the system. + * This will update the cardinality at each level within the trie. If quota is breached, + * QuotaReachedException will be thrown and quotaExceededProtocol will be invoked + * + * @param shardKey elements in the shard key of time series. For example: (ws, ns, name). Full shard key needed. + * @return current cardinality for each shard key prefix. There + * will be shardKeyLen + 1 items in the return value + */ + def incrementCount(shardKey: Seq[String]): Seq[Cardinality] = { + require(shardKey.length == shardKeyLen, "full shard key is needed") + + val toStore = ArrayBuffer[(Seq[String], Cardinality)]() + // first make sure there is no breach for any prefix + (0 to shardKey.length).foreach { i => + val prefix = shardKey.take(i) + val name = if (prefix.isEmpty) "" else prefix.last + val old = store.getOrZero(prefix, Cardinality(name, 0, 0, defaultChildrenQuota(i))) + val neu = old.copy(timeSeriesCount = old.timeSeriesCount + 1, + childrenCount = if (i == shardKeyLen) old.childrenCount + 1 else old.childrenCount) + if (i == shardKeyLen && neu.timeSeriesCount > neu.childrenQuota) { + quotaExceededProtocol.quotaExceeded(ref, shard, prefix, neu.childrenQuota) + throw QuotaReachedException(shardKey, prefix, neu.childrenQuota) + } + if (i > 0 && neu.timeSeriesCount == 1) { // parent's new child + val parent = toStore(i-1) + val neuParent = parent._2.copy(childrenCount = parent._2.childrenCount + 1) + toStore(i-1) = (parent._1, neuParent) + if (neuParent.childrenCount > neuParent.childrenQuota) { + quotaExceededProtocol.quotaExceeded(ref, shard, parent._1, neuParent.childrenQuota) + throw QuotaReachedException(shardKey, parent._1, neuParent.childrenQuota) + } + } + toStore += (prefix -> neu) + } + + toStore.map { case (prefix, neu) => + store.store(prefix, neu) + neu + } + } + + /** + * Fetch cardinality for given shard key or shard key prefix + * + * @param shardKeyPrefix zero or more elements that form a valid shard key prefix + */ + def getCardinality(shardKeyPrefix: Seq[String]): Cardinality = { + require(shardKeyPrefix.length <= shardKeyLen, s"Too many shard keys in $shardKeyPrefix - max $shardKeyLen") + val name = if (shardKeyPrefix.isEmpty) "" else shardKeyPrefix.last + store.getOrZero(shardKeyPrefix, Cardinality(name, 0, 0, defaultChildrenQuota(shardKeyPrefix.length))) + } + + /** + * Set quota for given shard key or shard key prefix + * + * @param shardKeyPrefix zero or more elements that form a valid shard key prefix + * @param childrenQuota maximum number of time series for this prefix + * @return current Cardinality for the prefix + */ + def setQuota(shardKeyPrefix: Seq[String], childrenQuota: Int): Cardinality = { + require(shardKeyPrefix.length <= shardKeyLen, s"Too many shard keys in $shardKeyPrefix - max $shardKeyLen") + require(childrenQuota > 0 && childrenQuota < 2000000, "Children quota invalid. Provide [1, 2000000)") + + logger.debug(s"Setting children quota for $shardKeyPrefix as $childrenQuota") + val name = if (shardKeyPrefix.isEmpty) "" else shardKeyPrefix.last + val old = store.getOrZero(shardKeyPrefix, Cardinality(name, 0, 0, defaultChildrenQuota(shardKeyPrefix.length))) + val neu = old.copy(childrenQuota = childrenQuota) + store.store(shardKeyPrefix, neu) + neu + } + + /** + * Call when an existing time series with the given shard key has been removed from the system. + * This will reduce the cardinality at each level within the trie. + * + * If cardinality reduces to 0, and the quota is the default quota, record will be removed from the store + * + * @param shardKey elements in the shard key of time series. + * For example: (ws, ns, name). Full shard key is needed. + * @return current cardinality for each shard key prefix. There + * will be shardKeyLen + 1 items in the return value + */ + def decrementCount(shardKey: Seq[String]): Seq[Cardinality] = { + require(shardKey.length == shardKeyLen, "full shard key is needed") + val toStore = (0 to shardKey.length).map { i => + val prefix = shardKey.take(i) + val old = store.getOrZero(prefix, Cardinality("", 0, 0, defaultChildrenQuota(i))) + if (old.timeSeriesCount == 0) + throw new IllegalArgumentException(s"$prefix count is already zero - cannot reduce further") + val neu = old.copy(timeSeriesCount = old.timeSeriesCount - 1) + (prefix, neu) + } + toStore.map { case (prefix, neu) => + val name = if (prefix.isEmpty) "" else prefix.last + if (neu == Cardinality(name, 0, 0, defaultChildrenQuota(prefix.length))) { + // node can be removed + store.remove(prefix) + } else { + store.store(prefix, neu) + } + neu + } + } + + /** + * Use this method to query the top-k cardinality consumers immediately + * under a provided shard key prefix + * + * @param k + * @param shardKeyPrefix zero or more elements that form a valid shard key prefix + * @return Top-K records, can the less than K if fewer children + */ + def topk(k: Int, shardKeyPrefix: Seq[String]): Seq[CardinalityRecord] = { + require(shardKeyPrefix.length <= shardKeyLen, s"Too many shard keys in $shardKeyPrefix - max $shardKeyLen") + implicit val ord = new Ordering[CardinalityRecord]() { + override def compare(x: CardinalityRecord, y: CardinalityRecord): Int = { + x.timeSeriesCount - y.timeSeriesCount + } + }.reverse + val heap = mutable.PriorityQueue[CardinalityRecord]() + store.scanChildren(shardKeyPrefix).foreach { card => + heap.enqueue( + CardinalityRecord(shard, card.name, + card.timeSeriesCount, + if (shardKeyPrefix.length == shardKeyLen - 1) card.timeSeriesCount else card.childrenCount, + card.childrenQuota)) + if (heap.size > k) heap.dequeue() + } + heap.toSeq + } + + def close(): Unit = { + store.close() + } +} diff --git a/core/src/main/scala/filodb.core/memstore/ratelimit/QuotaExceededProtocol.scala b/core/src/main/scala/filodb.core/memstore/ratelimit/QuotaExceededProtocol.scala new file mode 100644 index 0000000000..7bd01e72da --- /dev/null +++ b/core/src/main/scala/filodb.core/memstore/ratelimit/QuotaExceededProtocol.scala @@ -0,0 +1,33 @@ +package filodb.core.memstore.ratelimit + +import filodb.core.DatasetRef + +/** + * Allows for action to be taken when quota is breached + */ +trait QuotaExceededProtocol { + /** + * Invoked when quota is breached. + * + * Example actions that could be taken: + * 1. Send message to gateway to block ingestion of this shardKeyPrefix + * 2. Automatically increase quota if reasonable + * 3. Send a notification/alert to customers or operations + * + * Note that this can be invoked multiple times until either ingestion of invalid data stops + * or if quota is fixed. So implementations should ensure that actions are idempotent. + * + * @param ref dataset + * @param shardNum the shardNumber that breached the quota + * @param shardKeyPrefix the shardKeyPrefix for which quota was breached + * @param quota the actual quota number that was breached + */ + def quotaExceeded(ref: DatasetRef, shardNum: Int, shardKeyPrefix: Seq[String], quota: Int): Unit +} + +/** + * Default implementation which takes no action. + */ +object NoActionQuotaProtocol extends QuotaExceededProtocol { + def quotaExceeded(ref: DatasetRef, shardNum: Int, shardKeyPrefix: Seq[String], quota: Int): Unit = {} +} \ No newline at end of file diff --git a/core/src/main/scala/filodb.core/memstore/ratelimit/QuotaSource.scala b/core/src/main/scala/filodb.core/memstore/ratelimit/QuotaSource.scala new file mode 100644 index 0000000000..c524cd9319 --- /dev/null +++ b/core/src/main/scala/filodb.core/memstore/ratelimit/QuotaSource.scala @@ -0,0 +1,60 @@ +package filodb.core.memstore.ratelimit + +import com.typesafe.config.Config +import net.ceedubs.ficus.Ficus._ +import net.ceedubs.ficus.readers.ValueReader + +import filodb.core.DatasetRef + +case class QuotaRecord(shardKeyPrefix: Seq[String], quota: Int) + +/** + * Source of quotas for shard key prefixes. + */ +trait QuotaSource { + + /** + * Fetch all configured quotas. Invoked when a new Time Series Shard is bootstrapped. + * + * The quota represents number of immediate children allowed for the given + * shard key prefix within each shard. + */ + def getQuotas(dataset: DatasetRef): Iterator[QuotaRecord] + + /** + * Quota to use in case explicit quota record is not present. + * Return value is one item for each level of the tree. + * Hence number of items in the returned sequence should be + * shardKeyLen + 1 + */ + def getDefaults(dataset: DatasetRef): Seq[Int] +} + +/** + * QuotaSource implementation where static quota definitions are loaded from server configuration. + */ +class ConfigQuotaSource(filodbConfig: Config, shardKeyLen: Int) extends QuotaSource { + implicit val quotaReader: ValueReader[QuotaRecord] = ValueReader.relative { quotaConfig => + QuotaRecord(quotaConfig.as[Seq[String]]("shardKeyPrefix"), + quotaConfig.as[Int]("quota")) + } + + def getQuotas(dataset: DatasetRef): Iterator[QuotaRecord] = { + if (filodbConfig.hasPath(s"quotas.$dataset.custom")) { + filodbConfig.as[Seq[QuotaRecord]](s"quotas.$dataset.custom").iterator + } else { + Iterator.empty + } + } + + def getDefaults(dataset: DatasetRef): Seq[Int] = { + if (filodbConfig.hasPath(s"quotas.$dataset.custom")) { + val defaults = filodbConfig.as[Seq[Int]](s"quotas.$dataset.defaults") + require(defaults.length == shardKeyLen + 1, s"Quota defaults $defaults was not of length ${shardKeyLen + 1}") + defaults + } else { + val default = filodbConfig.as[Int]("quotas.default") + Seq.fill(shardKeyLen + 1)(default) + } + } +} \ No newline at end of file diff --git a/core/src/main/scala/filodb.core/memstore/ratelimit/RocksDbCardinalityStore.scala b/core/src/main/scala/filodb.core/memstore/ratelimit/RocksDbCardinalityStore.scala new file mode 100644 index 0000000000..364ec030b8 --- /dev/null +++ b/core/src/main/scala/filodb.core/memstore/ratelimit/RocksDbCardinalityStore.scala @@ -0,0 +1,251 @@ +package filodb.core.memstore.ratelimit + +import java.io.File + +import scala.collection.mutable.ArrayBuffer +import scala.concurrent.duration._ +import scala.reflect.io.Directory + +import com.esotericsoftware.kryo.{Kryo, Serializer} +import com.esotericsoftware.kryo.io.{Input, Output} +import com.typesafe.scalalogging.StrictLogging +import kamon.Kamon +import kamon.metric.MeasurementUnit +import kamon.tag.TagSet +import monix.reactive.Observable +import org.rocksdb._ +import spire.syntax.cfor._ + +import filodb.core.{DatasetRef, GlobalScheduler} +import filodb.memory.format.UnsafeUtils + +class CardinalitySerializer extends Serializer[Cardinality] { + def write(kryo: Kryo, output: Output, card: Cardinality): Unit = { + output.writeString(card.name) + output.writeInt(card.timeSeriesCount, true) + output.writeInt(card.childrenCount, true) + output.writeInt(card.childrenQuota, true) + } + + def read(kryo: Kryo, input: Input, t: Class[Cardinality]): Cardinality = { + Cardinality(input.readString(), input.readInt(true), input.readInt(true), input.readInt(true)) + } +} + +object RocksDbCardinalityStore { + private lazy val loadRocksDbLibrary = RocksDB.loadLibrary() + private val LastKeySeparator: Char = 0x1E + private val NotLastKeySeparator: Char = 0x1D + private val NotFound = UnsafeUtils.ZeroPointer.asInstanceOf[Array[Byte]] + + // ======= DB Tuning =========== + // not making them config intentionally since RocksDB tuning needs more care + private[ratelimit] val TOTAL_OFF_HEAP_SIZE = 32L << 20 // 32 MB + private[ratelimit] val LRU_CACHE_SIZE = 16L << 20 // 16 MB + private val BLOCK_SIZE = 4096L // 4 KB + private val NUM_WRITE_BUFFERS = 4 + private val WRITE_BUF_SIZE = 4L << 20 // 4 MB + +} + +class RocksDbCardinalityStore(ref: DatasetRef, shard: Int) extends CardinalityStore with StrictLogging { + + import RocksDbCardinalityStore._ + loadRocksDbLibrary + + // ======= DB Config =========== + private val cache = new LRUCache(LRU_CACHE_SIZE) + // caps total memory used by rocksdb memTables, blockCache + private val writeBufferManager = new WriteBufferManager(TOTAL_OFF_HEAP_SIZE, cache) + private val options = { + val opts = new Options().setCreateIfMissing(true) + + val tableConfig = new BlockBasedTableConfig() + tableConfig.setBlockCache(cache) + tableConfig.setCacheIndexAndFilterBlocks(true) + tableConfig.setCacheIndexAndFilterBlocksWithHighPriority(true) + tableConfig.setPinTopLevelIndexAndFilter(true) + tableConfig.setBlockSize(BLOCK_SIZE) + opts.setTableFormatConfig(tableConfig) + + opts.setWriteBufferManager(writeBufferManager) + opts.setMaxWriteBufferNumber(NUM_WRITE_BUFFERS) // number of memtables + opts.setWriteBufferSize(WRITE_BUF_SIZE) // size of each memtable + + opts + } + + private val baseDir = new File(System.getProperty("java.io.tmpdir")) + private val baseName = s"cardStore-$ref-$shard-${System.currentTimeMillis()}" + private val dbDirInTmp = new File(baseDir, baseName) + private val db = RocksDB.open(options, dbDirInTmp.getAbsolutePath) + logger.info(s"Opening new Cardinality DB for shard=$shard dataset=$ref at ${dbDirInTmp.getAbsolutePath}") + + private val kryo = new ThreadLocal[Kryo]() { + override def initialValue(): Kryo = { + val k = new Kryo() + k.addDefaultSerializer(classOf[Cardinality], classOf[CardinalitySerializer]) + k + } + } + + // ======= Metrics =========== + private val tags = Map("shard" -> shard.toString, "dataset" -> ref.toString) + private val diskSpaceUsedMetric = Kamon.gauge("card-store-disk-space-used", MeasurementUnit.information.bytes) + .withTags(TagSet.from(tags)) + private val memoryUsedMetric = Kamon.gauge("card-store-offheap-mem-used") + .withTags(TagSet.from(tags)) + private val compactionBytesPendingMetric = Kamon.gauge("card-store-compaction-pending", + MeasurementUnit.information.bytes).withTags(TagSet.from(tags)) + private val numRunningCompactionsMetric = Kamon.gauge("card-store-num-running-compactions") + .withTags(TagSet.from(tags)) + private val numKeysMetric = Kamon.gauge("card-store-est-num-keys") + .withTags(TagSet.from(tags)) + + private val metricsReporter = Observable.interval(1.minute) + .onErrorRestart(Int.MaxValue) + .foreach(_ => updateMetrics())(GlobalScheduler.globalImplicitScheduler) + + var lastMetricsReportTime = 0L + private def updateMetrics(): Unit = { + val now = System.currentTimeMillis() + // dump DB stats every 5 minutes + if (now - lastMetricsReportTime > 1000 * 60 * 5 ) { + logger.info(s"Card Store Stats dataset=$ref shard=$shard $statsAsString") + lastMetricsReportTime = now + } + diskSpaceUsedMetric.update(diskSpaceUsed) + numKeysMetric.update(estimatedNumKeys) + memoryUsedMetric.update(memTablesSize + blockCacheSize + tableReadersSize) + compactionBytesPendingMetric.update(compactionBytesPending) + numRunningCompactionsMetric.update(numRunningCompactions) + } + + // List of all RocksDB properties at https://github.com/facebook/rocksdb/blob/6.12.fb/include/rocksdb/db.h#L720 + def statsAsString: String = db.getProperty("rocksdb.stats") + def estimatedNumKeys: Long = db.getLongProperty("rocksdb.estimate-num-keys") + // Returns the total size, in bytes, of all the SST files. + // WAL files are not included in the calculation. + def diskSpaceUsed: Long = db.getLongProperty("rocksdb.total-sst-files-size") + def memTablesSize: Long = db.getLongProperty("rocksdb.size-all-mem-tables") + def blockCacheSize: Long = db.getLongProperty("rocksdb.block-cache-usage") + def tableReadersSize: Long = db.getLongProperty("rocksdb.estimate-table-readers-mem") + def compactionBytesPending: Long = db.getLongProperty("rocksdb.estimate-pending-compaction-bytes") + def numRunningCompactions: Long = db.getLongProperty("rocksdb.num-running-compactions") + // consider compaction-pending yes/no + + /** + * In order to enable quick prefix search, we formulate string based keys to the RocksDB + * key-value store. + * + * For example, here is the list of rocksDb keys for a few shard keys. {LastKeySeparator} and + * {NotLastKeySeparator} are special characters chosen as separator char between shard key elements. + * {LastKeySeparator} is used just prior to last shard key element. {NotLastKeySeparator} is used otherwise. + * This model helps with fast prefix searches to do top-k scans. + * + * BTW, Remove quote chars from actual string key. + * They are there just to emphasize the shard key element in the string. "" represents the root. + * + *
+   * ""
+   * ""{LastKeySeparator}"myWs1"
+   * ""{LastKeySeparator}"myWs2"
+   * ""{NotLastKeySeparator}"myWs1"{LastKeySeparator}"myNs11"
+   * ""{NotLastKeySeparator}"myWs1"{LastKeySeparator}"myNs12"
+   * ""{NotLastKeySeparator}"myWs2"{LastKeySeparator}"myNs21"
+   * ""{NotLastKeySeparator}"myWs2"{LastKeySeparator}"myNs22"
+   * ""{NotLastKeySeparator}"myWs1"{NotLastKeySeparator}"myNs11"{LastKeySeparator}"heap_usage"
+   * ""{NotLastKeySeparator}"myWs1"{NotLastKeySeparator}"myNs11"{LastKeySeparator}"cpu_usage"
+   * ""{NotLastKeySeparator}"myWs1"{NotLastKeySeparator}"myNs11"{LastKeySeparator}"network_usage"
+   * 
+ * + * In the above tree, we simply do a prefix search on
 ""{NotLastKeySeparator}"myWs1"{LastKeySeparator} 
+ * to get namespaces under workspace myWs1. + * + * @param shardKeyPrefix Zero or more elements that make up shard key prefix + * @param prefixSearch If true, returns key that can be used to perform prefix search to + * fetch immediate children in trie. Use false to fetch one specific node. + * @return string key to use to perform reads and writes of entries into RocksDB + */ + private def toStringKey(shardKeyPrefix: Seq[String], prefixSearch: Boolean): String = { + import RocksDbCardinalityStore._ + if (shardKeyPrefix.isEmpty) { + if (prefixSearch) LastKeySeparator.toString else "" + } else { + val b = new StringBuilder + cforRange { 0 until shardKeyPrefix.length - 1 } { i => + b.append(NotLastKeySeparator) + b.append(shardKeyPrefix(i)) + } + if (prefixSearch) { + b.append(NotLastKeySeparator) + b.append(shardKeyPrefix.last) + b.append(LastKeySeparator) + } else { + b.append(LastKeySeparator) + b.append(shardKeyPrefix.last) + } + b.toString() + } + } + + private def cardinalityToBytes(card: Cardinality): Array[Byte] = { + val out = new Output(500) + kryo.get().writeObject(out, card) + out.close() + out.toBytes + } + + private def bytesToCardinality(bytes: Array[Byte]): Cardinality = { + val inp = new Input(bytes) + val c = kryo.get().readObject(inp, classOf[Cardinality]) + inp.close() + c + } + + override def store(shardKeyPrefix: Seq[String], card: Cardinality): Unit = { + val key = toStringKey(shardKeyPrefix, false).getBytes() + logger.debug(s"Storing shard=$shard dataset=$ref ${new String(key)} with $card") + db.put(key, cardinalityToBytes(card)) + } + + def getOrZero(shardKeyPrefix: Seq[String], zero: Cardinality): Cardinality = { + val value = db.get(toStringKey(shardKeyPrefix, false).getBytes()) + if (value == NotFound) zero else bytesToCardinality(value) + } + + override def remove(shardKeyPrefix: Seq[String]): Unit = { + db.delete(toStringKey(shardKeyPrefix, false).getBytes()) + } + + override def scanChildren(shardKeyPrefix: Seq[String]): Seq[Cardinality] = { + val it = db.newIterator() + val searchPrefix = toStringKey(shardKeyPrefix, true) + logger.debug(s"Scanning shard=$shard dataset=$ref ${new String(searchPrefix)}") + it.seek(searchPrefix.getBytes()) + val buf = ArrayBuffer[Cardinality]() + import scala.util.control.Breaks._ + + breakable { + while (it.isValid()) { + val key = new String(it.key()) + if (key.startsWith(searchPrefix)) { + buf += bytesToCardinality(it.value()) + } else break // dont continue beyond valid results + it.next() + } + } + buf + } + + def close(): Unit = { + db.cancelAllBackgroundWork(true) + db.close() + writeBufferManager.close() + cache.close() + options.close() + val directory = new Directory(dbDirInTmp) + directory.deleteRecursively() + metricsReporter.cancel() + } +} diff --git a/core/src/main/scala/filodb.core/query/QueryContext.scala b/core/src/main/scala/filodb.core/query/QueryContext.scala index f15babbf3d..138308c15e 100644 --- a/core/src/main/scala/filodb.core/query/QueryContext.scala +++ b/core/src/main/scala/filodb.core/query/QueryContext.scala @@ -11,27 +11,40 @@ trait TsdbQueryParams * This class provides PromQl query parameters * Config has routing parameters */ -case class PromQlQueryParams(promQl: String, startSecs: Long, stepSecs: Long, endSecs: Long, spread: Option[Int] = None, - remoteQueryPath: Option[String] = None, processFailure: Boolean = true, - processMultiPartition: Boolean = false, verbose: Boolean = false) extends TsdbQueryParams +case class PromQlQueryParams(promQl: String, startSecs: Long, stepSecs: Long, endSecs: Long , remoteQueryPath: + Option[String] = None, verbose: Boolean = false) extends TsdbQueryParams + case object UnavailablePromQlQueryParams extends TsdbQueryParams +case class PlannerParams(applicationId: String = "filodb", + spread: Option[Int] = None, + spreadOverride: Option[SpreadProvider] = None, + shardOverrides: Option[Seq[Int]] = None, + queryTimeoutMillis: Int = 30000, + sampleLimit: Int = 1000000, + groupByCardLimit: Int = 100000, + joinQueryCardLimit: Int = 100000, + skipAggregatePresent: Boolean = false, + processFailure: Boolean = true, + processMultiPartition: Boolean = false) +object PlannerParams { + def apply(constSpread: Option[SpreadProvider], sampleLimit: Int): PlannerParams = + PlannerParams(spreadOverride = constSpread, sampleLimit = sampleLimit) +} /** * This class provides general query processing parameters */ final case class QueryContext(origQueryParams: TsdbQueryParams = UnavailablePromQlQueryParams, - spreadOverride: Option[SpreadProvider] = None, - queryTimeoutMillis: Int = 30000, - sampleLimit: Int = 1000000, - groupByCardLimit: Int = 100000, - joinQueryCardLimit: Int = 100000, - shardOverrides: Option[Seq[Int]] = None, queryId: String = UUID.randomUUID().toString, - submitTime: Long = System.currentTimeMillis()) + submitTime: Long = System.currentTimeMillis(), + plannerParams: PlannerParams = PlannerParams()) object QueryContext { def apply(constSpread: Option[SpreadProvider], sampleLimit: Int): QueryContext = - QueryContext(spreadOverride = constSpread, sampleLimit = sampleLimit) + QueryContext(plannerParams = PlannerParams(constSpread, sampleLimit)) + + def apply(queryParams: TsdbQueryParams, constSpread: Option[SpreadProvider]): QueryContext = + QueryContext(origQueryParams = queryParams, plannerParams = PlannerParams(spreadOverride = constSpread)) /** * Creates a spreadFunc that looks for a particular filter with keyName Equals a value, and then maps values diff --git a/core/src/main/scala/filodb.core/query/RangeVector.scala b/core/src/main/scala/filodb.core/query/RangeVector.scala index 9bc146ae5b..e2ccadd790 100644 --- a/core/src/main/scala/filodb.core/query/RangeVector.scala +++ b/core/src/main/scala/filodb.core/query/RangeVector.scala @@ -316,12 +316,13 @@ object SerializedRangeVector extends StrictLogging { // scalastyle:off null def apply(rv: RangeVector, builder: RecordBuilder, - schema: RecordSchema): SerializedRangeVector = { + schema: RecordSchema, + execPlan: String): SerializedRangeVector = { var numRows = 0 val oldContainerOpt = builder.currentContainer val startRecordNo = oldContainerOpt.map(_.numRecords).getOrElse(0) try { - ChunkMap.validateNoSharedLocks() + ChunkMap.validateNoSharedLocks(execPlan) val rows = rv.rows while (rows.hasNext) { numRows += 1 @@ -349,7 +350,7 @@ object SerializedRangeVector extends StrictLogging { */ def apply(rv: RangeVector, cols: Seq[ColumnInfo]): SerializedRangeVector = { val schema = toSchema(cols) - apply(rv, newBuilder(), schema) + apply(rv, newBuilder(), schema, "Test-Only-Plan") } // TODO: make this configurable.... diff --git a/core/src/main/scala/filodb.core/query/ResultTypes.scala b/core/src/main/scala/filodb.core/query/ResultTypes.scala index daf44f9920..602235c1b0 100644 --- a/core/src/main/scala/filodb.core/query/ResultTypes.scala +++ b/core/src/main/scala/filodb.core/query/ResultTypes.scala @@ -51,6 +51,8 @@ final case class ResultSchema(columns: Seq[ColumnInfo], numRowKeyColumns: Int, def isHistDouble: Boolean = columns.length == 3 && columns(1).colType == HistogramColumn && columns(2).colType == DoubleColumn def isHistogram: Boolean = columns.length == 2 && columns(1).colType == HistogramColumn + def isAvgAggregator: Boolean = columns.length == 3 && columns(2).name.equals("count") + def isStdValAggregator: Boolean = columns.length == 4 && columns(2).name.equals("mean") def hasSameColumnsAs(other: ResultSchema): Boolean = { // exclude fixedVectorLen & colIDs diff --git a/core/src/main/scala/filodb.core/store/ChunkSetInfo.scala b/core/src/main/scala/filodb.core/store/ChunkSetInfo.scala index 96bde268c0..8c8065033a 100644 --- a/core/src/main/scala/filodb.core/store/ChunkSetInfo.scala +++ b/core/src/main/scala/filodb.core/store/ChunkSetInfo.scala @@ -421,7 +421,7 @@ extends Iterator[ChunkSetInfoReader] { while (curWindowEnd > lastEndTime && infos.hasNext) { val next = infos.nextInfoReader val queryTimeElapsed = System.currentTimeMillis() - queryContext.submitTime - if (queryTimeElapsed >= queryContext.queryTimeoutMillis) + if (queryTimeElapsed >= queryContext.plannerParams.queryTimeoutMillis) throw QueryTimeoutException(queryTimeElapsed, this.getClass.getName) // Add if next chunkset is within window and not empty. Otherwise keep going diff --git a/core/src/main/scala/filodb.core/store/ChunkSource.scala b/core/src/main/scala/filodb.core/store/ChunkSource.scala index 502f10499a..d173b161f1 100644 --- a/core/src/main/scala/filodb.core/store/ChunkSource.scala +++ b/core/src/main/scala/filodb.core/store/ChunkSource.scala @@ -9,6 +9,7 @@ import monix.reactive.Observable import filodb.core._ import filodb.core.memstore.{PartLookupResult, SchemaMismatch, TimeSeriesShard} +import filodb.core.memstore.ratelimit.CardinalityRecord import filodb.core.metadata.{Schema, Schemas} import filodb.core.query._ @@ -167,6 +168,9 @@ trait ChunkSource extends RawChunkSource with StrictLogging { RawDataRangeVector(key, partition, lookupRes.chunkMethod, ids) } } + + def topKCardinality(ref: DatasetRef, shard: Seq[Int], shardKeyPrefix: Seq[String], k: Int): Seq[CardinalityRecord] + } /** diff --git a/core/src/main/scala/filodb.core/store/IngestionConfig.scala b/core/src/main/scala/filodb.core/store/IngestionConfig.scala index 6deedbd3d2..f5c745f7f6 100644 --- a/core/src/main/scala/filodb.core/store/IngestionConfig.scala +++ b/core/src/main/scala/filodb.core/store/IngestionConfig.scala @@ -10,6 +10,7 @@ import filodb.core.{DatasetRef, IngestionKeys} import filodb.core.downsample.DownsampleConfig final case class StoreConfig(flushInterval: FiniteDuration, + timeAlignedChunksEnabled: Boolean, diskTTLSeconds: Int, demandPagedRetentionPeriod: FiniteDuration, maxChunksSize: Int, @@ -37,10 +38,12 @@ final case class StoreConfig(flushInterval: FiniteDuration, ensureHeadroomPercent: Double, // filters on ingested records to log in detail traceFilters: Map[String, String], - maxDataPerShardQuery: Long) { + maxDataPerShardQuery: Long, + meteringEnabled: Boolean) { import collection.JavaConverters._ def toConfig: Config = ConfigFactory.parseMap(Map("flush-interval" -> (flushInterval.toSeconds + "s"), + "time-aligned-chunks-enabled" -> timeAlignedChunksEnabled, "disk-time-to-live" -> (diskTTLSeconds + "s"), "demand-paged-chunk-retention-period" -> (demandPagedRetentionPeriod.toSeconds + "s"), "max-chunks-size" -> maxChunksSize, @@ -61,7 +64,8 @@ final case class StoreConfig(flushInterval: FiniteDuration, "demand-paging-enabled" -> demandPagingEnabled, "max-data-per-shard-query" -> maxDataPerShardQuery, "evicted-pk-bloom-filter-capacity" -> evictedPkBfCapacity, - "ensure-headroom-percent" -> ensureHeadroomPercent).asJava) + "ensure-headroom-percent" -> ensureHeadroomPercent, + "metering-enabled" -> meteringEnabled).asJava) } final case class AssignShardConfig(address: String, shardList: Seq[Int]) @@ -97,16 +101,27 @@ object StoreConfig { |evicted-pk-bloom-filter-capacity = 5000000 |ensure-headroom-percent = 5.0 |trace-filters = {} + |metering-enabled = false + |time-aligned-chunks-enabled = false |""".stripMargin) /** Pass in the config inside the store {} */ def apply(storeConfig: Config): StoreConfig = { val config = storeConfig.withFallback(defaults) val flushInterval = config.as[FiniteDuration]("flush-interval") + + // switch buffers and create chunk when current sample's timestamp crosses flush boundary. + // e.g. for a flush-interval of 1hour, if new sample falls in different hour than last sample, then switch buffers + // and create chunk. This helps in aligning chunks across Active/Active HA clusters and facilitates chunk migration + // between the clusters during disaster recovery. + // Note: Enabling this might result into creation of smaller suboptimal chunks. + val timeAlignedChunksEnabled = config.getBoolean("time-aligned-chunks-enabled") + // maxChunkTime should atleast be length of flush interval to accommodate all data within one chunk. // better to be slightly greater so if more samples arrive within that flush period, two chunks are not created. val fallbackMaxChunkTime = (flushInterval.toMillis * 1.1).toLong.millis val maxChunkTime = config.as[Option[FiniteDuration]]("max-chunk-time").getOrElse(fallbackMaxChunkTime) StoreConfig(flushInterval, + timeAlignedChunksEnabled, config.as[FiniteDuration]("disk-time-to-live").toSeconds.toInt, config.as[FiniteDuration]("demand-paged-chunk-retention-period"), config.getInt("max-chunks-size"), @@ -128,7 +143,8 @@ object StoreConfig { config.getInt("evicted-pk-bloom-filter-capacity"), config.getDouble("ensure-headroom-percent"), config.as[Map[String, String]]("trace-filters"), - config.getMemorySize("max-data-per-shard-query").toBytes) + config.getMemorySize("max-data-per-shard-query").toBytes, + config.getBoolean("metering-enabled")) } } diff --git a/core/src/test/resources/application_test.conf b/core/src/test/resources/application_test.conf index 7699534503..62d17ce680 100644 --- a/core/src/test/resources/application_test.conf +++ b/core/src/test/resources/application_test.conf @@ -30,6 +30,11 @@ filodb { shardmap-publish-frequency = 1s } + quotas { + default = 1000000 + } + + columnstore { # Number of cache entries for the table cache tablecache-size = 50 diff --git a/core/src/test/scala/filodb.core/TestData.scala b/core/src/test/scala/filodb.core/TestData.scala index 2552a4f7a8..f3b2a759ad 100644 --- a/core/src/test/scala/filodb.core/TestData.scala +++ b/core/src/test/scala/filodb.core/TestData.scala @@ -320,7 +320,8 @@ object MachineMetricsData { } } - val dataset2 = Dataset("metrics2", Seq("series:string", "tags:map"), columns) + val dataset2 = Dataset("metrics2", Seq("series:string", "tags:map"), columns, + options = DatasetOptions(shardKeyColumns = Seq("_ws_", "_ns_", "_metric_"), "_metric_")) val schema2 = dataset2.schema def withMap(data: Stream[Seq[Any]], n: Int = 5, extraTags: UTF8Map = Map.empty): Stream[Seq[Any]] = @@ -339,7 +340,7 @@ object MachineMetricsData { val histDataset = Dataset("histogram", Seq("metric:string", "tags:map"), Seq("timestamp:ts", "count:long", "sum:long", "h:hist:counter=false"), - DatasetOptions.DefaultOptions.copy(metricColumn = "metric")) + options = DatasetOptions(shardKeyColumns = Seq("_ws_", "_ns_", "metric"), "metric")) var histBucketScheme: bv.HistogramBuckets = _ def linearHistSeries(startTs: Long = 100000L, numSeries: Int = 10, timeStep: Int = 1000, numBuckets: Int = 8, @@ -398,9 +399,9 @@ object MachineMetricsData { val histPartKey = histKeyBuilder.partKeyFromObjects(histDataset.schema, "request-latency", extraTags) val blockStore = new PageAlignedBlockManager(100 * 1024 * 1024, new MemoryStats(Map("test"-> "test")), null, 16) - val histIngestBH = new BlockMemFactory(blockStore, None, histDataset.schema.data.blockMetaSize, + val histIngestBH = new BlockMemFactory(blockStore, histDataset.schema.data.blockMetaSize, dummyContext, true) - val histMaxBH = new BlockMemFactory(blockStore, None, histMaxDS.schema.data.blockMetaSize, + val histMaxBH = new BlockMemFactory(blockStore, histMaxDS.schema.data.blockMetaSize, dummyContext, true) private val histBufferPool = new WriteBufferPool(TestData.nativeMem, histDataset.schema.data, TestData.storeConf) @@ -411,7 +412,8 @@ object MachineMetricsData { val histData = linearHistSeries(startTS, 1, pubFreq.toInt, numBuckets, infBucket).take(numSamples) val container = records(ds, histData).records val part = TimeSeriesPartitionSpec.makePart(0, ds, partKey=histPartKey, bufferPool=pool) - container.iterate(ds.ingestionSchema).foreach { row => part.ingest(0, row, histIngestBH, 1.hour.toMillis) } + container.iterate(ds.ingestionSchema).foreach { row => part.ingest(0, row, histIngestBH, + false, Option.empty, 1.hour.toMillis) } // Now flush and ingest the rest to ensure two separate chunks part.switchBuffers(histIngestBH, encode = true) (histData, RawDataRangeVector(null, part, AllChunkScan, Array(0, 3))) // select timestamp and histogram columns only @@ -425,7 +427,8 @@ object MachineMetricsData { val histData = histMax(linearHistSeries(startTS, 1, pubFreq.toInt, numBuckets)).take(numSamples) val container = records(histMaxDS, histData).records val part = TimeSeriesPartitionSpec.makePart(0, histMaxDS, partKey=histPartKey, bufferPool=histMaxBP) - container.iterate(histMaxDS.ingestionSchema).foreach { row => part.ingest(0, row, histMaxBH, 1.hour.toMillis) } + container.iterate(histMaxDS.ingestionSchema).foreach { row => part.ingest(0, row, histMaxBH, false, + Option.empty, 1.hour.toMillis) } // Now flush and ingest the rest to ensure two separate chunks part.switchBuffers(histMaxBH, encode = true) // Select timestamp, hist, max diff --git a/core/src/test/scala/filodb.core/downsample/ShardDownsamplerSpec.scala b/core/src/test/scala/filodb.core/downsample/ShardDownsamplerSpec.scala index bf0cf8fc85..762ec46506 100644 --- a/core/src/test/scala/filodb.core/downsample/ShardDownsamplerSpec.scala +++ b/core/src/test/scala/filodb.core/downsample/ShardDownsamplerSpec.scala @@ -48,7 +48,7 @@ class ShardDownsamplerSpec extends AnyFunSpec with Matchers with BeforeAndAfterA val customSchema = customDataset.schema private val blockStore = MMD.blockStore - protected val ingestBlockHolder = new BlockMemFactory(blockStore, None, promDataset.schema.data.blockMetaSize, + protected val ingestBlockHolder = new BlockMemFactory(blockStore, promDataset.schema.data.blockMetaSize, MMD.dummyContext, true) val storeConf = TestData.storeConf.copy(maxChunksSize = 200) @@ -74,7 +74,7 @@ class ShardDownsamplerSpec extends AnyFunSpec with Matchers with BeforeAndAfterA def timeValueRV(tuples: Seq[(Long, Double)]): RawDataRangeVector = { val part = TimeSeriesPartitionSpec.makePart(0, promDataset, partKeyOffset, bufferPool = tsBufferPool) val readers = tuples.map { case (ts, d) => TupleRowReader((Some(ts), Some(d))) } - readers.foreach { row => part.ingest(0, row, ingestBlockHolder) } + readers.foreach { row => part.ingest(0, row, ingestBlockHolder, false, Option.empty) } // Now flush and ingest the rest to ensure two separate chunks part.switchBuffers(ingestBlockHolder, encode = true) // part.encodeAndReleaseBuffers(ingestBlockHolder) diff --git a/core/src/test/scala/filodb.core/memstore/DemandPagedChunkStoreSpec.scala b/core/src/test/scala/filodb.core/memstore/DemandPagedChunkStoreSpec.scala index 6943defe80..703c80bbcd 100644 --- a/core/src/test/scala/filodb.core/memstore/DemandPagedChunkStoreSpec.scala +++ b/core/src/test/scala/filodb.core/memstore/DemandPagedChunkStoreSpec.scala @@ -58,13 +58,7 @@ class DemandPagedChunkStoreSpec extends AnyFunSpec with AsyncTest { tsPartition.numChunks shouldEqual 10 // write buffers + 9 chunks above } - pageManager.numTimeOrderedBlocks should be > 1 pageManager.numFreeBlocks should be >= (initFreeBlocks - 12) - val buckets = pageManager.timeBuckets - buckets.foreach { b => pageManager.hasTimeBucket(b) shouldEqual true } - - // Now, reclaim four time buckets, even if they are not full - pageManager.markBucketedBlocksReclaimable(buckets(4)) // try and ODP more data. Load older data than chunk retention, should still be able to load val data2 = linearMultiSeries(start - 2.hours.toMillis, timeStep=100000).take(20) diff --git a/core/src/test/scala/filodb.core/memstore/PartitionSetSpec.scala b/core/src/test/scala/filodb.core/memstore/PartitionSetSpec.scala index f267896384..eec350fbf2 100644 --- a/core/src/test/scala/filodb.core/memstore/PartitionSetSpec.scala +++ b/core/src/test/scala/filodb.core/memstore/PartitionSetSpec.scala @@ -32,7 +32,7 @@ class PartitionSetSpec extends MemFactoryCleanupTest with ScalaFutures { private val blockStore = new PageAlignedBlockManager(100 * 1024 * 1024, new MemoryStats(Map("test"-> "test")), reclaimer, 1) protected val bufferPool = new WriteBufferPool(memFactory, dataset2.schema.data, TestData.storeConf) - private val ingestBlockHolder = new BlockMemFactory(blockStore, None, dataset2.schema.data.blockMetaSize, + private val ingestBlockHolder = new BlockMemFactory(blockStore, dataset2.schema.data.blockMetaSize, dummyContext, true) val builder = new RecordBuilder(memFactory) diff --git a/core/src/test/scala/filodb.core/memstore/TimeSeriesPartitionSpec.scala b/core/src/test/scala/filodb.core/memstore/TimeSeriesPartitionSpec.scala index e5f6e5e851..160054cdc8 100644 --- a/core/src/test/scala/filodb.core/memstore/TimeSeriesPartitionSpec.scala +++ b/core/src/test/scala/filodb.core/memstore/TimeSeriesPartitionSpec.scala @@ -1,27 +1,30 @@ package filodb.core.memstore import scala.concurrent.Future +import scala.concurrent.duration._ import com.typesafe.config.ConfigFactory import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll} import org.scalatest.concurrent.ScalaFutures +import org.scalatest.funspec.AnyFunSpec +import org.scalatest.matchers.should.Matchers import org.scalatest.time.{Millis, Seconds, Span} import filodb.core._ import filodb.core.metadata.Dataset import filodb.core.store._ import filodb.memory._ -import filodb.memory.format.UnsafeUtils -import org.scalatest.funspec.AnyFunSpec -import org.scalatest.matchers.should.Matchers +import filodb.memory.format.{RowReader, TupleRowReader, UnsafeUtils} object TimeSeriesPartitionSpec { - import MachineMetricsData._ import BinaryRegion.NativePointer + import MachineMetricsData._ val memFactory = new NativeMemoryManager(50 * 1024 * 1024) val maxChunkSize = TestData.storeConf.maxChunksSize + private val flushIntervalMillis = Option(TestData.storeConf.flushInterval.toMillis) + private val timeAlignedChunksEnabled = TestData.storeConf.timeAlignedChunksEnabled protected val myBufferPool = new WriteBufferPool(memFactory, schema1.data, TestData.storeConf) def makePart(partNo: Int, dataset: Dataset, @@ -71,7 +74,7 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { private val blockStore = new PageAlignedBlockManager(200 * 1024 * 1024, new MemoryStats(Map("test"-> "test")), reclaimer, 1) - protected val ingestBlockHolder = new BlockMemFactory(blockStore, None, schema1.data.blockMetaSize, + protected val ingestBlockHolder = new BlockMemFactory(blockStore, schema1.data.blockMetaSize, dummyContext, true) before { @@ -81,7 +84,8 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { it("should be able to read immediately after ingesting one row") { part = makePart(0, dataset1) val data = singleSeriesReaders().take(5) - part.ingest(0, data(0), ingestBlockHolder) // just one row + part.ingest(0, data(0), ingestBlockHolder, timeAlignedChunksEnabled, + flushIntervalMillis = flushIntervalMillis) // just one row part.numChunks shouldEqual 1 part.appendingChunkLen shouldEqual 1 part.unflushedChunksets shouldEqual 1 @@ -95,7 +99,8 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { val data = singleSeriesReaders().take(11) val minData = data.map(_.getDouble(1)) val initTS = data(0).getLong(0) - data.take(10).zipWithIndex.foreach { case (r, i) => part.ingest(0, r, ingestBlockHolder) } + data.take(10).zipWithIndex.foreach { case (r, i) => part.ingest(0, r, ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled) } val origPoolSize = myBufferPool.poolSize @@ -114,10 +119,11 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { val data1 = part.timeRangeRows(AllChunkScan, Array(1)).map(_.getDouble(0)).toBuffer data1 shouldEqual (minData take 10) - val blockHolder = new BlockMemFactory(blockStore, None, schema1.data.blockMetaSize, dummyContext) + val blockHolder = new BlockMemFactory(blockStore, schema1.data.blockMetaSize, dummyContext) // Task needs to fully iterate over the chunks, to release the shared lock. val flushFut = Future(part.makeFlushChunks(blockHolder).toBuffer) - data.drop(10).zipWithIndex.foreach { case (r, i) => part.ingest(0, r, ingestBlockHolder) } + data.drop(10).zipWithIndex.foreach { case (r, i) => part.ingest(0, r, ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled) } val chunkSets = flushFut.futureValue // After flush, the old writebuffers should be returned to pool, but new one allocated for ingesting @@ -146,36 +152,73 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { it("should enforce user time length in each chunk") { part = makePart(0, dataset1) // user time maximum is not enforced, so just one chunk - singleSeriesReaders().take(35).foreach { r => part.ingest(0, r, ingestBlockHolder, Long.MaxValue) } + singleSeriesReaders().take(35).foreach { r => part.ingest(0, r, ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled) } part.numChunks shouldEqual 1 part = makePart(0, dataset1) // 11 samples per chunk since maxChunkTime is 10 seconds - singleSeriesReaders().take(33).foreach { r => part.ingest(0, r, ingestBlockHolder, 10000) } + singleSeriesReaders().take(33).foreach { r => part.ingest(0, r, ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled, maxChunkTime = 10000) } part.numChunks shouldEqual 3 part = makePart(0, dataset1) // 11 samples per chunk since maxChunkTime is 10 seconds - singleSeriesReaders().take(34).foreach { r => part.ingest(0, r, ingestBlockHolder, 10000) } + singleSeriesReaders().take(34).foreach { r => part.ingest(0, r, ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled, maxChunkTime = 10000) } part.numChunks shouldEqual 4 } + it("should enforce write-buffer-switching/chunk-creation at flush boundary when the functionality is enabled") { + val currentTIme = System.currentTimeMillis() + def timeAlignedSeriesReaders(): Stream[RowReader] = + singleSeriesData(initTs = currentTIme - currentTIme%(1 minutes).toMillis).map(TupleRowReader) + part = makePart(0, dataset1) + // chunk creation on crossing flush boundary is not enforced, so will create 1 chunk + timeAlignedSeriesReaders().take(70).foreach { r => part.ingest(0, r, ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled) } + part.numChunks shouldEqual 1 + + // chunk creation on crossing flush boundary is enabled with flushInterval of 1 min. + // Each chunk will contain 60 samples. + part = makePart(0, dataset1) + timeAlignedSeriesReaders().take(61).foreach { r => part.ingest(0, r, ingestBlockHolder, + flushIntervalMillis = Option((1 minutes).toMillis), createChunkAtFlushBoundary = true) } + part.numChunks shouldEqual 2 // ingesting 61 samples results into 2 chunks + + // chunk creation on crossing flush boundary is enabled with flushInterval of 1 min. + // Each chunk will contain 60 samples, + part = makePart(0, dataset1) + timeAlignedSeriesReaders().take(180).foreach { r => part.ingest(0, r, ingestBlockHolder, + flushIntervalMillis = Option((1 minutes).toMillis), createChunkAtFlushBoundary = true, maxChunkTime = Long.MaxValue) } + part.numChunks shouldEqual 3 // ingesting 180 samples results into 2 chunks + + // chunk creation on crossing flush boundary is enabled with flushInterval of 1 min. + // Each chunk will contain 60 samples, + part = makePart(0, dataset1) + timeAlignedSeriesReaders().take(200).foreach { r => part.ingest(0, r, ingestBlockHolder, + flushIntervalMillis = Option((1 minutes).toMillis), createChunkAtFlushBoundary = true, maxChunkTime = Long.MaxValue) } + part.numChunks shouldEqual 4 // ingesting 200 samples results into 4 chunks + } + it("should be able to read a time range of ingested data") { part = makePart(0, dataset1) val data = singleSeriesReaders().take(11) val initTS = data(0).getLong(0) val appendingTS = data.last.getLong(0) val minData = data.map(_.getDouble(1)) - data.take(10).foreach { r => part.ingest(0, r, ingestBlockHolder) } + data.take(10).foreach { r => part.ingest(0, r, ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled) } // First 10 rows ingested. Now flush in a separate Future while ingesting the remaining row part.switchBuffers(ingestBlockHolder) part.appendingChunkLen shouldEqual 0 - val blockHolder = new BlockMemFactory(blockStore, None, schema1.data.blockMetaSize, dummyContext) + val blockHolder = new BlockMemFactory(blockStore, schema1.data.blockMetaSize, dummyContext) // Task needs to fully iterate over the chunks, to release the shared lock. val flushFut = Future(part.makeFlushChunks(blockHolder).toBuffer) - data.drop(10).zipWithIndex.foreach { case (r, i) => part.ingest(0, r, ingestBlockHolder) } + data.drop(10).zipWithIndex.foreach { case (r, i) => part.ingest(0, r, ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled) } // there should be a frozen chunk of 10 records plus 1 record in currently appending chunks part.numChunks shouldEqual 2 @@ -222,17 +265,19 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { part = makePart(0, dataset1) val data = singleSeriesReaders().take(21) val minData = data.map(_.getDouble(1)) - data.take(10).zipWithIndex.foreach { case (r, i) => part.ingest(0, r, ingestBlockHolder) } + data.take(10).zipWithIndex.foreach { case (r, i) => part.ingest(0, r, ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled) } val origPoolSize = myBufferPool.poolSize // First 10 rows ingested. Now flush in a separate Future while ingesting 6 more rows part.switchBuffers(ingestBlockHolder) myBufferPool.poolSize shouldEqual origPoolSize // current chunks become null, no new allocation yet - val blockHolder = new BlockMemFactory(blockStore, None, schema1.data.blockMetaSize, dummyContext) + val blockHolder = new BlockMemFactory(blockStore, schema1.data.blockMetaSize, dummyContext) // Task needs to fully iterate over the chunks, to release the shared lock. val flushFut = Future(part.makeFlushChunks(blockHolder).toBuffer) - data.drop(10).take(6).zipWithIndex.foreach { case (r, i) => part.ingest(0, r, ingestBlockHolder) } + data.drop(10).take(6).zipWithIndex.foreach { case (r, i) => part.ingest(0, r, ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled) } val chunkSets = flushFut.futureValue // After flush, the old writebuffers should be returned to pool, but new one allocated too @@ -260,10 +305,11 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { // Now, switch buffers and flush again, ingesting 5 more rows // There should now be 3 chunks total, the current write buffers plus the two flushed ones part.switchBuffers(ingestBlockHolder) - val holder2 = new BlockMemFactory(blockStore, None, schema1.data.blockMetaSize, dummyContext) + val holder2 = new BlockMemFactory(blockStore, schema1.data.blockMetaSize, dummyContext) // Task needs to fully iterate over the chunks, to release the shared lock. val flushFut2 = Future(part.makeFlushChunks(holder2).toBuffer) - data.drop(16).zipWithIndex.foreach { case (r, i) => part.ingest(0, r, ingestBlockHolder) } + data.drop(16).zipWithIndex.foreach { case (r, i) => part.ingest(0, r, ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled) } val chunkSets2 = flushFut2.futureValue part.numChunks shouldEqual 3 @@ -283,13 +329,14 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { it("should not switch buffers and flush when current chunks are empty") { part = makePart(0, dataset1) val data = singleSeriesReaders().take(11) - data.zipWithIndex.foreach { case (r, i) => part.ingest(0, r, ingestBlockHolder) } + data.zipWithIndex.foreach { case (r, i) => part.ingest(0, r, ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled) } part.numChunks shouldEqual 1 part.appendingChunkLen shouldEqual 11 // Now, switch buffers and flush. Appenders will be empty. part.switchBuffers(ingestBlockHolder) - val blockHolder = new BlockMemFactory(blockStore, None, schema1.data.blockMetaSize, dummyContext) + val blockHolder = new BlockMemFactory(blockStore, schema1.data.blockMetaSize, dummyContext) val chunkSets = part.makeFlushChunks(blockHolder) chunkSets.isEmpty shouldEqual false part.numChunks shouldEqual 1 @@ -318,7 +365,8 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { makePart(partNo, dataset1) } (0 to 9).foreach { i => - data.foreach { case d => partitions(i).ingest(0, d, ingestBlockHolder) } + data.foreach { case d => partitions(i).ingest(0, d, ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled) } partitions(i).numChunks shouldEqual 1 partitions(i).appendingChunkLen shouldEqual 10 val infos = partitions(i).infos(AllChunkScan) @@ -339,7 +387,8 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { // Do this 4 more times so that we get old recycled metadata back (0 until 4).foreach { n => (0 to 9).foreach { i => - moreData.drop(n*10).take(10).foreach { case d => partitions(i).ingest(0, d, ingestBlockHolder) } + moreData.drop(n*10).take(10).foreach { case d => partitions(i).ingest(0, d, ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled) } partitions(i).appendingChunkLen shouldEqual 10 partitions(i).switchBuffers(ingestBlockHolder, true) } @@ -347,7 +396,8 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { // Now ingest again but don't switch buffers. Ensure appendingChunkLen is appropriate. (0 to 9).foreach { i => - moreData.drop(40).foreach { case d => partitions(i).ingest(0, d, ingestBlockHolder) } + moreData.drop(40).foreach { case d => partitions(i).ingest(0, d, ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled) } partitions(i).appendingChunkLen shouldEqual 10 } } @@ -358,7 +408,8 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { part = makePart(0, dataset1) val data = singleSeriesReaders().take(maxChunkSize + 10) - data.take(maxChunkSize - 10).zipWithIndex.foreach { case (r, i) => part.ingest(0, r, ingestBlockHolder) } + data.take(maxChunkSize - 10).zipWithIndex.foreach { case (r, i) => part.ingest(0, r, ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled) } part.numChunks shouldEqual 1 part.appendingChunkLen shouldEqual (maxChunkSize - 10) part.unflushedChunksets shouldEqual 1 @@ -366,7 +417,8 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { myBufferPool.poolSize shouldEqual (origPoolSize - 1) // Now ingest 20 more. Verify new chunks encoded. 10 rows after switch at 100. Verify can read everything. - data.drop(maxChunkSize - 10).zipWithIndex.foreach { case (r, i) => part.ingest(0, r, ingestBlockHolder) } + data.drop(maxChunkSize - 10).zipWithIndex.foreach { case (r, i) => part.ingest(0, r, ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled) } part.numChunks shouldEqual 2 part.appendingChunkLen shouldEqual 10 part.unflushedChunksets shouldEqual 2 @@ -379,7 +431,7 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { // Now simulate a flush, verify that both chunksets flushed // Now, switch buffers and flush. Appenders will be empty. part.switchBuffers(ingestBlockHolder) - val blockHolder = new BlockMemFactory(blockStore, None, schema1.data.blockMetaSize, dummyContext) + val blockHolder = new BlockMemFactory(blockStore, schema1.data.blockMetaSize, dummyContext) val chunkSets = part.makeFlushChunks(blockHolder).toSeq chunkSets should have length (2) part.numChunks shouldEqual 2 @@ -393,15 +445,22 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { val minData = data.map(_.getDouble(1)) // Ingest first 5, then: 8th, 6th, 7th, 9th, 10th - data.take(5).foreach { r => part.ingest(0, r, ingestBlockHolder) } - part.ingest(0, data(7), ingestBlockHolder) - part.ingest(0, data(5), ingestBlockHolder) - part.ingest(0, data(6), ingestBlockHolder) - part.ingest(0, data(8), ingestBlockHolder) - part.ingest(0, data(9), ingestBlockHolder) + data.take(5).foreach { r => part.ingest(0, r, ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled) } + part.ingest(0, data(7), ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled) + part.ingest(0, data(5), ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled) + part.ingest(0, data(6), ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled) + part.ingest(0, data(8), ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled) + part.ingest(0, data(9), ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled) // Try ingesting old sample now at the end. Verify that end time of chunkInfo is not incorrectly changed. - part.ingest(0, data(2), ingestBlockHolder) + part.ingest(0, data(2), ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled) // 8 of first 10 ingested, 2 should be dropped. Switch buffers, and try ingesting out of order again. part.appendingChunkLen shouldEqual 8 part.infoLast.numRows shouldEqual 8 @@ -411,10 +470,13 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { part.appendingChunkLen shouldEqual 0 // Now try ingesting an old smaple again at first element of next chunk. - part.ingest(0, data(8), ingestBlockHolder) // This one should be dropped + part.ingest(0, data(8), ingestBlockHolder, flushIntervalMillis = flushIntervalMillis, + createChunkAtFlushBoundary = timeAlignedChunksEnabled) // This one should be dropped part.appendingChunkLen shouldEqual 0 - part.ingest(0, data(10), ingestBlockHolder) - part.ingest(0, data(11), ingestBlockHolder) + part.ingest(0, data(10), ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled) + part.ingest(0, data(11), ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled) // there should be a frozen chunk of 10 records plus 2 records in currently appending chunks part.numChunks shouldEqual 2 @@ -431,7 +493,8 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { val data = singleSeriesReaders().take(11) val minData = data.map(_.getDouble(1)) val initTS = data(0).getLong(0) - data.take(10).zipWithIndex.foreach { case (r, i) => part.ingest(0, r, ingestBlockHolder) } + data.take(10).zipWithIndex.foreach { case (r, i) => part.ingest(0, r, ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled) } val origPoolSize = myBufferPool.poolSize diff --git a/core/src/test/scala/filodb.core/memstore/ratelimit/CardinalityTrackerSpec.scala b/core/src/test/scala/filodb.core/memstore/ratelimit/CardinalityTrackerSpec.scala new file mode 100644 index 0000000000..9596f4ccdb --- /dev/null +++ b/core/src/test/scala/filodb.core/memstore/ratelimit/CardinalityTrackerSpec.scala @@ -0,0 +1,249 @@ +package filodb.core.memstore.ratelimit + +import org.scalatest.funspec.AnyFunSpec +import org.scalatest.matchers.should.Matchers + +import filodb.core.{DatasetRef, MachineMetricsData} + +class CardinalityTrackerSpec extends AnyFunSpec with Matchers { + + val ref = MachineMetricsData.dataset2.ref + + private def newCardStore = { + new RocksDbCardinalityStore(DatasetRef("test"), 0) + } + + it("should enforce quota when set explicitly for all levels") { + val t = new CardinalityTracker(ref, 0, 3, Seq(4, 4, 4, 4), newCardStore) + t.setQuota(Seq("a", "aa", "aaa"), 1) shouldEqual Cardinality("aaa", 0, 0, 1) + t.setQuota(Seq("a", "aa"), 2) shouldEqual Cardinality("aa", 0, 0, 2) + t.setQuota(Seq("a"), 1) shouldEqual Cardinality("a",0, 0, 1) + t.incrementCount(Seq("a", "aa", "aaa")) shouldEqual + Seq(Cardinality("", 1, 1, 4), + Cardinality("a", 1, 1, 1), + Cardinality("aa", 1, 1, 2), + Cardinality("aaa", 1, 1, 1)) + t.incrementCount(Seq("a", "aa", "aab")) shouldEqual + Seq(Cardinality("", 2, 1, 4), + Cardinality("a", 2, 1, 1), + Cardinality("aa", 2, 2, 2), + Cardinality("aab", 1, 1, 4)) + + val ex = intercept[QuotaReachedException] { + t.incrementCount(Seq("a", "aa", "aac")) + } + ex.prefix shouldEqual (Seq("a", "aa")) + + // increment should not have been applied for any prefix + t.getCardinality(Seq("a")) shouldEqual Cardinality("a", 2, 1, 1) + t.getCardinality(Seq("a", "aa")) shouldEqual Cardinality("aa", 2, 2, 2) + t.getCardinality(Seq("a", "aa", "aac")) shouldEqual Cardinality("aac", 0, 0, 4) + t.close() + } + + it("should invoke quota exceeded protocol when breach occurs") { + + class MyQEP extends QuotaExceededProtocol { + var breachedPrefix: Seq[String] = Nil + var breachedQuota = -1 + def quotaExceeded(ref: DatasetRef, shard: Int, shardKeyPrefix: Seq[String], quota: Int): Unit = { + breachedPrefix = shardKeyPrefix + breachedQuota = quota + } + } + + val qp = new MyQEP + val t = new CardinalityTracker(ref, 0, 3, Seq(1, 1, 1, 1), newCardStore, qp) + t.incrementCount(Seq("a", "aa", "aaa")) + val ex = intercept[QuotaReachedException] { + t.incrementCount(Seq("a", "aa", "aaa")) + } + qp.breachedQuota shouldEqual 1 + qp.breachedPrefix shouldEqual Seq("a", "aa", "aaa") + t.close() + } + + it("should enforce quota when not set for any level") { + val t = new CardinalityTracker(ref, 0, 3, Seq(4, 4, 4, 4), newCardStore) + t.incrementCount(Seq("a", "ab", "aba")) shouldEqual + Seq(Cardinality("", 1, 1, 4), + Cardinality("a", 1, 1, 4), + Cardinality("ab", 1, 1, 4), + Cardinality("aba", 1, 1, 4)) + t.close() + } + + it("should be able to enforce for top 2 levels always, and enforce for 3rd level only in some cases") { + val t = new CardinalityTracker(ref, 0, 3, Seq(20, 20, 20, 20), newCardStore) + t.setQuota(Seq("a"), 10) shouldEqual Cardinality("a", 0, 0, 10) + t.setQuota(Seq("a", "aa"), 10) shouldEqual Cardinality("aa", 0, 0, 10) + // enforce for 3rd level only for aaa + t.setQuota(Seq("a", "aa", "aaa"), 2) shouldEqual Cardinality("aaa", 0, 0, 2) + t.incrementCount(Seq("a", "aa", "aaa")) shouldEqual + Seq(Cardinality("", 1, 1, 20), + Cardinality("a", 1, 1, 10), + Cardinality("aa", 1, 1, 10), + Cardinality("aaa", 1, 1, 2)) + t.incrementCount(Seq("a", "aa", "aaa")) shouldEqual + Seq(Cardinality("", 2, 1, 20), + Cardinality("a", 2, 1, 10), + Cardinality("aa", 2, 1, 10), + Cardinality("aaa", 2, 2, 2)) + t.incrementCount(Seq("a", "aa", "aab")) shouldEqual + Seq(Cardinality("", 3, 1, 20), + Cardinality("a", 3, 1, 10), + Cardinality("aa", 3, 2, 10), + Cardinality("aab", 1, 1, 20)) + t.incrementCount(Seq("a", "aa", "aab")) shouldEqual + Seq(Cardinality("", 4, 1, 20), + Cardinality("a", 4, 1, 10), + Cardinality("aa", 4, 2, 10), + Cardinality("aab", 2, 2, 20)) + t.incrementCount(Seq("a", "aa", "aab")) shouldEqual + Seq(Cardinality("", 5, 1, 20), + Cardinality("a", 5, 1, 10), + Cardinality("aa", 5, 2, 10), + Cardinality("aab", 3, 3, 20)) + t.incrementCount(Seq("a", "aa", "aab")) shouldEqual + Seq(Cardinality("", 6, 1, 20), + Cardinality("a", 6, 1, 10), + Cardinality("aa", 6, 2, 10), + Cardinality("aab", 4, 4, 20)) + + val ex = intercept[QuotaReachedException] { + t.incrementCount(Seq("a", "aa", "aaa")) + } + ex.prefix shouldEqual Seq("a", "aa", "aaa") + t.close() + + } + + it("should be able to increase and decrease quota after it has been set before") { + val t = new CardinalityTracker(ref, 0, 3, Seq(20, 20, 20, 20), newCardStore) + t.setQuota(Seq("a"), 10) shouldEqual Cardinality("a", 0, 0, 10) + t.setQuota(Seq("a", "aa"), 10) shouldEqual Cardinality("aa", 0, 0, 10) + // enforce for 3rd level only for aaa + t.setQuota(Seq("a", "aa", "aaa"), 2) shouldEqual Cardinality("aaa", 0, 0, 2) + t.incrementCount(Seq("a", "aa", "aaa")) shouldEqual + Seq(Cardinality("", 1, 1, 20), + Cardinality("a", 1, 1, 10), + Cardinality("aa", 1, 1, 10), + Cardinality("aaa", 1, 1, 2)) + t.incrementCount(Seq("a", "aa", "aaa")) shouldEqual + Seq(Cardinality("", 2, 1, 20), + Cardinality("a", 2, 1, 10), + Cardinality("aa", 2, 1, 10), + Cardinality("aaa", 2, 2, 2)) + + val ex = intercept[QuotaReachedException] { + t.incrementCount(Seq("a", "aa", "aaa")) + } + ex.prefix shouldEqual (Seq("a", "aa", "aaa")) + + // increase quota + t.setQuota(Seq("a", "aa", "aaa"), 5) shouldEqual Cardinality("aaa", 2, 2, 5) + t.incrementCount(Seq("a", "aa", "aaa")) shouldEqual + Seq(Cardinality("", 3, 1, 20), + Cardinality("a", 3, 1, 10), + Cardinality("aa", 3, 1, 10), + Cardinality("aaa", 3, 3, 5)) + + // decrease quota + t.setQuota(Seq("a", "aa", "aaa"), 4) shouldEqual Cardinality("aaa", 3, 3, 4) + t.incrementCount(Seq("a", "aa", "aaa")) shouldEqual + Seq(Cardinality("", 4, 1, 20), + Cardinality("a", 4, 1, 10), + Cardinality("aa", 4, 1, 10), + Cardinality("aaa", 4, 4, 4)) + val ex2 = intercept[QuotaReachedException] { + t.incrementCount(Seq("a", "aa", "aaa")) + } + ex2.prefix shouldEqual Seq("a", "aa", "aaa") + t.close() + } + + it("should be able to decrease quota if count is higher than new quota") { + val t = new CardinalityTracker(ref, 0, 3, Seq(20, 20, 20, 20), newCardStore) + t.setQuota(Seq("a"), 10) shouldEqual Cardinality("a", 0, 0, 10) + t.setQuota(Seq("a", "aa"), 10) shouldEqual Cardinality("aa", 0, 0, 10) + t.incrementCount(Seq("a", "aa", "aab")) shouldEqual + Seq(Cardinality("", 1, 1, 20), + Cardinality("a", 1, 1, 10), + Cardinality("aa", 1, 1, 10), + Cardinality("aab", 1, 1, 20)) + t.incrementCount(Seq("a", "aa", "aab")) shouldEqual + Seq(Cardinality("", 2, 1, 20), + Cardinality("a", 2, 1, 10), + Cardinality("aa", 2, 1, 10), + Cardinality("aab", 2, 2, 20)) + t.incrementCount(Seq("a", "aa", "aab")) shouldEqual + Seq(Cardinality("", 3, 1, 20), + Cardinality("a", 3, 1, 10), + Cardinality("aa", 3, 1, 10), + Cardinality("aab", 3, 3, 20)) + t.incrementCount(Seq("a", "aa", "aab")) shouldEqual + Seq(Cardinality("", 4, 1, 20), + Cardinality("a", 4, 1, 10), + Cardinality("aa", 4, 1, 10), + Cardinality("aab", 4, 4, 20)) + + t.getCardinality(Seq("a", "aa", "aab")) shouldEqual Cardinality("aab", 4, 4, 20) + + t.setQuota(Seq("a", "aa", "aab"), 3) + t.getCardinality(Seq("a", "aa", "aab")) shouldEqual Cardinality("aab", 4, 4, 3) + val ex2 = intercept[QuotaReachedException] { + t.incrementCount(Seq("a", "aa", "aab")) + } + ex2.prefix shouldEqual Seq("a", "aa", "aab") + t.close() + } + + it ("should be able to do topk") { + val t = new CardinalityTracker(ref, 0, 3, Seq(100, 100, 100, 100), newCardStore) + (1 to 10).foreach(_ => t.incrementCount(Seq("a", "ac", "aca"))) + (1 to 20).foreach(_ => t.incrementCount(Seq("a", "ac", "acb"))) + (1 to 11).foreach(_ => t.incrementCount(Seq("a", "ac", "acc"))) + (1 to 6).foreach(_ => t.incrementCount(Seq("a", "ac", "acd"))) + (1 to 1).foreach(_ => t.incrementCount(Seq("a", "ac", "ace"))) + (1 to 9).foreach(_ => t.incrementCount(Seq("a", "ac", "acf"))) + (1 to 15).foreach(_ => t.incrementCount(Seq("a", "ac", "acg"))) + + (1 to 15).foreach(_ => t.incrementCount(Seq("b", "bc", "bcg"))) + (1 to 9).foreach(_ => t.incrementCount(Seq("b", "bc", "bch"))) + (1 to 9).foreach(_ => t.incrementCount(Seq("b", "bd", "bdh"))) + + (1 to 3).foreach(_ => t.incrementCount(Seq("c", "cc", "ccg"))) + (1 to 2).foreach(_ => t.incrementCount(Seq("c", "cc", "cch"))) + + t.incrementCount(Seq("a", "aa", "aaa")) + t.incrementCount(Seq("a", "aa", "aab")) + t.incrementCount(Seq("a", "aa", "aac")) + t.incrementCount(Seq("a", "aa", "aad")) + t.incrementCount(Seq("b", "ba", "baa")) + t.incrementCount(Seq("b", "bb", "bba")) + t.incrementCount(Seq("a", "ab", "aba")) + t.incrementCount(Seq("a", "ab", "abb")) + t.incrementCount(Seq("a", "ab", "abc")) + t.incrementCount(Seq("a", "ab", "abd")) + t.incrementCount(Seq("a", "ab", "abe")) + + t.topk(3, Seq("a", "ac")) shouldEqual Seq( + CardinalityRecord(0, "acc", 11, 11, 100), + CardinalityRecord(0, "acg", 15, 15, 100), + CardinalityRecord(0, "acb", 20, 20, 100) + ) + + t.topk(3, Seq("a")) shouldEqual Seq( + CardinalityRecord(0, "aa", 4, 4, 100), + CardinalityRecord(0, "ab", 5, 5, 100), + CardinalityRecord(0, "ac", 72, 7, 100) + ) + + t.topk(3, Nil) shouldEqual Seq( + CardinalityRecord(0, "c", 5, 1, 100), + CardinalityRecord(0, "a", 81, 3, 100), + CardinalityRecord(0, "b", 35, 4, 100) + ) + t.close() + } +} \ No newline at end of file diff --git a/core/src/test/scala/filodb.core/memstore/ratelimit/RocksDbCardinalityStoreMemoryCapSpec.scala b/core/src/test/scala/filodb.core/memstore/ratelimit/RocksDbCardinalityStoreMemoryCapSpec.scala new file mode 100644 index 0000000000..5d7bbf230c --- /dev/null +++ b/core/src/test/scala/filodb.core/memstore/ratelimit/RocksDbCardinalityStoreMemoryCapSpec.scala @@ -0,0 +1,55 @@ +package filodb.core.memstore.ratelimit + +import org.scalatest.funspec.AnyFunSpec +import org.scalatest.matchers.should.Matchers + +import filodb.core.MachineMetricsData +import filodb.core.memstore.ratelimit.RocksDbCardinalityStore._ + +class RocksDbCardinalityStoreMemoryCapSpec extends AnyFunSpec with Matchers { + + val ref = MachineMetricsData.dataset2.ref + + val db = new RocksDbCardinalityStore(ref, 0) + val tracker = new CardinalityTracker(ref, 0, 3, Seq(100, 100, 1000, 1000), db) + + it("should be able to write keys quickly and cap memory usage") { + + def dumpStats() = { + println(db.statsAsString) + println(s"memTablesSize=${db.memTablesSize}") + println(s"blockCacheSize=${db.blockCacheSize}") + println(s"diskSpaceUsed=${db.diskSpaceUsed}") + println(s"estimatedNumKeys=${db.estimatedNumKeys}") + println() + } + + def assertStats() = { + db.blockCacheSize should be < LRU_CACHE_SIZE + (db.memTablesSize + db.blockCacheSize) should be < TOTAL_OFF_HEAP_SIZE + db.diskSpaceUsed should be < (100L << 20) + } + + val start = System.nanoTime() + for { ws <- 0 until 5 + ns <- 0 until 20 + name <- 0 until 50 + ts <- 0 until 100 } { + val mName = s"name_really_really_really_really_very_really_long_metric_name_$name" + tracker.incrementCount(Seq( s"ws_prefix_$ws", s"ns_prefix_$ns", mName)) + if (name == 0 && ts ==0 ) assertStats() + } + val end = System.nanoTime() + + assertStats() + dumpStats() + val numTimeSeries = 5 * 20 * 100 * 50 + val totalTimeSecs = (end-start) / 1000000000L + val timePerIncrementMicroSecs = (end-start) / numTimeSeries / 1000 + println(s"Was able to increment $numTimeSeries time series, $timePerIncrementMicroSecs" + + s"us each increment total of ${totalTimeSecs}s") + timePerIncrementMicroSecs should be < 200L + + } + +} diff --git a/core/src/test/scala/filodb.core/query/RangeVectorSpec.scala b/core/src/test/scala/filodb.core/query/RangeVectorSpec.scala index 8c15be7390..02b355c118 100644 --- a/core/src/test/scala/filodb.core/query/RangeVectorSpec.scala +++ b/core/src/test/scala/filodb.core/query/RangeVectorSpec.scala @@ -50,7 +50,7 @@ class RangeVectorSpec extends AnyFunSpec with Matchers { val builder = SerializedRangeVector.newBuilder() // Sharing one builder across multiple input RangeVectors - val srvs = rvs.map(rv => SerializedRangeVector(rv, builder, schema)) + val srvs = rvs.map(rv => SerializedRangeVector(rv, builder, schema, "RangeVectorSpec")) // Now verify each of them val observedTs = srvs(0).rows.toSeq.map(_.getLong(0)) diff --git a/http/src/main/scala/filodb/http/PrometheusApiRoute.scala b/http/src/main/scala/filodb/http/PrometheusApiRoute.scala index 014282fcd2..e6e4106eaa 100644 --- a/http/src/main/scala/filodb/http/PrometheusApiRoute.scala +++ b/http/src/main/scala/filodb/http/PrometheusApiRoute.scala @@ -48,7 +48,7 @@ class PrometheusApiRoute(nodeCoord: ActorRef, settings: HttpSettings)(implicit a // No cross-cluster failure routing in this API, hence we pass empty config askQueryAndRespond(dataset, logicalPlan, explainOnly.getOrElse(false), verbose.getOrElse(false), - spread, PromQlQueryParams(query, start.toLong, step.toLong, end.toLong, spread), histMap.getOrElse(false)) + spread, PromQlQueryParams(query, start.toLong, step.toLong, end.toLong), histMap.getOrElse(false)) } } } ~ @@ -64,7 +64,7 @@ class PrometheusApiRoute(nodeCoord: ActorRef, settings: HttpSettings)(implicit a val stepLong = step.map(_.toLong).getOrElse(0L) val logicalPlan = Parser.queryToLogicalPlan(query, time.toLong, stepLong) askQueryAndRespond(dataset, logicalPlan, explainOnly.getOrElse(false), - verbose.getOrElse(false), spread, PromQlQueryParams(query, time.toLong, stepLong, time.toLong, spread), + verbose.getOrElse(false), spread, PromQlQueryParams(query, time.toLong, stepLong, time.toLong), histMap.getOrElse(false)) } } diff --git a/jmh/src/main/scala/filodb.jmh/HistogramQueryBenchmark.scala b/jmh/src/main/scala/filodb.jmh/HistogramQueryBenchmark.scala index f5c355be1e..f22a4539ba 100644 --- a/jmh/src/main/scala/filodb.jmh/HistogramQueryBenchmark.scala +++ b/jmh/src/main/scala/filodb.jmh/HistogramQueryBenchmark.scala @@ -19,7 +19,7 @@ import filodb.core.{MachineMetricsData, MetricsTestData, SpreadChange, TestData} import filodb.core.binaryrecord2.RecordBuilder import filodb.core.memstore._ import filodb.core.metadata.Schemas -import filodb.core.query.{QueryConfig, QueryContext, QuerySession} +import filodb.core.query.{PlannerParams, QueryConfig, QueryContext, QuerySession} import filodb.core.store._ import filodb.memory.MemFactory import filodb.memory.format.SeqRowReader @@ -91,8 +91,11 @@ class HistogramQueryBenchmark { // Single-threaded query test val numQueries = 500 + QueryContext(plannerParams= PlannerParams(sampleLimit = 1000)) + QueryContext(plannerParams = PlannerParams(shardOverrides = Some(Seq(0)))) + val qContext = QueryContext(Some(new StaticSpreadProvider(SpreadChange(0, 1))), 100). - copy(shardOverrides = Some(Seq(0)), queryTimeoutMillis = 60000) + copy(plannerParams = PlannerParams(shardOverrides = Some(Seq(0)), queryTimeoutMillis = 60000)) val hLogicalPlan = Parser.queryToLogicalPlan(histQuery, startTime/1000, 1000) val hExecPlan = hEngine.materialize(hLogicalPlan, qContext) val querySched = Scheduler.singleThread(s"benchmark-query") diff --git a/memory/src/main/scala/filodb.memory/Block.scala b/memory/src/main/scala/filodb.memory/Block.scala index 4e6831e72a..153e32a185 100644 --- a/memory/src/main/scala/filodb.memory/Block.scala +++ b/memory/src/main/scala/filodb.memory/Block.scala @@ -107,16 +107,6 @@ class Block(val address: Long, val capacity: Long, val reclaimListener: ReclaimL owner = None } - /** - * Marks this block as reclaimable if unowned, or if the owner hasn't used the block in a while. - */ - def tryMarkReclaimable(): Unit = { - owner match { - case None => markReclaimable - case Some(bmf) => bmf.tryMarkReclaimable - } - } - /** * Marks this memory as free. Also zeroes all the bytes from the beginning address until capacity */ diff --git a/memory/src/main/scala/filodb.memory/BlockManager.scala b/memory/src/main/scala/filodb.memory/BlockManager.scala index eff2d484e0..af399b8860 100644 --- a/memory/src/main/scala/filodb.memory/BlockManager.scala +++ b/memory/src/main/scala/filodb.memory/BlockManager.scala @@ -1,14 +1,13 @@ package filodb.memory -import java.lang.{Long => jLong} -import java.util import java.util.concurrent.TimeUnit import java.util.concurrent.locks.ReentrantLock import com.kenai.jffi.{MemoryIO, PageManager} import com.typesafe.scalalogging.StrictLogging +import java.util import kamon.Kamon -import kamon.metric.Counter +import kamon.metric.{Counter, Gauge} import kamon.tag.TagSet final case class MemoryRequestException(msg: String) extends Exception(msg) @@ -28,25 +27,20 @@ trait BlockManager { */ def numFreeBlocks: Int - /** - * @return true if the time bucket has blocks allocated - */ - def hasTimeBucket(bucket: Long): Boolean - /** * @param memorySize The size of memory in bytes for which blocks are to be allocated - * @param bucketTime the timebucket (timestamp) from which to allocate block(s), or None for the general list + * @param odp true if this is for paging ODPed chunks * @param owner the BlockMemFactory that will be owning this block, until reclaim. Used for debugging. * @return A sequence of blocks totaling up in memory requested or empty if unable to allocate */ - def requestBlocks(memorySize: Long, bucketTime: Option[Long], owner: Option[BlockMemFactory] = None): Seq[Block] + def requestBlocks(memorySize: Long, odp: Boolean, owner: Option[BlockMemFactory] = None): Seq[Block] /** - * @param bucketTime the timebucket from which to allocate block(s), or None for the general list + * @param odp true if requesting for ODP * @param owner the BlockMemFactory that will be owning this block, until reclaim. Used for debugging. * @return One block of memory */ - def requestBlock(bucketTime: Option[Long], owner: Option[BlockMemFactory] = None): Option[Block] + def requestBlock(odp: Boolean, owner: Option[BlockMemFactory] = None): Option[Block] /** * Attempts to reclaim as many blocks as necessary to ensure that enough free blocks are @@ -90,11 +84,6 @@ trait BlockManager { */ def releaseBlocks(): Unit - /** - * Marks all time-bucketed blocks in buckets up to upTo as reclaimable - */ - def markBucketedBlocksReclaimable(upTo: Long): Unit - /** * @return Memory stats for recording */ @@ -102,13 +91,14 @@ trait BlockManager { } class MemoryStats(tags: Map[String, String]) { - val usedBlocksMetric = Kamon.gauge("blockstore-used-blocks").withTags(TagSet.from(tags)) + val usedIngestionBlocksMetric = Kamon.gauge("blockstore-used-ingestion-blocks").withTags(TagSet.from(tags)) val freeBlocksMetric = Kamon.gauge("blockstore-free-blocks").withTags(TagSet.from(tags)) val requestedBlocksMetric = Kamon.counter("blockstore-blocks-requested").withTags(TagSet.from(tags)) - val usedBlocksTimeOrderedMetric = Kamon.gauge("blockstore-used-time-ordered-blocks").withTags(TagSet.from(tags)) - val timeOrderedBlocksReclaimedMetric = Kamon.counter("blockstore-time-ordered-blocks-reclaimed") + val usedOdpBlocksMetric = Kamon.gauge("blockstore-used-odp-blocks").withTags(TagSet.from(tags)) + val odpBlocksReclaimedMetric = Kamon.counter("blockstore-odp-blocks-reclaimed") + .withTags(TagSet.from(tags)) + val ingestionBlocksReclaimedMetric = Kamon.counter("blockstore-ingestion-blocks-reclaimed") .withTags(TagSet.from(tags)) - val blocksReclaimedMetric = Kamon.counter("blockstore-blocks-reclaimed").withTags(TagSet.from(tags)) /** * How much time a thread was potentially stalled while attempting to ensure @@ -153,8 +143,8 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, protected var firstPageAddress: Long = 0L protected val freeBlocks: util.ArrayDeque[Block] = allocate() - protected[memory] val usedBlocks: util.ArrayDeque[Block] = new util.ArrayDeque[Block]() - protected[memory] val usedBlocksTimeOrdered = new util.TreeMap[Long, util.ArrayDeque[Block]] + protected[memory] val usedIngestionBlocks: util.ArrayDeque[Block] = new util.ArrayDeque[Block]() + protected[memory] val usedOdpBlocks: util.ArrayDeque[Block] = new util.ArrayDeque[Block]() val reclaimLog = new collection.mutable.Queue[ReclaimEvent] protected val lock = new ReentrantLock() @@ -164,12 +154,12 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, override def blockSizeInBytes: Long = PageManager.getInstance().pageSize() * numPagesPerBlock - def usedMemory: Long = usedBlocks.size * blockSizeInBytes + def usedMemory: Long = usedIngestionBlocks.size * blockSizeInBytes override def numFreeBlocks: Int = freeBlocks.size - override def requestBlock(bucketTime: Option[Long], bmf: Option[BlockMemFactory] = None): Option[Block] = { - val blocks = requestBlocks(blockSizeInBytes, bucketTime, bmf) + override def requestBlock(odp: Boolean, bmf: Option[BlockMemFactory] = None): Option[Block] = { + val blocks = requestBlocks(blockSizeInBytes, odp, bmf) blocks.size match { case 0 => None case 1 => Some(blocks.head) @@ -177,41 +167,33 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, } } - /* Used in tests for assertion */ - def usedBlocksSize(bucketTime: Option[Long]): Int = { - bucketTime match { - case Some(t) => usedBlocksTimeOrdered.get(t).size() - case None => usedBlocks.size() - } - } - /** * Allocates requested number of blocks. If enough blocks are not available, * then uses the ReclaimPolicy to check if blocks can be reclaimed * Uses a lock to ensure that concurrent requests are safe. * - * If bucketTime is provided, a MemoryRequestException is thrown when no blocks are - * currently available. In other words, time ordered block allocation doesn't force + * If odp is true, a MemoryRequestException is thrown when sufficient blocks are not + * currently free. In other words, ODP block request doesn't attempt * reclamation. Instead, a background task must be running which calls ensureFreeBlocks. - * Time ordered blocks are used for on-demand-paging only (ODP), initiated by a query, and + * ODP blocks are used for on-demand-paging only (ODP), initiated by a query, and * reclamation during ODP can end up causing the query results to have "holes". Throwing an * exception isn't a perfect solution, but it can suffice until a proper block pinning * mechanism is in place. Queries which fail with this exception can retry, perhaps after * calling ensureFreeBLocks explicitly. */ override def requestBlocks(memorySize: Long, - bucketTime: Option[Long], - bmf: Option[BlockMemFactory] = None): Seq[Block] = { + odp: Boolean, + ownerBmf: Option[BlockMemFactory] = None): Seq[Block] = { val num: Int = Math.ceil(memorySize / blockSizeInBytes).toInt stats.requestedBlocksMetric.increment(num) lock.lock() try { if (freeBlocks.size < num) { - if (bucketTime.isEmpty) { + if (!odp) { tryReclaimOnDemand(num) } else { - val msg = s"Unable to allocate time ordered block(s) without forcing a reclamation: " + + val msg = s"Unable to allocate ODP block(s) without forcing a reclamation: " + s"num_blocks=$num num_bytes=$memorySize freeBlocks=${freeBlocks.size}" throw new MemoryRequestException(msg) } @@ -221,8 +203,8 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, val allocated = new Array[Block](num) (0 until num).foreach { i => val block = freeBlocks.remove() - if (bmf.nonEmpty) block.setOwner(bmf.get) - use(block, bucketTime) + if (ownerBmf.nonEmpty) block.setOwner(ownerBmf.get) + use(block, odp) allocated(i) = block } allocated @@ -375,18 +357,14 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, blocks } - protected def use(block: Block, bucketTime: Option[Long]) = { + protected def use(block: Block, odp: Boolean) = { block.markInUse - bucketTime match { - case Some(bucket) => val blockList = Option(usedBlocksTimeOrdered.get(bucket)).getOrElse { - val list = new util.ArrayDeque[Block]() - usedBlocksTimeOrdered.put(bucket, list) - list - } - blockList.add(block) - stats.usedBlocksTimeOrderedMetric.update(numTimeOrderedBlocks) - case None => usedBlocks.add(block) - stats.usedBlocksMetric.update(usedBlocks.size()) + if (odp) { + usedOdpBlocks.add(block) + stats.usedOdpBlocksMetric.update(usedOdpBlocks.size()) + } else { + usedIngestionBlocks.add(block) + stats.usedIngestionBlocksMetric.update(usedIngestionBlocks.size()) } stats.freeBlocksMetric.update(freeBlocks.size()) } @@ -397,56 +375,29 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, reclaimLog += event } - //scalastyle:off protected[memory] def tryReclaim(num: Int): Int = { var reclaimed = 0 - - // First reclaim time-ordered blocks which are marked as reclaimable. - reclaimTimeOrdered(false); - + reclaimFrom(usedOdpBlocks, stats.odpBlocksReclaimedMetric, stats.usedOdpBlocksMetric) + if (reclaimed < num) + reclaimFrom(usedIngestionBlocks, stats.ingestionBlocksReclaimedMetric, stats.usedIngestionBlocksMetric) + // if we do not get required blocks even after reclaim call if (reclaimed < num) { - // Not enough reclaimed, so try reclaiming non-time-ordered blocks which are marked as reclaimable. - reclaimFrom(usedBlocks, stats.blocksReclaimedMetric, false) - - if (reclaimed < num) { - // Still not enough? Forcibly reclaim time-ordered blocks. - reclaimTimeOrdered(true); - - if (reclaimed < num) { - // Still not enough, but forcibly reclaiming non-time-ordered blocks is dangerous. - logger.warn(s"$num blocks to reclaim but only reclaimed $reclaimed. usedblocks=${usedBlocks.size} " + - s"usedBlocksTimeOrdered=${usedBlocksTimeOrdered.asScala.toList.map{case(n, l) => (n, l.size)}}") - } - } + logger.warn(s"$num blocks to reclaim but only reclaimed $reclaimed. " + + s"usedIngestionBlocks=${usedIngestionBlocks.size} usedOdpBlocks=${usedOdpBlocks.size()}") } - def reclaimTimeOrdered(forced: Boolean): Unit = { - val timeOrderedListIt = usedBlocksTimeOrdered.entrySet.iterator - while ( reclaimed < num && - timeOrderedListIt.hasNext ) { - val entry = timeOrderedListIt.next - val prevReclaimed = reclaimed - val removed = reclaimFrom(entry.getValue, stats.timeOrderedBlocksReclaimedMetric, forced) - if (removed.nonEmpty) { - logger.info(s"timeBlockReclaim: Reclaimed ${removed.length} time ordered blocks " + - s"from list at t=${entry.getKey} (${(System.currentTimeMillis - entry.getKey)/3600000} hrs ago) " + - s"\nReclaimed blocks: ${removed.map(b => jLong.toHexString(b.address)).mkString(" ")}") - } - // If the block list is now empty, remove it from tree map - if (entry.getValue.isEmpty) timeOrderedListIt.remove() - } - } - - def reclaimFrom(list: util.ArrayDeque[Block], reclaimedCounter: Counter, forced: Boolean): Seq[Block] = { + def reclaimFrom(list: util.ArrayDeque[Block], + reclaimedCounter: Counter, + usedBlocksStats: Gauge): Seq[Block] = { val entries = list.iterator val removed = new collection.mutable.ArrayBuffer[Block] while (entries.hasNext && reclaimed < num) { val block = entries.next - if (forced || block.canReclaim) { + if (block.canReclaim) { entries.remove() removed += block addToReclaimLog(block) - block.reclaim(forced) + block.reclaim() block.clearOwner() freeBlocks.add(block) stats.freeBlocksMetric.update(freeBlocks.size()) @@ -454,45 +405,20 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, reclaimed = reclaimed + 1 } } + usedBlocksStats.update(list.size()) removed } reclaimed } - //scalastyle:on - - def numTimeOrderedBlocks: Int = usedBlocksTimeOrdered.values.asScala.map(_.size).sum - - def timeBuckets: Seq[Long] = usedBlocksTimeOrdered.keySet.asScala.toSeq - - def markBucketedBlocksReclaimable(upTo: Long): Unit = { - lock.lock() - try { - logger.info(s"timeBlockReclaim: Marking ($upTo) - this is -${(System.currentTimeMillis - upTo)/3600000}hrs") - val keys = usedBlocksTimeOrdered.headMap(upTo).keySet.asScala - logger.info(s"timeBlockReclaim: Marking lists $keys as reclaimable") - usedBlocksTimeOrdered.headMap(upTo).values.asScala.foreach { list => - list.asScala.foreach(_.tryMarkReclaimable) - } - } finally { - lock.unlock() - } - } - - def hasTimeBucket(bucket: Long): Boolean = { - lock.lock() - val result = usedBlocksTimeOrdered.containsKey(bucket) - lock.unlock() - result - } /** * Used during testing only to try and reclaim all existing blocks */ def reclaimAll(): Unit = { logger.warn(s"Reclaiming all used blocks -- THIS BETTER BE A TEST!!!") - markBucketedBlocksReclaimable(Long.MaxValue) - usedBlocks.asScala.foreach(_.markReclaimable) - tryReclaim(usedBlocks.size + numTimeOrderedBlocks) + usedIngestionBlocks.asScala.foreach(_.markReclaimable()) + usedOdpBlocks.asScala.foreach(_.markReclaimable()) + tryReclaim(usedIngestionBlocks.size + usedOdpBlocks.size()) } /** @@ -505,9 +431,7 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, def timeBlocksForPtr(ptr: BinaryRegion.NativePointer): Seq[Block] = { lock.lock() try { - usedBlocksTimeOrdered.entrySet.iterator.asScala.flatMap { entry => - BlockDetective.containsPtr(ptr, entry.getValue) - }.toBuffer + BlockDetective.containsPtr(ptr, usedOdpBlocks) } finally { lock.unlock() } @@ -527,5 +451,5 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, } } - override def finalize(): Unit = releaseBlocks + override def finalize(): Unit = releaseBlocks() } diff --git a/memory/src/main/scala/filodb.memory/BlockMemFactoryPool.scala b/memory/src/main/scala/filodb.memory/BlockMemFactoryPool.scala index 736744f91f..ee186e218c 100644 --- a/memory/src/main/scala/filodb.memory/BlockMemFactoryPool.scala +++ b/memory/src/main/scala/filodb.memory/BlockMemFactoryPool.scala @@ -27,7 +27,7 @@ class BlockMemFactoryPool(blockStore: BlockManager, factoryPool.dequeue } else { logger.debug(s"Nothing in BlockMemFactory pool. Creating a new one") - new BlockMemFactory(blockStore, None, metadataAllocSize, baseTags) + new BlockMemFactory(blockStore, metadataAllocSize, baseTags) } fact.tags = baseTags ++ moreTags fact diff --git a/memory/src/main/scala/filodb.memory/MemFactory.scala b/memory/src/main/scala/filodb.memory/MemFactory.scala index f6d59d8938..74e2d1f4f8 100644 --- a/memory/src/main/scala/filodb.memory/MemFactory.scala +++ b/memory/src/main/scala/filodb.memory/MemFactory.scala @@ -202,14 +202,12 @@ object BlockMemFactory { * * @param blockStore The BlockManager which is used to request more blocks when the current * block is full. - * @param bucketTime the timebucket (timestamp) from which to allocate block(s), or None for the general list * @param metadataAllocSize the additional size in bytes to ensure is free for writing metadata, per chunk * @param tags a set of keys/values to identify the purpose of this MemFactory for debugging * @param markFullBlocksAsReclaimable Immediately mark and fully used block as reclaimable. * Typically true during on-demand paging of optimized chunks from persistent store */ class BlockMemFactory(blockStore: BlockManager, - bucketTime: Option[Long], metadataAllocSize: Int, var tags: Map[String, String], markFullBlocksAsReclaimable: Boolean = false) extends MemFactory with StrictLogging { @@ -217,12 +215,13 @@ class BlockMemFactory(blockStore: BlockManager, val optionSelf = Some(this) // tracks fully populated blocks not marked reclaimable yet (typically waiting for flush) + // NOT used in ODP block mem factories where markFullBlocksAsReclaimable = true val fullBlocksToBeMarkedAsReclaimable = ListBuffer[Block]() // tracks block currently being populated var currentBlock = requestBlock() - private def requestBlock() = blockStore.requestBlock(bucketTime, optionSelf).get + private def requestBlock() = blockStore.requestBlock(markFullBlocksAsReclaimable, optionSelf).get // tracks blocks that should share metadata private val metadataSpan: ListBuffer[Block] = ListBuffer[Block]() @@ -262,6 +261,9 @@ class BlockMemFactory(blockStore: BlockManager, /** * Starts tracking a span of multiple Blocks over which the same metadata should be applied. * An example would be chunk metadata for chunks written to potentially more than 1 block. + * + * IMPORTANT: Acquire blockMemFactory.synchronized before calling startMetaSpan and release after endMetaSpan + * */ def startMetaSpan(): Unit = { metadataSpan.clear() @@ -271,6 +273,8 @@ class BlockMemFactory(blockStore: BlockManager, /** * Stops tracking the blocks that the same metadata should be applied to, and allocates and writes metadata * for those spanned blocks. + * IMPORTANT: Acquire blockMemFactory.synchronized before calling startMetaSpan and release after endMetaSpan + * * @param metadataWriter the function to write metadata to each block. Param is the long metadata address. * @param metaSize the number of bytes the piece of metadata takes * @return the Long native address of the last metadata block written @@ -296,7 +300,7 @@ class BlockMemFactory(blockStore: BlockManager, if (markFullBlocksAsReclaimable) { // We know that all the blocks in the span except the last one is full, so mark them reclaimable blk.markReclaimable() - } else synchronized { + } else { fullBlocksToBeMarkedAsReclaimable += blk } } @@ -385,7 +389,8 @@ class BlockMemFactory(blockStore: BlockManager, def shutdown(): Unit = {} def debugString: String = - s"BlockMemFactory($bucketTime, $metadataAllocSize) ${tags.map { case (k, v) => s"$k=$v" }.mkString(" ")}" + s"BlockMemFactory($markFullBlocksAsReclaimable, $metadataAllocSize) " + + s"${tags.map { case (k, v) => s"$k=$v" }.mkString(" ")}" } diff --git a/memory/src/main/scala/filodb.memory/data/ChunkMap.scala b/memory/src/main/scala/filodb.memory/data/ChunkMap.scala index 4c310790ee..609c1391a1 100644 --- a/memory/src/main/scala/filodb.memory/data/ChunkMap.scala +++ b/memory/src/main/scala/filodb.memory/data/ChunkMap.scala @@ -1,5 +1,7 @@ package filodb.memory.data +import java.util.concurrent.ConcurrentHashMap + import scala.collection.mutable.{HashMap, Map} import scala.concurrent.duration._ @@ -58,6 +60,13 @@ object ChunkMap extends StrictLogging { override def initialValue() = new HashMap[ChunkMap, Int] } + /** + * FIXME: Remove this after debugging is done. + * This keeps track of which thread is running which execPlan. + * Entry is added on lock acquisition, removed when lock is released. + */ + private val execPlanTracker = new ConcurrentHashMap[Thread, String] + // Returns true if the current thread has acquired the shared lock at least once. private def hasSharedLock(inst: ChunkMap): Boolean = sharedLockCounts.get.contains(inst) @@ -90,16 +99,24 @@ object ChunkMap extends StrictLogging { if (amt > 0) { total += amt sharedLockLingering.increment(amt) - _logger.warn(s"Releasing all shared locks for: $inst, amount: $amt") + _logger.error(s"Lingering locks while releasing all shared locks for pk $inst, amount=$amt " + + s"Contents of execPlanTracker for current thread: ${execPlanTracker.get(Thread.currentThread())}", + new RuntimeException) var lockState = 0 do { lockState = UnsafeUtils.getIntVolatile(inst, lockStateOffset) + if ((lockState & Int.MaxValue) - amt < 0) { + _logger.error(s"Negative lock state while releasing all shared locks for pk: $inst, amount=$amt " + + s"Contents of execPlanTracker for current thread: ${execPlanTracker.get(Thread.currentThread())}", + new RuntimeException) + } } while (!UnsafeUtils.unsafe.compareAndSwapInt(inst, lockStateOffset, lockState, lockState - amt)) } } countMap.clear } + execPlanTracker.remove(Thread.currentThread()) total } //scalastyle:on null @@ -109,7 +126,12 @@ object ChunkMap extends StrictLogging { * consumption from a query iterator. If there are lingering locks, * it is quite possible a lock acquire or release bug exists */ - def validateNoSharedLocks(unitTest: Boolean = false): Unit = { + def validateNoSharedLocks(execPlan: String, unitTest: Boolean = false): Unit = { + val t = Thread.currentThread() + if (execPlanTracker.containsKey(t)) { + logger.error(s"Current thread ${t.getName} did not release lock for execPlan: ${execPlanTracker.get(t)}") + } + // Count up the number of held locks. var total = 0 val countMap = sharedLockCounts.get @@ -128,6 +150,8 @@ object ChunkMap extends StrictLogging { s"This is indicative of a possible lock acquisition/release bug.") Shutdown.haltAndCatchFire(ex) } + + execPlanTracker.put(t, execPlan) } } @@ -253,6 +277,7 @@ class ChunkMap(val memFactory: NativeMemoryManager, var capacity: Int) { var warned = false // scalastyle:off null + var locks1: ConcurrentHashMap[Thread, String] = null while (true) { if (tryAcquireExclusive(timeoutNanos)) { if (arrayPtr == 0) { @@ -273,10 +298,14 @@ class ChunkMap(val memFactory: NativeMemoryManager, var capacity: Int) { } exclusiveLockWait.increment() _logger.warn(s"Waiting for exclusive lock: $this") + locks1 = new ConcurrentHashMap[Thread, String](execPlanTracker) warned = true } else if (warned && timeoutNanos >= MaxExclusiveRetryTimeoutNanos) { + val locks2 = new ConcurrentHashMap[Thread, String](execPlanTracker) + locks2.entrySet().retainAll(locks1.entrySet()) val lockState = UnsafeUtils.getIntVolatile(this, lockStateOffset) - Shutdown.haltAndCatchFire(new RuntimeException(s"Unable to acquire exclusive lock: $lockState")) + Shutdown.haltAndCatchFire(new RuntimeException(s"Following execPlan locks have not been released for a" + + s"while: $locks2 $locks1 $execPlanTracker $lockState")) } } } @@ -375,6 +404,11 @@ class ChunkMap(val memFactory: NativeMemoryManager, var capacity: Int) { var lockState = 0 do { lockState = UnsafeUtils.getIntVolatile(this, lockStateOffset) + if ((lockState & Int.MaxValue) - 1 < 0) { + _logger.error(s"Negative lock state while releasing single shared lock for pk: $this " + + s"Contents of execPlanTracker for current thread: ${execPlanTracker.get(Thread.currentThread())}", + new RuntimeException) + } } while (!UnsafeUtils.unsafe.compareAndSwapInt(this, lockStateOffset, lockState, lockState - 1)) adjustSharedLockCount(this, -1) } diff --git a/memory/src/main/scala/filodb.memory/data/Shutdown.scala b/memory/src/main/scala/filodb.memory/data/Shutdown.scala index 3fdf2d6aff..2427bd3bed 100644 --- a/memory/src/main/scala/filodb.memory/data/Shutdown.scala +++ b/memory/src/main/scala/filodb.memory/data/Shutdown.scala @@ -1,5 +1,7 @@ package filodb.memory.data +import scala.concurrent.Await + import com.typesafe.scalalogging.StrictLogging import kamon.Kamon @@ -10,6 +12,8 @@ object Shutdown extends StrictLogging { forcedShutdowns.increment() if (unitTest) throw e logger.error(s"Shutting down process since it may be in an unstable/corrupt state", e) + import scala.concurrent.duration._ + Await.result(Kamon.stopModules(), 5.minutes) Runtime.getRuntime.halt(189) } diff --git a/memory/src/main/scala/filodb.memory/format/MemoryLogger.scala b/memory/src/main/scala/filodb.memory/format/MemoryLogger.scala new file mode 100644 index 0000000000..39094ed67e --- /dev/null +++ b/memory/src/main/scala/filodb.memory/format/MemoryLogger.scala @@ -0,0 +1,8 @@ +package filodb.memory.format + +import com.typesafe.scalalogging.{Logger, StrictLogging} + +object MemoryLogger extends StrictLogging { + protected[memory] val mLogger: Logger = logger + +} 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 a1808c20ea..2e8f91e2c6 100644 --- a/memory/src/main/scala/filodb.memory/format/vectors/DoubleVector.scala +++ b/memory/src/main/scala/filodb.memory/format/vectors/DoubleVector.scala @@ -341,10 +341,18 @@ extends DoubleVectorDataReader { override def correctedValue(acc2: MemoryReader, vector: BinaryVectorPtr, n: Int, correctionMeta: CorrectionMeta): Double = { assert(vector == vect && acc == acc2) - correctionMeta match { - // corrected value + any carryover correction - case DoubleCorrection(_, corr) => corrected(n) + corr - case NoCorrection => corrected(n) + try { + correctionMeta match { + // corrected value + any carryover correction + case DoubleCorrection(_, corr) => corrected(n) + corr + case NoCorrection => corrected(n) + } + } catch { case e: ArrayIndexOutOfBoundsException => + MemoryLogger.mLogger.error(s"ArrayIndexOutOfBoundsException Vector is [${toHexString(acc2, vector)}] ") + MemoryLogger.mLogger.error(s"ArrayIndexOutOfBoundsException corrected is ${corrected.mkString(",")}") + MemoryLogger.mLogger.error(s"ArrayIndexOutOfBoundsException vectorLength is ${length(acc2, vector)}") + MemoryLogger.mLogger.error(s"ArrayIndexOutOfBoundsException _drops is ${_drops}") + throw e } } diff --git a/memory/src/test/scala/filodb.memory/BlockMemFactorySpec.scala b/memory/src/test/scala/filodb.memory/BlockMemFactorySpec.scala index c79f14c66e..11ebc7e7fc 100644 --- a/memory/src/test/scala/filodb.memory/BlockMemFactorySpec.scala +++ b/memory/src/test/scala/filodb.memory/BlockMemFactorySpec.scala @@ -15,7 +15,7 @@ class BlockMemFactorySpec extends AnyFlatSpec with Matchers { it should "Mark all blocks of BlockMemFactory as reclaimable when used as done in ingestion pipeline" in { val stats = new MemoryStats(Map("test1" -> "test1")) val blockManager = new PageAlignedBlockManager(2048 * 1024, stats, testReclaimer, 1) - val bmf = new BlockMemFactory(blockManager, None, 50, Map("test" -> "val"), false) + val bmf = new BlockMemFactory(blockManager, 50, Map("test" -> "val"), false) // simulate encoding of multiple ts partitions in flush group @@ -37,13 +37,102 @@ class BlockMemFactorySpec extends AnyFlatSpec with Matchers { } // only the current block is not reclaimable - blockManager.usedBlocks.asScala.count(!_.canReclaim) shouldEqual 1 + blockManager.usedIngestionBlocks.asScala.count(!_.canReclaim) shouldEqual 1 - blockManager.usedBlocks.size shouldEqual 12 + blockManager.usedIngestionBlocks.size shouldEqual 12 blockManager.tryReclaim(3) shouldEqual 3 - blockManager.usedBlocks.size shouldEqual 9 // 3 are reclaimed + blockManager.usedIngestionBlocks.size shouldEqual 9 // 3 are reclaimed blockManager.releaseBlocks() } + it should "Mark all blocks of BlockMemFactory as reclaimable when used in ODP by DemandPagedChunkStore" in { + val stats = new MemoryStats(Map("test1" -> "test1")) + val blockManager = new PageAlignedBlockManager(2048 * 1024, stats, testReclaimer, 1) + + // create block mem factories for different time buckets + val bmf = new BlockMemFactory(blockManager, 50, Map("test" -> "val"), true) + + // simulate paging in chunks from cassandra + for {tsParts <- 0 to 10} { + bmf.startMetaSpan() + for {chunks <- 0 to 3} { + bmf.allocateOffheap(1000) + } + bmf.endMetaSpan(d => {}, 45) + } + + // we dont track full blocks in ODP mode + bmf.fullBlocksToBeMarkedAsReclaimable.isEmpty shouldEqual true + + // usedBlocks is not used for ODP mode + blockManager.usedIngestionBlocks.isEmpty shouldEqual true + + // time ordered blocks is used in ODP mode + // 11 blocks are used, out of which 10 are reclaimable, except the "current block" + blockManager.usedOdpBlocks.asScala.count(_.canReclaim) shouldEqual 10 + bmf.currentBlock.canReclaim shouldEqual false // current blocks cannot be reclaimed + + // There should be 11 time ordered blocks used before reclaim + blockManager.usedOdpBlocks.asScala.size shouldEqual 11 + blockManager.tryReclaim(5) shouldEqual 5 + // after reclaiming 5 blocks, only 11-5 == 6 time ordered blocks should be used + blockManager.usedOdpBlocks.asScala.size shouldEqual 6 + + // even if you try to reclaim all of them, you cannot reclaim current block + blockManager.tryReclaim(7) shouldEqual 5 + blockManager.usedOdpBlocks.asScala.size shouldEqual 1 + + blockManager.releaseBlocks() + } + + + it should "Reclaim Ingestion and ODP blocks in right order when used together" in { + val stats = new MemoryStats(Map("test1" -> "test1")) + val blockManager = new PageAlignedBlockManager(2048 * 1024, stats, testReclaimer, 1) + + val ingestionFactory = new BlockMemFactory(blockManager, 50, Map("test" -> "val"), false) + + // create block mem factories for different time buckets + val odpFactory = new BlockMemFactory(blockManager, 50, Map("test" -> "val"), true) + + // simulate encoding of multiple ts partitions in flush group + for { flushGroup <- 0 to 1 } { + for {tsParts <- 0 to 5} { + ingestionFactory.startMetaSpan() + for {chunks <- 0 to 3} { + ingestionFactory.allocateOffheap(1000) + } + ingestionFactory.endMetaSpan(d => {}, 45) + } + // full blocks are tracked as they are allocated + flushGroup match { + case 0 => ingestionFactory.fullBlocksToBeMarkedAsReclaimable.size shouldEqual 5 + case 1 => ingestionFactory.fullBlocksToBeMarkedAsReclaimable.size shouldEqual 6 + } + // full blocks are marked as reclaimable + ingestionFactory.markFullBlocksReclaimable() + } + + // simulate paging in chunks from cassandra + for {tsParts <- 0 to 10} { + odpFactory.startMetaSpan() + for {chunks <- 0 to 3} { + odpFactory.allocateOffheap(1000) + } + odpFactory.endMetaSpan(d => {}, 45) + } + + // here are the use block counts before reclaim call + blockManager.usedOdpBlocks.size shouldEqual 11 + blockManager.usedIngestionBlocks.size shouldEqual 12 + blockManager.tryReclaim(15) shouldEqual 15 + + // after reclaim, only 1 odp block + blockManager.usedOdpBlocks.asScala.size shouldEqual 1 + + // ingestion blocks should be reclaimed only if we cannot get reclaim ODP blocks. + blockManager.usedIngestionBlocks.asScala.size shouldEqual 7 + + } } diff --git a/memory/src/test/scala/filodb.memory/BlockSpec.scala b/memory/src/test/scala/filodb.memory/BlockSpec.scala index 0a6bcc2049..4c4d4e82f8 100644 --- a/memory/src/test/scala/filodb.memory/BlockSpec.scala +++ b/memory/src/test/scala/filodb.memory/BlockSpec.scala @@ -22,7 +22,7 @@ class BlockSpec extends AnyFlatSpec with Matchers with BeforeAndAfter with Befor } it should "allocate metadata and report remaining bytes accurately" in { - val block = blockManager.requestBlock(None).get + val block = blockManager.requestBlock(false).get block.capacity shouldEqual 4096 block.remaining shouldEqual 4096 @@ -36,7 +36,7 @@ class BlockSpec extends AnyFlatSpec with Matchers with BeforeAndAfter with Befor } it should "return null when allocate metadata if not enough space" in { - val block = blockManager.requestBlock(None).get + val block = blockManager.requestBlock(false).get block.capacity shouldEqual 4096 block.remaining shouldEqual 4096 @@ -47,13 +47,13 @@ class BlockSpec extends AnyFlatSpec with Matchers with BeforeAndAfter with Befor } it should "not reclaim when block has not been marked reclaimable" in { - val block = blockManager.requestBlock(None).get + val block = blockManager.requestBlock(false).get intercept[IllegalStateException] { block.reclaim() } } it should "call reclaimListener with address of all allocated metadatas" in { - val block = blockManager.requestBlock(None).get + val block = blockManager.requestBlock(false).get block.capacity shouldEqual 4096 block.remaining shouldEqual 4096 diff --git a/memory/src/test/scala/filodb.memory/PageAlignedBlockManagerConcurrentSpec.scala b/memory/src/test/scala/filodb.memory/PageAlignedBlockManagerConcurrentSpec.scala index ee48cfd77a..3130c1368a 100644 --- a/memory/src/test/scala/filodb.memory/PageAlignedBlockManagerConcurrentSpec.scala +++ b/memory/src/test/scala/filodb.memory/PageAlignedBlockManagerConcurrentSpec.scala @@ -23,7 +23,7 @@ with ConductorFixture with Matchers with BeforeAndAfterAll { threadNamed("Random guy") { //1 page - val blocks = blockManager.requestBlocks(pageSize, None) + val blocks = blockManager.requestBlocks(pageSize, false) blocks.size should be(1) val block = blocks.head block.position(block.position() + 1) @@ -31,7 +31,7 @@ with ConductorFixture with Matchers with BeforeAndAfterAll { } threadNamed("Another dude") { //2 page - val blocks = blockManager.requestBlocks(2 * pageSize, None) + val blocks = blockManager.requestBlocks(2 * pageSize, false) blocks.size should be(2) val block = blocks.head block.position(block.position() + 1) @@ -39,7 +39,7 @@ with ConductorFixture with Matchers with BeforeAndAfterAll { } threadNamed("Yet another dude") { //3 page - val blocks = blockManager.requestBlocks(3 * pageSize, None) + val blocks = blockManager.requestBlocks(3 * pageSize, false) blocks.size should be(3) val block = blocks.head block.position(block.position() + 1) diff --git a/memory/src/test/scala/filodb.memory/PageAlignedBlockManagerSpec.scala b/memory/src/test/scala/filodb.memory/PageAlignedBlockManagerSpec.scala index 59afa16b6e..4b82874cca 100644 --- a/memory/src/test/scala/filodb.memory/PageAlignedBlockManagerSpec.scala +++ b/memory/src/test/scala/filodb.memory/PageAlignedBlockManagerSpec.scala @@ -35,7 +35,7 @@ class PageAlignedBlockManagerSpec extends AnyFlatSpec with Matchers with BeforeA // val fbm = freeBlocksMetric(stats) // fbm.max should be(512) val blockSize = blockManager.blockSizeInBytes - val blocks = blockManager.requestBlocks(blockSize * 10, None) + val blocks = blockManager.requestBlocks(blockSize * 10, false) blocks.size should be(10) // val ubm = usedBlocksMetric(stats) // ubm.max should be(10) @@ -61,7 +61,7 @@ class PageAlignedBlockManagerSpec extends AnyFlatSpec with Matchers with BeforeA val blockSize = blockManager.blockSizeInBytes // val fbm = freeBlocksMetric(stats) // fbm.max should be(2) - val firstRequest = blockManager.requestBlocks(blockSize * 2, None) + val firstRequest = blockManager.requestBlocks(blockSize * 2, false) //used 2 out of 2 firstRequest.size should be(2) // val ubm = usedBlocksMetric(stats) @@ -69,7 +69,7 @@ class PageAlignedBlockManagerSpec extends AnyFlatSpec with Matchers with BeforeA //cannot fulfill // val fbm2 = freeBlocksMetric(stats) // fbm2.min should be(0) - val secondRequest = blockManager.requestBlocks(blockSize * 2, None) + val secondRequest = blockManager.requestBlocks(blockSize * 2, false) secondRequest should be(Seq.empty) blockManager.releaseBlocks() @@ -80,14 +80,14 @@ class PageAlignedBlockManagerSpec extends AnyFlatSpec with Matchers with BeforeA val stats = new MemoryStats(Map("test4" -> "test4")) val blockManager = new PageAlignedBlockManager(2 * pageSize, stats, testReclaimer, 1) val blockSize = blockManager.blockSizeInBytes - val firstRequest = blockManager.requestBlocks(blockSize * 2, None) + val firstRequest = blockManager.requestBlocks(blockSize * 2, false) //used 2 out of 2 firstRequest.size should be(2) //simulate writing to the block firstRequest.head.position(blockSize.toInt - 1) //mark them as reclaimable firstRequest.foreach(_.markReclaimable()) - val secondRequest = blockManager.requestBlocks(blockSize * 2, None) + val secondRequest = blockManager.requestBlocks(blockSize * 2, false) // val brm = reclaimedBlocksMetric(stats) // brm.count should be(2) //this request will fulfill @@ -102,11 +102,11 @@ class PageAlignedBlockManagerSpec extends AnyFlatSpec with Matchers with BeforeA val stats = new MemoryStats(Map("test5" -> "test5")) val blockManager = new PageAlignedBlockManager(4 * pageSize, stats, testReclaimer, 1) val blockSize = blockManager.blockSizeInBytes - val firstRequest = blockManager.requestBlocks(blockSize * 2, None) + val firstRequest = blockManager.requestBlocks(blockSize * 2, false) //used 2 out of 4 firstRequest.size should be(2) //only 2 left - cannot fulfill request - val secondRequest = blockManager.requestBlocks(blockSize * 3, None) + val secondRequest = blockManager.requestBlocks(blockSize * 3, false) // val brm = reclaimedBlocksMetric(stats) // brm.count should be(0) secondRequest should be(Seq.empty) @@ -176,28 +176,23 @@ class PageAlignedBlockManagerSpec extends AnyFlatSpec with Matchers with BeforeA // This block manager has 5 blocks capacity val blockManager = new PageAlignedBlockManager(5 * pageSize, stats, testReclaimer, 1) - blockManager.usedBlocks.size() shouldEqual 0 - blockManager.numTimeOrderedBlocks shouldEqual 0 - blockManager.usedBlocksTimeOrdered.size shouldEqual 0 - - // first allocate non-time ordered block - blockManager.requestBlock(None).map(_.markReclaimable).isDefined shouldEqual true - blockManager.usedBlocks.size shouldEqual 1 - - blockManager.requestBlock(Some(1000L)).map(_.markReclaimable).isDefined shouldEqual true - blockManager.requestBlock(Some(1000L)).map(_.markReclaimable).isDefined shouldEqual true - blockManager.requestBlock(Some(1000L)).isDefined shouldEqual true - blockManager.usedBlocksTimeOrdered.get(1000L).size() shouldEqual 3 + blockManager.usedIngestionBlocks.size() shouldEqual 0 + blockManager.usedOdpBlocks.size() shouldEqual 0 - blockManager.requestBlock(Some(9000L)).map(_.markReclaimable).isDefined shouldEqual true - blockManager.usedBlocksTimeOrdered.get(9000L).size() shouldEqual 1 + // first allocate 1 regular block + blockManager.requestBlock(false).isDefined shouldEqual true + blockManager.usedIngestionBlocks.size shouldEqual 1 - blockManager.numTimeOrderedBlocks shouldEqual 4 - blockManager.usedBlocksTimeOrdered.size shouldEqual 2 + // first allocate 4 odp blocks + blockManager.requestBlock(true).isDefined shouldEqual true + blockManager.requestBlock(true).isDefined shouldEqual true + blockManager.requestBlock(true).isDefined shouldEqual true + blockManager.requestBlock(true).isDefined shouldEqual true + blockManager.usedOdpBlocks.size() shouldEqual 4 - // reclaim from time ordered blocks should fail now + // reclaim should fail now because none of the blocks are reclaimable try { - blockManager.requestBlock(Some(10000L)) + blockManager.requestBlock(true) fail } catch { case e: MemoryRequestException => // expected @@ -211,34 +206,29 @@ class PageAlignedBlockManagerSpec extends AnyFlatSpec with Matchers with BeforeA // This block manager has 5 blocks capacity val blockManager = new PageAlignedBlockManager(5 * pageSize, stats, testReclaimer, 1) - blockManager.usedBlocks.size() shouldEqual 0 - blockManager.numTimeOrderedBlocks shouldEqual 0 - blockManager.usedBlocksTimeOrdered.size shouldEqual 0 + blockManager.usedIngestionBlocks.size() shouldEqual 0 + blockManager.usedOdpBlocks.size() shouldEqual 0 - val factory = new BlockMemFactory(blockManager, Some(10000L), 24, Map("foo" -> "bar"), false) + val factory = new BlockMemFactory(blockManager, 24, Map("foo" -> "bar"), true) // There should be one time ordered block allocated, owned by factory - blockManager.usedBlocks.size shouldEqual 0 - blockManager.numTimeOrderedBlocks shouldEqual 1 - blockManager.hasTimeBucket(10000L) shouldEqual true + blockManager.usedIngestionBlocks.size shouldEqual 0 + blockManager.usedOdpBlocks.size() shouldEqual 1 factory.currentBlock.owner shouldEqual Some(factory) // Now allocate 4 more regular blocks, that will use up all blocks - blockManager.requestBlock(None).isDefined shouldEqual true - blockManager.requestBlock(None).isDefined shouldEqual true - blockManager.requestBlock(None).isDefined shouldEqual true - blockManager.requestBlock(None).isDefined shouldEqual true - blockManager.usedBlocks.size shouldEqual 4 - blockManager.numTimeOrderedBlocks shouldEqual 1 + blockManager.requestBlock(false).isDefined shouldEqual true + blockManager.requestBlock(false).isDefined shouldEqual true + blockManager.requestBlock(false).isDefined shouldEqual true + blockManager.requestBlock(false).isDefined shouldEqual true + blockManager.usedIngestionBlocks.size shouldEqual 4 + blockManager.usedOdpBlocks.size() shouldEqual 1 // Mark as reclaimable the blockMemFactory's block. Then request more blocks, that one will be reclaimed. // Check ownership is now cleared. factory.currentBlock.markReclaimable blockManager.ensureFreeBlocks(1) - blockManager.requestBlock(Some(9000L)).isDefined shouldEqual true - blockManager.hasTimeBucket(10000L) shouldEqual false - blockManager.hasTimeBucket(9000L) shouldEqual true factory.currentBlock.owner shouldEqual None // new requestor did not have owner } @@ -252,23 +242,23 @@ class PageAlignedBlockManagerSpec extends AnyFlatSpec with Matchers with BeforeA blockManager.ensureFreePercent(50) blockManager.numFreeBlocks shouldEqual 5 - blockManager.requestBlock(None).map(_.markReclaimable).isDefined shouldEqual true + blockManager.requestBlock(false).map(_.markReclaimable).isDefined shouldEqual true blockManager.numFreeBlocks shouldEqual 4 blockManager.ensureFreePercent(50) blockManager.numFreeBlocks shouldEqual 4 - blockManager.requestBlock(None).map(_.markReclaimable).isDefined shouldEqual true + blockManager.requestBlock(false).map(_.markReclaimable).isDefined shouldEqual true blockManager.numFreeBlocks shouldEqual 3 blockManager.ensureFreePercent(50) blockManager.numFreeBlocks shouldEqual 3 - blockManager.requestBlock(None).map(_.markReclaimable).isDefined shouldEqual true + blockManager.requestBlock(false).map(_.markReclaimable).isDefined shouldEqual true blockManager.numFreeBlocks shouldEqual 2 blockManager.ensureFreePercent(50) // Should actually have done something this time. blockManager.numFreeBlocks shouldEqual 3 - blockManager.requestBlock(None).map(_.markReclaimable).isDefined shouldEqual true + blockManager.requestBlock(false).map(_.markReclaimable).isDefined shouldEqual true blockManager.numFreeBlocks shouldEqual 2 blockManager.ensureFreePercent(90) // Should reclaim multiple blocks. diff --git a/memory/src/test/scala/filodb.memory/format/vectors/IntBinaryVectorTest.scala b/memory/src/test/scala/filodb.memory/format/vectors/IntBinaryVectorTest.scala index 795f4f1ab6..c24c7d8ce9 100644 --- a/memory/src/test/scala/filodb.memory/format/vectors/IntBinaryVectorTest.scala +++ b/memory/src/test/scala/filodb.memory/format/vectors/IntBinaryVectorTest.scala @@ -119,7 +119,7 @@ class IntBinaryVectorTest extends NativeVectorTest { new MemoryStats(Map("test"-> "test")), null, 16) { freeBlocks.asScala.foreach(_.set(0x55)) // initialize blocks to nonzero value } - val blockFactory = new BlockMemFactory(blockStore, None, 24, Map("foo" -> "bar"), true) + val blockFactory = new BlockMemFactory(blockStore, 24, Map("foo" -> "bar"), true) // original values will get mixed with nonzero contents if append does not overwrite original memory val builder = IntBinaryVector.appendingVectorNoNA(blockFactory, 10, nbits = 4, signed = false) diff --git a/project/Dependencies.scala b/project/Dependencies.scala index 53c70769ed..809f30edf9 100644 --- a/project/Dependencies.scala +++ b/project/Dependencies.scala @@ -71,7 +71,10 @@ object Dependencies { "com.github.rholder.fauxflake" % "fauxflake-core" % "1.1.0", "org.scalactic" %% "scalactic" % "3.2.0" withJavadoc(), "org.apache.lucene" % "lucene-core" % "7.3.0" withJavadoc(), - "com.github.alexandrnikitin" %% "bloom-filter" % "0.11.0" + "com.github.alexandrnikitin" %% "bloom-filter" % "0.11.0", + "org.rocksdb" % "rocksdbjni" % "6.11.4", + "com.esotericsoftware" % "kryo" % "4.0.0" excludeAll(excludeMinlog), + "com.dorkbox" % "MinLog-SLF4J" % "1.12" ) lazy val sparkJobsDeps = commonDeps ++ Seq( diff --git a/prometheus/src/main/scala/filodb/prometheus/query/PrometheusModel.scala b/prometheus/src/main/scala/filodb/prometheus/query/PrometheusModel.scala index c89cbebcbc..4032228a0a 100644 --- a/prometheus/src/main/scala/filodb/prometheus/query/PrometheusModel.scala +++ b/prometheus/src/main/scala/filodb/prometheus/query/PrometheusModel.scala @@ -7,6 +7,7 @@ import filodb.core.metadata.Column.ColumnType import filodb.core.metadata.PartitionSchema import filodb.core.query.{ColumnFilter, ColumnInfo, Filter, RangeVector, RangeVectorKey} import filodb.query.{QueryResult => FiloQueryResult, _} +import filodb.query.AggregationOperator.Avg import filodb.query.exec.{ExecPlan, HistToPromSeriesMapper} object PrometheusModel { @@ -152,6 +153,34 @@ object PrometheusModel { } } + def toAvgResult(srv: RangeVector, + verbose: Boolean, + typ: QueryResultType, + processMultiPartition: Boolean = true): Result = { + val tags = srv.key.labelValues.map { case (k, v) => (k.toString, v.toString)} ++ + (if (verbose) makeVerboseLabels(srv.key) + else Map.empty) + val samples = srv.rows.map { r => AvgSampl(r.getLong(0)/1000, r.getDouble(1), + r.getLong(2)) + }.toSeq + + Result(tags, None, None, Some(AggregateResponse(Avg.entryName, samples))) + } + + def toStdValResult(srv: RangeVector, + verbose: Boolean, + typ: QueryResultType, + processMultiPartition: Boolean = true): Result = { + val tags = srv.key.labelValues.map { case (k, v) => (k.toString, v.toString)} ++ + (if (verbose) makeVerboseLabels(srv.key) + else Map.empty) + val samples = srv.rows.map { r => StdValSampl(r.getLong(0)/1000, r.getDouble(1), + r.getDouble(2), r.getLong(3)) + }.toSeq + + Result(tags, None, None, Some(AggregateResponse(QueryFunctionConstants.stdVal, samples))) + } + def makeVerboseLabels(rvk: RangeVectorKey): Map[String, String] = { Map("_shards_" -> rvk.sourceShards.mkString(","), "_partIds_" -> rvk.partIds.mkString(","), diff --git a/query/src/main/scala/filodb/query/PlanEnums.scala b/query/src/main/scala/filodb/query/PlanEnums.scala index 4d0fe5bc74..87aaede047 100644 --- a/query/src/main/scala/filodb/query/PlanEnums.scala +++ b/query/src/main/scala/filodb/query/PlanEnums.scala @@ -82,6 +82,10 @@ object FiloFunctionId extends Enum[FiloFunctionId] { sealed abstract class AggregationOperator(override val entryName: String) extends EnumEntry +object QueryFunctionConstants { + val stdVal = "stdval" +} + object AggregationOperator extends Enum[AggregationOperator] { val values = findValues diff --git a/query/src/main/scala/filodb/query/PromCirceSupport.scala b/query/src/main/scala/filodb/query/PromCirceSupport.scala index 99a4d09da4..35434350ad 100644 --- a/query/src/main/scala/filodb/query/PromCirceSupport.scala +++ b/query/src/main/scala/filodb/query/PromCirceSupport.scala @@ -3,6 +3,8 @@ package filodb.query import io.circe.{Decoder, Encoder, HCursor, Json} import io.circe.syntax._ +import filodb.query.AggregationOperator.Avg + object PromCirceSupport { import cats.syntax.either._ // necessary to encode sample in promql response as an array with long and double value as string @@ -13,6 +15,29 @@ object PromCirceSupport { case m @ MetadataSampl(v) => Json.fromValues(Seq(v.asJson)) } + implicit val decodeAvgSample: Decoder[AvgSampl] = new Decoder[AvgSampl] { + final def apply(c: HCursor): Decoder.Result[AvgSampl] = { + for { timestamp <- c.downArray.as[Long] + value <- c.downArray.right.as[String] + count <- c.downArray.right.right.as[Long] + } yield { + AvgSampl(timestamp, value.toDouble, count) + } + } + } + + implicit val decodeStdValSampl: Decoder[StdValSampl] = new Decoder[StdValSampl] { + final def apply(c: HCursor): Decoder.Result[StdValSampl] = { + for {timestamp <- c.downArray.as[Long] + stddev <- c.downArray.right.as[String] + mean <- c.downArray.right.right.as[String] + count <- c.downArray.right.right.right.as[Long] + } yield { + StdValSampl(timestamp, stddev.toDouble, mean.toDouble, count) + } + } + } + implicit val decodeFoo: Decoder[DataSampl] = new Decoder[DataSampl] { final def apply(c: HCursor): Decoder.Result[DataSampl] = { val tsResult = c.downArray.as[Long] @@ -30,4 +55,22 @@ object PromCirceSupport { } } } + + implicit val decodeAggregate: Decoder[AggregateResponse] = new Decoder[AggregateResponse] { + final def apply(c: HCursor): Decoder.Result[AggregateResponse] = { + val functionName = c.downField("function").as[String] match { + case Right(fn) => fn + case Left(ex) => throw ex + } + + val aggregateSamples = functionName match { + case Avg.entryName => c.downField ("aggregateValues").as[List[AvgSampl]] + case QueryFunctionConstants.stdVal => c.downField("aggregateValues").as[List[StdValSampl]] + } + + for { + sample <- aggregateSamples + } yield AggregateResponse(functionName, sample) + } + } } diff --git a/query/src/main/scala/filodb/query/PromQueryResponse.scala b/query/src/main/scala/filodb/query/PromQueryResponse.scala index 5b3ad39aca..e8dd842cef 100644 --- a/query/src/main/scala/filodb/query/PromQueryResponse.scala +++ b/query/src/main/scala/filodb/query/PromQueryResponse.scala @@ -15,10 +15,15 @@ final case class Data(resultType: String, result: Seq[Result]) final case class MetadataSuccessResponse(data: Seq[Map[String, String]], status: String = "success") extends PromQueryResponse -final case class Result(metric: Map[String, String], values: Option[Seq[DataSampl]], value: Option[DataSampl] = None) +final case class Result(metric: Map[String, String], values: Option[Seq[DataSampl]], value: Option[DataSampl] = None, + aggregateResponse: Option[AggregateResponse] = None) sealed trait DataSampl +sealed trait AggregateSampl + +case class AggregateResponse(function: String, aggregateSampl: Seq[AggregateSampl]) + /** * Metric value for a given timestamp * @param timestamp in seconds since epoch @@ -29,3 +34,7 @@ final case class Sampl(timestamp: Long, value: Double) extends DataSampl final case class HistSampl(timestamp: Long, buckets: Map[String, Double]) extends DataSampl final case class MetadataSampl(values: Map[String, String]) extends DataSampl + +final case class AvgSampl(timestamp: Long, value: Double, count: Long) extends AggregateSampl + +final case class StdValSampl(timestamp: Long, stddev: Double, mean: Double, count: Long) extends AggregateSampl \ No newline at end of file diff --git a/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala b/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala index 5825fc297a..050046ec2d 100644 --- a/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala +++ b/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala @@ -1,7 +1,5 @@ package filodb.query.exec -import scala.collection.mutable.ArrayBuffer - import com.typesafe.scalalogging.StrictLogging import monix.eval.Task import monix.reactive.Observable @@ -36,11 +34,10 @@ trait ReduceAggregateExec extends NonLeafExecPlan { yield { val aggregator = RowAggregator(aggrOp, aggrParams, schema) RangeVectorAggregator.mapReduce(aggregator, skipMapPhase = true, results, rv => rv.key, - querySession.qContext.groupByCardLimit) + querySession.qContext.plannerParams.groupByCardLimit) } Observable.fromTask(task).flatten } - } /** @@ -104,11 +101,11 @@ final case class AggregateMapReduce(aggrOp: AggregationOperator, RangeVectorAggregator.fastReduce(aggregator, false, source, numWindows) }.getOrElse { RangeVectorAggregator.mapReduce(aggregator, skipMapPhase = false, source, grouping, - querySession.qContext.groupByCardLimit) + querySession.qContext.plannerParams.groupByCardLimit) } } else { RangeVectorAggregator.mapReduce(aggregator, skipMapPhase = false, source, grouping, - querySession.qContext.groupByCardLimit) + querySession.qContext.plannerParams.groupByCardLimit) } } @@ -121,6 +118,7 @@ final case class AggregateMapReduce(aggrOp: AggregationOperator, final case class AggregatePresenter(aggrOp: AggregationOperator, aggrParams: Seq[Any], + rangeParams: RangeParams, funcParams: Seq[FuncArgs] = Nil) extends RangeVectorTransformer { protected[exec] def args: String = s"aggrOp=$aggrOp, aggrParams=$aggrParams" @@ -131,7 +129,7 @@ final case class AggregatePresenter(aggrOp: AggregationOperator, sourceSchema: ResultSchema, paramResponse: Seq[Observable[ScalarRangeVector]]): Observable[RangeVector] = { val aggregator = RowAggregator(aggrOp, aggrParams, sourceSchema) - RangeVectorAggregator.present(aggregator, source, limit) + RangeVectorAggregator.present(aggregator, source, limit, rangeParams) } override def schema(source: ResultSchema): ResultSchema = { @@ -185,8 +183,9 @@ object RangeVectorAggregator extends StrictLogging { */ def present(aggregator: RowAggregator, source: Observable[RangeVector], - limit: Int): Observable[RangeVector] = { - source.flatMap(rv => Observable.fromIterable(aggregator.present(rv, limit))) + limit: Int, + rangeParams: RangeParams): Observable[RangeVector] = { + source.flatMap(rv => Observable.fromIterable(aggregator.present(rv, limit, rangeParams))) } private def mapReduceInternal(rvs: List[RangeVector], @@ -236,8 +235,6 @@ object RangeVectorAggregator extends StrictLogging { // Can't use an Array here because rowAgg.AggHolderType does not have a ClassTag val accs = collection.mutable.ArrayBuffer.fill(outputLen)(rowAgg.zero) var count = 0 - // keeps track of all iters to close - val toClose = ArrayBuffer.empty[RangeVectorCursor] // FoldLeft means we create the source PeriodicMapper etc and process immediately. We can release locks right away // NOTE: ChunkedWindowIterator automatically releases locks after last window. So it should all just work. :) @@ -245,9 +242,12 @@ object RangeVectorAggregator extends StrictLogging { source.foldLeftF(accs) { case (_, rv) => count += 1 val rowIter = rv.rows - toClose += rowIter - cforRange { 0 until outputLen } { i => - accs(i) = rowAgg.reduceAggregate(accs(i), rowIter.next) + try { + cforRange { 0 until outputLen } { i => + accs(i) = rowAgg.reduceAggregate(accs(i), rowIter.next) + } + } finally { + rowIter.close() } accs } @@ -256,19 +256,24 @@ object RangeVectorAggregator extends StrictLogging { source.foldLeftF(accs) { case (_, rv) => count += 1 val rowIter = rv.rows - toClose += rowIter - cforRange { 0 until outputLen } { i => - val mapped = rowAgg.map(rv.key, rowIter.next, mapIntos(i)) - accs(i) = rowAgg.reduceMappedRow(accs(i), mapped) + try { + cforRange { 0 until outputLen } { i => + val mapped = rowAgg.map(rv.key, rowIter.next, mapIntos(i)) + accs(i) = rowAgg.reduceMappedRow(accs(i), mapped) + } + } finally { + rowIter.close() } accs } } + // convert the aggregations to range vectors aggObs.flatMap { _ => if (count > 0) { - val iter = new CustomCloseCursor(accs.toIterator.map(_.toRowReader))(toClose.foreach(_.close())) - Observable.now(IteratorBackedRangeVector(CustomRangeVectorKey.empty, iter)) + import NoCloseCursor._ // The base range vectors are already closed, so no close propagation needed + Observable.now(IteratorBackedRangeVector(CustomRangeVectorKey.empty, + NoCloseCursor(accs.toIterator.map(_.toRowReader)))) } else { Observable.empty } diff --git a/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala b/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala index d4cc446df4..ba992a2317 100644 --- a/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala +++ b/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala @@ -69,9 +69,9 @@ final case class BinaryJoinExec(queryContext: QueryContext, val span = Kamon.currentSpan() val taskOfResults = childResponses.map { case (QueryResult(_, _, result), _) - if (result.size > queryContext.joinQueryCardLimit && cardinality == Cardinality.OneToOne) => - throw new BadQueryException(s"This query results in more than ${queryContext.joinQueryCardLimit} " + - s"join cardinality. Try applying more filters.") + if (result.size > queryContext.plannerParams.joinQueryCardLimit && cardinality == Cardinality.OneToOne) => + throw new BadQueryException(s"This query results in more than ${queryContext.plannerParams.joinQueryCardLimit}" + + s" join cardinality. Try applying more filters.") case (QueryResult(_, _, result), i) => (result, i) case (QueryError(_, ex), _) => throw ex }.toListL.map { resp => @@ -113,9 +113,9 @@ final case class BinaryJoinExec(queryContext: QueryContext, resultKeySet.add(resKey) // OneToOne cardinality case is already handled. this condition handles OneToMany case - if (resultKeySet.size > queryContext.joinQueryCardLimit) - throw new BadQueryException(s"This query results in more than ${queryContext.joinQueryCardLimit} " + - s"join cardinality. Try applying more filters.") + if (resultKeySet.size > queryContext.plannerParams.joinQueryCardLimit) + throw new BadQueryException(s"This query results in more than ${queryContext.plannerParams. + joinQueryCardLimit} " + s"join cardinality. Try applying more filters.") val res = if (lhsIsOneSide) binOp(rvOne.rows, rvOther.rows) else binOp(rvOther.rows, rvOne.rows) IteratorBackedRangeVector(resKey, res) diff --git a/query/src/main/scala/filodb/query/exec/ExecPlan.scala b/query/src/main/scala/filodb/query/exec/ExecPlan.scala index c153fefe58..ef3927c9a5 100644 --- a/query/src/main/scala/filodb/query/exec/ExecPlan.scala +++ b/query/src/main/scala/filodb/query/exec/ExecPlan.scala @@ -99,27 +99,26 @@ trait ExecPlan extends QueryCommand { val startExecute = querySession.qContext.submitTime - val parentSpan = Kamon.currentSpan() + val span = Kamon.currentSpan() // NOTE: we launch the preparatory steps as a Task too. This is important because scanPartitions, // Lucene index lookup, and On-Demand Paging orchestration work could suck up nontrivial time and // we don't want these to happen in a single thread. // Step 1: initiate doExecute: make result schema and set up the async monix pipeline to create RVs lazy val step1 = Task { - val span = Kamon.spanBuilder(s"execute-step1-${getClass.getSimpleName}") - .asChildOf(parentSpan) - .tag("query-id", queryContext.queryId) - .start() + span.mark(s"execute-step1-start-${getClass.getSimpleName}") FiloSchedulers.assertThreadName(QuerySchedName) // Please note that the following needs to be wrapped inside `runWithSpan` so that the context will be propagated // across threads. Note that task/observable will not run on the thread where span is present since // kamon uses thread-locals. - Kamon.runWithSpan(span, true) { + // Dont finish span since this code didnt create it + Kamon.runWithSpan(span, false) { val doEx = doExecute(source, querySession) Kamon.histogram("query-execute-time-elapsed-step1-done", MeasurementUnit.time.milliseconds) .withTag("plan", getClass.getSimpleName) .record(Math.max(0, System.currentTimeMillis - startExecute)) + span.mark(s"execute-step1-end-${getClass.getSimpleName}") doEx } } @@ -129,22 +128,19 @@ trait ExecPlan extends QueryCommand { Kamon.histogram("query-execute-time-elapsed-step2-start", MeasurementUnit.time.milliseconds) .withTag("plan", getClass.getSimpleName) .record(Math.max(0, System.currentTimeMillis - startExecute)) - val span = Kamon.spanBuilder(s"execute-step2-${getClass.getSimpleName}") - .asChildOf(parentSpan) - .tag("query-id", queryContext.queryId) - .start() + span.mark(s"execute-step2-start-${getClass.getSimpleName}") FiloSchedulers.assertThreadName(QuerySchedName) val dontRunTransformers = if (allTransformers.isEmpty) true else !allTransformers.forall(_.canHandleEmptySchemas) - span.tag("dontRunTransformers", dontRunTransformers) // It is possible a null schema is returned (due to no time series). In that case just return empty results val resultTask = if (resSchema == ResultSchema.empty && dontRunTransformers) { qLogger.debug(s"queryId: ${queryContext.queryId} Empty plan $this, returning empty results") span.mark("empty-plan") + span.mark(s"execute-step2-end-${getClass.getSimpleName}") Task.eval(QueryResult(queryContext.queryId, resSchema, Nil)) } else { val finalRes = allTransformers.foldLeft((res.rvs, resSchema)) { (acc, transf) => val paramRangeVector: Seq[Observable[ScalarRangeVector]] = transf.funcParams.map(_.getResult) - (transf.apply(acc._1, querySession, queryContext.sampleLimit, acc._2, + (transf.apply(acc._1, querySession, queryContext.plannerParams.sampleLimit, acc._2, paramRangeVector), transf.schema(acc._2)) } val recSchema = SerializedRangeVector.toSchema(finalRes._2.columns, finalRes._2.brSchemas) @@ -152,7 +148,6 @@ trait ExecPlan extends QueryCommand { MeasurementUnit.time.milliseconds) .withTag("plan", getClass.getSimpleName) .record(Math.max(0, System.currentTimeMillis - startExecute)) - span.mark("step2-transformer-pipeline-setup") val builder = SerializedRangeVector.newBuilder() @volatile var numResultSamples = 0 // BEWARE - do not modify concurrently!! finalRes._1 @@ -161,18 +156,18 @@ trait ExecPlan extends QueryCommand { case srv: SerializableRangeVector => numResultSamples += srv.numRowsInt // fail the query instead of limiting range vectors and returning incomplete/inaccurate results - if (enforceLimit && numResultSamples > queryContext.sampleLimit) - throw new BadQueryException(s"This query results in more than ${queryContext.sampleLimit} samples. " + - s"Try applying more filters or reduce time range.") + if (enforceLimit && numResultSamples > queryContext.plannerParams.sampleLimit) + throw new BadQueryException(s"This query results in more than ${queryContext.plannerParams. + sampleLimit} samples.Try applying more filters or reduce time range.") srv case rv: RangeVector => // materialize, and limit rows per RV - val srv = SerializedRangeVector(rv, builder, recSchema) + val srv = SerializedRangeVector(rv, builder, recSchema, queryWithPlanName(queryContext)) numResultSamples += srv.numRowsInt // fail the query instead of limiting range vectors and returning incomplete/inaccurate results - if (enforceLimit && numResultSamples > queryContext.sampleLimit) - throw new BadQueryException(s"This query results in more than ${queryContext.sampleLimit} samples. " + - s"Try applying more filters or reduce time range.") + if (enforceLimit && numResultSamples > queryContext.plannerParams.sampleLimit) + throw new BadQueryException(s"This query results in more than ${queryContext.plannerParams. + sampleLimit} samples. Try applying more filters or reduce time range.") srv } .doOnTerminate(_ => span.mark("after-last-materialized-result-rv")) @@ -190,26 +185,20 @@ trait ExecPlan extends QueryCommand { // 250 RVs * (250 bytes for RV-Key + 200 samples * 32 bytes per sample) // is < 2MB qLogger.warn(s"queryId: ${queryContext.queryId} result was large size $numBytes. May need to " + - s"tweak limits. ExecPlan was: ${printTree()} ; Limit was: ${queryContext.sampleLimit}") + s"tweak limits. Query was: ${queryContext.origQueryParams}" + + s"; Limit was: ${queryContext.plannerParams.sampleLimit}") } span.mark(s"num-result-samples: $numResultSamples") span.mark(s"num-range-vectors: ${r.size}") - span.finish() + span.mark(s"execute-step2-end-${getClass.getSimpleName}") QueryResult(queryContext.queryId, finalRes._2, r) } } resultTask.onErrorHandle { case ex: Throwable => - if (!ex.isInstanceOf[BadQueryException]) // dont log user errors - qLogger.error(s"queryId: ${queryContext.queryId} Exception during execution of query: " + - s"${printTree(false)}", ex) - span.fail(ex) QueryError(queryContext.queryId, ex) } }.flatten .onErrorRecover { case NonFatal(ex) => - if (!ex.isInstanceOf[BadQueryException]) // dont log user errors - qLogger.error(s"queryId: ${queryContext.queryId} Exception during orchestration of query:" + - s" ${printTree(false)}", ex) QueryError(queryContext.queryId, ex) } @@ -250,6 +239,10 @@ trait ExecPlan extends QueryCommand { ((transf :+ curNode) ++ childr).mkString(if (useNewline) "\n" else " @@@ ") } + protected def queryWithPlanName(queryContext: QueryContext): String = { + s"${this.getClass.getSimpleName}-${queryContext.origQueryParams}" + } + def curNodeText(level: Int): String = s"${"-"*level}E~${getClass.getSimpleName}($args) on ${dispatcher}" @@ -318,21 +311,19 @@ final case class ExecPlanFuncArgs(execPlan: ExecPlan, timeStepParams: RangeParam override def getResult(implicit sched: Scheduler): Observable[ScalarRangeVector] = { Observable.fromTask( execPlan.dispatcher.dispatch(execPlan).onErrorHandle { case ex: Throwable => - qLogger.error(s"queryId: ${execPlan.queryContext.queryId} Execution failed for sub-query" + - s" ${execPlan.printTree()}", ex) - QueryError(execPlan.queryContext.queryId, ex) - }.map { - case QueryResult(_, _, result) => // Result is empty because of NaN so create ScalarFixedDouble with NaN - if (result.isEmpty) { - ScalarFixedDouble(timeStepParams, Double.NaN) - } else { - result.head match { - case f: ScalarFixedDouble => f - case s: ScalarVaryingDouble => s + QueryError(execPlan.queryContext.queryId, ex) + }.map { + case QueryResult(_, _, result) => // Result is empty because of NaN so create ScalarFixedDouble with NaN + if (result.isEmpty) { + ScalarFixedDouble(timeStepParams, Double.NaN) + } else { + result.head match { + case f: ScalarFixedDouble => f + case s: ScalarVaryingDouble => s + } } - } - case QueryError(_, ex) => throw ex - }) + case QueryError(_, ex) => throw ex + }) } override def toString: String = execPlan.printTree() + "\n" @@ -374,9 +365,9 @@ abstract class NonLeafExecPlan extends ExecPlan { // Please note that the following needs to be wrapped inside `runWithSpan` so that the context will be propagated // across threads. Note that task/observable will not run on the thread where span is present since // kamon uses thread-locals. + // Dont finish span since this code didnt create it Kamon.runWithSpan(span, false) { plan.dispatcher.dispatch(plan).onErrorHandle { case ex: Throwable => - qLogger.error(s"queryId: ${queryContext.queryId} Execution failed for sub-query ${plan.printTree()}", ex) QueryError(queryContext.queryId, ex) } } @@ -392,13 +383,9 @@ abstract class NonLeafExecPlan extends ExecPlan { final def doExecute(source: ChunkSource, querySession: QuerySession) (implicit sched: Scheduler): ExecResult = { - val parentSpan = Kamon.currentSpan() - - val span = Kamon.spanBuilder(s"execute-step1-child-result-composition-${getClass.getSimpleName}") - .asChildOf(parentSpan) - .tag("query-id", queryContext.queryId) - .start() + val span = Kamon.currentSpan() + span.mark(s"execute-step1-child-result-composition-start-${getClass.getSimpleName}") // whether child tasks need to be executed sequentially. // parallelism 1 means, only one worker thread to process underlying tasks. val parallelism: Int = if (parallelChildTasks) @@ -411,7 +398,7 @@ abstract class NonLeafExecPlan extends ExecPlan { val childTasks = Observable.fromIterable(children.zipWithIndex) .mapAsync(parallelism) { case (plan, i) => val task = dispatchRemotePlan(plan, span).map((_, i)) - span.mark(s"plan-dispatched-${plan.getClass.getSimpleName}") + span.mark(s"child-plan-$i-dispatched-${plan.getClass.getSimpleName}") task } @@ -433,9 +420,10 @@ abstract class NonLeafExecPlan extends ExecPlan { val outputSchema = processedTasks.collect { case (QueryResult(_, schema, _), _) => schema }.firstOptionL.map(_.getOrElse(ResultSchema.empty)) + // Dont finish span since this code didnt create it Kamon.runWithSpan(span, false) { val outputRvs = compose(processedTasks, outputSchema, querySession) - .doOnTerminate(_ => span.finish()) + .doOnTerminate(_ => span.mark(s"execute-step1-child-result-composition-end-${getClass.getSimpleName}")) ExecResult(outputRvs, outputSchema) } } diff --git a/query/src/main/scala/filodb/query/exec/InProcessPlanDispatcher.scala b/query/src/main/scala/filodb/query/exec/InProcessPlanDispatcher.scala index d760603dd0..d9aa78f688 100644 --- a/query/src/main/scala/filodb/query/exec/InProcessPlanDispatcher.scala +++ b/query/src/main/scala/filodb/query/exec/InProcessPlanDispatcher.scala @@ -7,6 +7,7 @@ import monix.reactive.Observable import filodb.core.{DatasetRef, Types} import filodb.core.memstore.PartLookupResult +import filodb.core.memstore.ratelimit.CardinalityRecord import filodb.core.metadata.Schemas import filodb.core.query.{EmptyQueryConfig, QueryConfig, QuerySession} import filodb.core.store._ @@ -29,13 +30,13 @@ case object InProcessPlanDispatcher extends PlanDispatcher { // Please note that the following needs to be wrapped inside `runWithSpan` so that the context will be propagated // across threads. Note that task/observable will not run on the thread where span is present since // kamon uses thread-locals. + // Dont finish span since this code didnt create it Kamon.runWithSpan(Kamon.currentSpan(), false) { // translate implicit ExecutionContext to monix.Scheduler val querySession = QuerySession(plan.queryContext, queryConfig) plan.execute(source, querySession) } } - } /** @@ -70,10 +71,13 @@ case class UnsupportedChunkSource() extends ChunkSource { chunkMethod: ChunkScanMethod): Observable[RawPartData] = throw new UnsupportedOperationException("This operation is not supported") - /** - * True if this store is in the mode of serving downsampled data. - * This is used to switch ingestion and query behaviors for downsample cluster. - */ override def isDownsampleStore: Boolean = false + + override def topKCardinality(ref: DatasetRef, + shards: Seq[Int], + shardKeyPrefix: scala.Seq[String], + k: Int): scala.Seq[CardinalityRecord] = + throw new UnsupportedOperationException("This operation is not supported") + } diff --git a/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala b/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala index 47f2b112e2..839b998408 100644 --- a/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala +++ b/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala @@ -84,7 +84,7 @@ final case class PartKeysExec(queryContext: QueryContext, val rvs = source match { case memStore: MemStore => val response = memStore.partKeysWithFilters(dataset, shard, filters, - fetchFirstLastSampleTimes, end, start, queryContext.sampleLimit) + fetchFirstLastSampleTimes, end, start, queryContext.plannerParams.sampleLimit) import NoCloseCursor._ Observable.now(IteratorBackedRangeVector( new CustomRangeVectorKey(Map.empty), UTF8MapIteratorRowReader(response))) @@ -95,7 +95,7 @@ final case class PartKeysExec(queryContext: QueryContext, ExecResult(rvs, Task.eval(sch)) } - def args: String = s"shard=$shard, filters=$filters, limit=${queryContext.sampleLimit}" + def args: String = s"shard=$shard, filters=$filters, limit=${queryContext.plannerParams.sampleLimit}" } final case class LabelValuesExec(queryContext: QueryContext, @@ -117,13 +117,12 @@ final case class LabelValuesExec(queryContext: QueryContext, val memStore = source.asInstanceOf[MemStore] val response = filters.isEmpty match { // retrieves label values for a single label - no column filter - case true if (columns.size == 1) => memStore.labelValues(dataset, shard, columns.head, queryContext.sampleLimit) - .map(termInfo => Map(columns.head.utf8 -> termInfo.term)) - .toIterator + case true if (columns.size == 1) => memStore.labelValues(dataset, shard, columns.head, queryContext. + plannerParams.sampleLimit).map(termInfo => Map(columns.head.utf8 -> termInfo.term)).toIterator case true => throw new BadQueryException("either label name is missing " + "or there are multiple label names without filter") case false => memStore.labelValuesWithFilters(dataset, shard, filters, columns, endMs, startMs, - queryContext.sampleLimit) + queryContext.plannerParams.sampleLimit) } import NoCloseCursor._ Observable.now(IteratorBackedRangeVector(new CustomRangeVectorKey(Map.empty), @@ -135,6 +134,6 @@ final case class LabelValuesExec(queryContext: QueryContext, ExecResult(rvs, Task.eval(sch)) } - def args: String = s"shard=$shard, filters=$filters, col=$columns, limit=${queryContext.sampleLimit}, " + - s"startMs=$startMs, endMs=$endMs" + def args: String = s"shard=$shard, filters=$filters, col=$columns, limit=${queryContext.plannerParams.sampleLimit}," + + s" startMs=$startMs, endMs=$endMs" } diff --git a/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala b/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala index 5b54519f1f..477f08cd8b 100644 --- a/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala +++ b/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala @@ -1,9 +1,9 @@ package filodb.query.exec -import kamon.Kamon +import scala.concurrent.Future + import kamon.trace.Span import monix.execution.Scheduler -import scala.concurrent.Future import filodb.core.DatasetRef import filodb.core.metadata.Column.ColumnType @@ -18,7 +18,7 @@ case class MetadataRemoteExec(queryEndpoint: String, queryContext: QueryContext, dispatcher: PlanDispatcher, dataset: DatasetRef, - params: PromQlQueryParams) extends RemoteExec { + remoteExecHttpClient: RemoteExecHttpClient) extends RemoteExec { private val columns = Seq(ColumnInfo("Labels", ColumnType.MapColumn)) private val resultSchema = ResultSchema(columns, 1) @@ -27,7 +27,8 @@ case class MetadataRemoteExec(queryEndpoint: String, override def sendHttpRequest(execPlan2Span: Span, httpTimeoutMs: Long) (implicit sched: Scheduler): Future[QueryResponse] = { - PromRemoteExec.httpMetadataGet(queryEndpoint, httpTimeoutMs, queryContext.submitTime, getUrlParams()) + remoteExecHttpClient.httpMetadataGet(queryContext.plannerParams.applicationId, queryEndpoint, + httpTimeoutMs, queryContext.submitTime, getUrlParams()) .map { response => response.unsafeBody match { case Left(error) => QueryError(queryContext.queryId, error.error) @@ -37,20 +38,15 @@ case class MetadataRemoteExec(queryEndpoint: String, } def toQueryResponse(data: Seq[Map[String, String]], id: String, parentSpan: kamon.trace.Span): QueryResponse = { - val span = Kamon.spanBuilder(s"create-queryresponse-${getClass.getSimpleName}") - .asChildOf(parentSpan) - .tag("query-id", id) - .start() - val iteratorMap = data.map { r => r.map { v => (v._1.utf8, v._2.utf8) }} import NoCloseCursor._ val rangeVector = IteratorBackedRangeVector(new CustomRangeVectorKey(Map.empty), - new UTF8MapIteratorRowReader(iteratorMap.toIterator)) + UTF8MapIteratorRowReader(iteratorMap.toIterator)) - val srvSeq = Seq(SerializedRangeVector(rangeVector, builder, recordSchema)) + val srvSeq = Seq(SerializedRangeVector(rangeVector, builder, recordSchema, + queryWithPlanName(queryContext))) - span.finish() QueryResult(id, resultSchema, srvSeq) } } diff --git a/query/src/main/scala/filodb/query/exec/MultiSchemaPartitionsExec.scala b/query/src/main/scala/filodb/query/exec/MultiSchemaPartitionsExec.scala index 5b5fb12be0..e843495176 100644 --- a/query/src/main/scala/filodb/query/exec/MultiSchemaPartitionsExec.scala +++ b/query/src/main/scala/filodb/query/exec/MultiSchemaPartitionsExec.scala @@ -46,7 +46,7 @@ final case class MultiSchemaPartitionsExec(queryContext: QueryContext, Kamon.currentSpan().mark("lookup-partitions-done") val queryTimeElapsed = System.currentTimeMillis() - queryContext.submitTime - if (queryTimeElapsed >= queryContext.queryTimeoutMillis) + if (queryTimeElapsed >= queryContext.plannerParams.queryTimeoutMillis) throw QueryTimeoutException(queryTimeElapsed, this.getClass.getName) // Find the schema if one wasn't supplied diff --git a/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala b/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala index 55e51f0f25..eb97361e94 100644 --- a/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala +++ b/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala @@ -237,10 +237,12 @@ extends WrappedCursor(rv.rows()) with StrictLogging { val valReader = rv.partition.schema.data.reader(rv.valueColID, nextInfo.getValueVectorAccessor, nextInfo.getValueVectorAddr) - qLogger.error(s"addChunks Exception: info.numRows=${nextInfo.numRows} " + - s"info.endTime=${nextInfo.endTime} curWindowEnd=${wit.curWindowEnd} tsReader=$tsReader " + + qLogger.error(s"addChunks Exception: ChunkInfo=[${nextInfo.debugString}] " + + s"curWinStart=${wit.curWindowStart} curWindowEnd=${wit.curWindowEnd} tsReader=$tsReader " + s"timestampVectorLength=${tsReader.length(nextInfo.getTsVectorAccessor, nextInfo.getTsVectorAddr)} " + - s"valueVectorLength=${valReader.length(nextInfo.getValueVectorAccessor, nextInfo.getValueVectorAddr)}", e) + s"valueVectorLength=${valReader.length(nextInfo.getValueVectorAccessor, nextInfo.getValueVectorAddr)} " + + s"partition ${rv.partition.stringPartition} " + + s"start=$start end=$end step=$step", e) throw e } } diff --git a/query/src/main/scala/filodb/query/exec/PlanDispatcher.scala b/query/src/main/scala/filodb/query/exec/PlanDispatcher.scala index 2fb68aa7b8..b948ac3ac6 100644 --- a/query/src/main/scala/filodb/query/exec/PlanDispatcher.scala +++ b/query/src/main/scala/filodb/query/exec/PlanDispatcher.scala @@ -30,7 +30,7 @@ case class ActorPlanDispatcher(target: ActorRef) extends PlanDispatcher { def dispatch(plan: ExecPlan)(implicit sched: Scheduler): Task[QueryResponse] = { val queryTimeElapsed = System.currentTimeMillis() - plan.queryContext.submitTime - val remainingTime = plan.queryContext.queryTimeoutMillis - queryTimeElapsed + val remainingTime = plan.queryContext.plannerParams.queryTimeoutMillis - queryTimeElapsed // Don't send if time left is very small if (remainingTime < 1) { Task.raiseError(QueryTimeoutException(queryTimeElapsed, this.getClass.getName)) diff --git a/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala b/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala index 2b800faa29..a9cc3cbc59 100644 --- a/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala +++ b/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala @@ -1,111 +1,52 @@ package filodb.query.exec -import java.util.concurrent.TimeUnit - import scala.concurrent.Future -import scala.concurrent.duration._ -import scala.sys.ShutdownHookThread -import com.softwaremill.sttp.asynchttpclient.future.AsyncHttpClientFutureBackend -import com.softwaremill.sttp.circe._ -import com.typesafe.scalalogging.StrictLogging -import kamon.Kamon import kamon.trace.Span -import monix.eval.Task import monix.execution.Scheduler import filodb.core.DatasetRef import filodb.core.metadata.Column.ColumnType import filodb.core.query._ -import filodb.core.store.ChunkSource import filodb.memory.format.ZeroCopyUTF8String._ import filodb.memory.format.vectors.{CustomBuckets, MutableHistogram} import filodb.query._ - -trait RemoteExec extends LeafExecPlan with StrictLogging { - - val params: PromQlQueryParams - - val queryEndpoint: String - - val requestTimeoutMs: Long - - val urlParams: Map[String, Any] - - def args: String = s"${params.toString}, queryEndpoint=$queryEndpoint, " + - s"requestTimeoutMs=$requestTimeoutMs, limit=${queryContext.sampleLimit}" - - def limit: Int = ??? - - /** - * Since execute is already overrided here, doExecute() can be empty. - */ - def doExecute(source: ChunkSource, - querySession: QuerySession) - (implicit sched: Scheduler): ExecResult = ??? - - override def execute(source: ChunkSource, - querySession: QuerySession) - (implicit sched: Scheduler): Task[QueryResponse] = { - val execPlan2Span = Kamon.spanBuilder(s"execute-${getClass.getSimpleName}") - .asChildOf(Kamon.currentSpan()) - .tag("query-id", queryContext.queryId) - .start() - - if (queryEndpoint == null) { - throw new BadQueryException("Remote Query endpoint can not be null in RemoteExec.") - } - - // Please note that the following needs to be wrapped inside `runWithSpan` so that the context will be propagated - // across threads. Note that task/observable will not run on the thread where span is present since - // kamon uses thread-locals. - Kamon.runWithSpan(execPlan2Span, true) { - Task.fromFuture(sendHttpRequest(execPlan2Span, requestTimeoutMs)) - } - } - - def sendHttpRequest(execPlan2Span: Span, httpTimeoutMs: Long) - (implicit sched: Scheduler): Future[QueryResponse] - - def getUrlParams(): Map[String, Any] = { - var finalUrlParams = urlParams ++ - Map("start" -> params.startSecs, - "end" -> params.endSecs, - "time" -> params.endSecs, - "step" -> params.stepSecs, - "processFailure" -> params.processFailure, - "processMultiPartition" -> params.processMultiPartition, - "histogramMap" -> "true", - "verbose" -> params.verbose) - if (params.spread.isDefined) finalUrlParams = finalUrlParams + ("spread" -> params.spread.get) - logger.debug("URLParams for RemoteExec:" + finalUrlParams) - finalUrlParams - } - -} +import filodb.query.AggregationOperator.Avg case class PromQlRemoteExec(queryEndpoint: String, requestTimeoutMs: Long, queryContext: QueryContext, dispatcher: PlanDispatcher, dataset: DatasetRef, - params: PromQlQueryParams) extends RemoteExec { + remoteExecHttpClient: RemoteExecHttpClient) extends RemoteExec { private val defaultColumns = Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), ColumnInfo("value", ColumnType.DoubleColumn)) - private val defaultRecSchema = SerializedRangeVector.toSchema(defaultColumns) - private val defaultResultSchema = ResultSchema(defaultColumns, 1) - private val histColumns = Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), - ColumnInfo("h", ColumnType.HistogramColumn)) - private val histRecSchema = SerializedRangeVector.toSchema(histColumns) - private val histResultSchema = ResultSchema(histColumns, 1) +//TODO Don't use PromQL API to talk across clusters + val columns= Map("histogram" -> Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), + ColumnInfo("h", ColumnType.HistogramColumn)), + Avg.entryName -> (defaultColumns :+ ColumnInfo("count", ColumnType.LongColumn)) , + "default" -> defaultColumns, QueryFunctionConstants.stdVal -> (defaultColumns ++ + Seq(ColumnInfo("mean", ColumnType.DoubleColumn), ColumnInfo("count", ColumnType.LongColumn)))) + + val recordSchema = Map("histogram" -> SerializedRangeVector.toSchema(columns.get("histogram").get), + Avg.entryName -> SerializedRangeVector.toSchema(columns.get(Avg.entryName).get), + "default" -> SerializedRangeVector.toSchema(columns.get("default").get), + QueryFunctionConstants.stdVal -> SerializedRangeVector.toSchema(columns.get(QueryFunctionConstants.stdVal).get)) + + val resultSchema = Map("histogram" -> ResultSchema(columns.get("histogram").get, 1), + Avg.entryName -> ResultSchema(columns.get(Avg.entryName).get, 1), + "default" -> ResultSchema(columns.get("default").get, 1), + QueryFunctionConstants.stdVal -> ResultSchema(columns.get(QueryFunctionConstants.stdVal).get, 1)) + private val builder = SerializedRangeVector.newBuilder() - override val urlParams = Map("query" -> params.promQl) + override val urlParams = Map("query" -> promQlQueryParams.promQl) override def sendHttpRequest(execPlan2Span: Span, httpTimeoutMs: Long) (implicit sched: Scheduler): Future[QueryResponse] = { - PromRemoteExec.httpGet(queryEndpoint, requestTimeoutMs, queryContext.submitTime, getUrlParams()) + remoteExecHttpClient.httpGet(queryContext.plannerParams.applicationId, queryEndpoint, + requestTimeoutMs, queryContext.submitTime, getUrlParams()) .map { response => response.unsafeBody match { case Left(error) => QueryError(queryContext.queryId, error.error) @@ -118,29 +59,40 @@ case class PromQlRemoteExec(queryEndpoint: String, // schema. Would need to detect ahead of time to use TransientHistRow(), so we'd need to add schema to output, // and detect it in execute() above. Need to discuss compatibility issues with Prometheus. def toQueryResponse(data: Data, id: String, parentSpan: kamon.trace.Span): QueryResponse = { - val span = Kamon.spanBuilder(s"create-queryresponse-${getClass.getSimpleName}") - .asChildOf(parentSpan) - .tag("query-id", id) - .start() - val queryResponse = if (data.result.isEmpty) { logger.debug("PromQlRemoteExec generating empty QueryResult as result is empty") QueryResult(id, ResultSchema.empty, Seq.empty) } else { - val samples = data.result.head.values.getOrElse(Seq(data.result.head.value.get)) - if (samples.isEmpty) { - logger.debug("PromQlRemoteExec generating empty QueryResult as samples is empty") - QueryResult(id, ResultSchema.empty, Seq.empty) - } else { - // Passing histogramMap = true so DataSampl will be HistSampl for histograms - if (samples.head.isInstanceOf[HistSampl]) genHistQueryResult(data, id) - else genDefaultQueryResult(data, id) + if (data.result.head.aggregateResponse.isDefined) genAggregateResult(data, id) + else { + val samples = data.result.head.values.getOrElse(Seq(data.result.head.value.get)) + if (samples.isEmpty) { + logger.debug("PromQlRemoteExec generating empty QueryResult as samples is empty") + QueryResult(id, ResultSchema.empty, Seq.empty) + } else { + samples.head match { + // Passing histogramMap = true so DataSampl will be HistSampl for histograms + case HistSampl(timestamp, buckets) => genHistQueryResult(data, id) + case _ => genDefaultQueryResult(data, id) + } + } } } - span.finish() queryResponse } + def genAggregateResult(data: Data, id: String): QueryResult = { + + val aggregateResponse = data.result.head.aggregateResponse.get + if (aggregateResponse.aggregateSampl.isEmpty) QueryResult(id, ResultSchema.empty, Seq.empty) + else { + aggregateResponse.aggregateSampl.head match { + case AvgSampl(timestamp, value, count) => genAvgQueryResult(data, id) + case StdValSampl(timestamp, stddev, mean, count) => genStdValQueryResult(data, id) + } + } + } + def genDefaultQueryResult(data: Data, id: String): QueryResult = { val rangeVectors = data.result.map { r => val samples = r.values.getOrElse(Seq(r.value.get)) @@ -158,14 +110,14 @@ case class PromQlRemoteExec(queryEndpoint: String, row } } - override def numRows: Option[Int] = Option(samples.size) } - SerializedRangeVector(rv, builder, defaultRecSchema) + SerializedRangeVector(rv, builder, recordSchema.get("default").get, + queryWithPlanName(queryContext)) // TODO: Handle stitching with verbose flag } - QueryResult(id, defaultResultSchema, rangeVectors) + QueryResult(id, resultSchema.get("default").get, rangeVectors) } def genHistQueryResult(data: Data, id: String): QueryResult = { @@ -196,58 +148,62 @@ case class PromQlRemoteExec(queryEndpoint: String, override def numRows: Option[Int] = Option(samples.size) } - SerializedRangeVector(rv, builder, histRecSchema) + SerializedRangeVector(rv, builder, recordSchema.get("histogram").get, queryContext.origQueryParams.toString) // TODO: Handle stitching with verbose flag } - QueryResult(id, histResultSchema, rangeVectors) + QueryResult(id, resultSchema.get("histogram").get, rangeVectors) } -} + def genAvgQueryResult(data: Data, id: String): QueryResult = { + val rangeVectors = data.result.map { d => + val rv = new RangeVector { + val row = new AvgAggTransientRow() + + override def key: RangeVectorKey = CustomRangeVectorKey(d.metric.map(m => m._1.utf8 -> m._2.utf8)) + + override def rows(): RangeVectorCursor = { + import NoCloseCursor._ + d.aggregateResponse.get.aggregateSampl.iterator.collect { case a: AvgSampl => + row.setLong(0, a.timestamp * 1000) + row.setDouble(1, a.value) + row.setLong(2, a.count) + row + } + } + override def numRows: Option[Int] = Option(d.aggregateResponse.get.aggregateSampl.size) + } + SerializedRangeVector(rv, builder, recordSchema.get(Avg.entryName).get, + queryWithPlanName(queryContext)) + } -object PromRemoteExec extends StrictLogging { - - import com.softwaremill.sttp._ - import io.circe.generic.auto._ - - // DO NOT REMOVE PromCirceSupport import below assuming it is unused - Intellij removes it in auto-imports :( . - // Needed to override Sampl case class Encoder. - import PromCirceSupport._ - implicit val backend = AsyncHttpClientFutureBackend() - - ShutdownHookThread(shutdown()) - - def httpGet(httpEndpoint: String, httpTimeoutMs: Long, submitTime: Long, urlParams: Map[String, Any]) - (implicit scheduler: Scheduler): - Future[Response[scala.Either[DeserializationError[io.circe.Error], SuccessResponse]]] = { - val queryTimeElapsed = System.currentTimeMillis() - submitTime - val readTimeout = FiniteDuration(httpTimeoutMs - queryTimeElapsed, TimeUnit.MILLISECONDS) - val url = uri"$httpEndpoint?$urlParams" - logger.debug("promQlExec url={}", url) - sttp - .get(url) - .readTimeout(readTimeout) - .response(asJson[SuccessResponse]) - .send() + // TODO: Handle stitching with verbose flag + QueryResult(id, resultSchema.get(Avg.entryName).get, rangeVectors) } - def httpMetadataGet(httpEndpoint: String, httpTimeoutMs: Long, submitTime: Long, urlParams: Map[String, Any]) - (implicit scheduler: Scheduler): - Future[Response[scala.Either[DeserializationError[io.circe.Error], MetadataSuccessResponse]]] = { - val queryTimeElapsed = System.currentTimeMillis() - submitTime - val readTimeout = FiniteDuration(httpTimeoutMs - queryTimeElapsed, TimeUnit.MILLISECONDS) - val url = uri"$httpEndpoint?$urlParams" - logger.debug("promMetadataExec url={}", url) - sttp - .get(url) - .readTimeout(readTimeout) - .response(asJson[MetadataSuccessResponse]) - .send() - } + def genStdValQueryResult(data: Data, id: String): QueryResult = { + val rangeVectors = data.result.map { d => + val rv = new RangeVector { + val row = new StdValAggTransientRow() + + override def key: RangeVectorKey = CustomRangeVectorKey(d.metric.map(m => m._1.utf8 -> m._2.utf8)) + + override def rows(): RangeVectorCursor = { + import NoCloseCursor._ + d.aggregateResponse.get.aggregateSampl.iterator.collect { case a: StdValSampl => + row.setLong(0, a.timestamp * 1000) + row.setDouble(1, a.stddev) + row.setDouble(2, a.mean) + row.setLong(3, a.count) + row + } + } + override def numRows: Option[Int] = Option(d.aggregateResponse.get.aggregateSampl.size) + } + SerializedRangeVector(rv, builder, recordSchema.get(QueryFunctionConstants.stdVal).get, + queryWithPlanName(queryContext)) + } - def shutdown(): Unit = - { - logger.info("Shutting PromQlExec http") - backend.close() + // TODO: Handle stitching with verbose flag + QueryResult(id, resultSchema.get("stdval").get, rangeVectors) } } - diff --git a/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala b/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala index 06506327c7..0f03c411fd 100644 --- a/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala +++ b/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala @@ -279,7 +279,7 @@ final case class SortFunctionMapper(function: SortFunctionId) extends RangeVecto // Create SerializedRangeVector so that sorting does not consume rows iterator val resultRv = source.toListL.map { rvs => - rvs.map(SerializedRangeVector(_, builder, recSchema)). + rvs.map(SerializedRangeVector(_, builder, recSchema, s"SortRangeVectorTransformer: $args")). sortBy { rv => if (rv.rows.hasNext) rv.rows.next().getDouble(1) else Double.NaN }(ordering) diff --git a/query/src/main/scala/filodb/query/exec/RemoteExec.scala b/query/src/main/scala/filodb/query/exec/RemoteExec.scala new file mode 100644 index 0000000000..a0a7b0387b --- /dev/null +++ b/query/src/main/scala/filodb/query/exec/RemoteExec.scala @@ -0,0 +1,190 @@ +package filodb.query.exec + +import java.util.concurrent.TimeUnit + +import scala.concurrent.Future +import scala.concurrent.duration.FiniteDuration +import scala.sys.ShutdownHookThread + +import com.softwaremill.sttp.{DeserializationError, Response, SttpBackendOptions} +import com.softwaremill.sttp.SttpBackendOptions.ProxyType.{Http, Socks} +import com.softwaremill.sttp.asynchttpclient.future.AsyncHttpClientFutureBackend +import com.softwaremill.sttp.circe.asJson +import com.typesafe.scalalogging.StrictLogging +import kamon.Kamon +import kamon.trace.Span +import monix.eval.Task +import monix.execution.Scheduler +import org.asynchttpclient.{AsyncHttpClientConfig, DefaultAsyncHttpClientConfig} +import org.asynchttpclient.proxy.ProxyServer + +import filodb.core.query.{PromQlQueryParams, QuerySession} +import filodb.core.store.ChunkSource +import filodb.query.{BadQueryException, MetadataSuccessResponse, PromCirceSupport, QueryResponse, SuccessResponse} + +trait RemoteExec extends LeafExecPlan with StrictLogging { + + def queryEndpoint: String + + def remoteExecHttpClient: RemoteExecHttpClient + + def requestTimeoutMs: Long + + def urlParams: Map[String, Any] + + def promQlQueryParams: PromQlQueryParams = queryContext.origQueryParams.asInstanceOf[PromQlQueryParams] + + def args: String = s"${promQlQueryParams.toString}, ${queryContext.plannerParams}, queryEndpoint=$queryEndpoint, " + + s"requestTimeoutMs=$requestTimeoutMs" + + def limit: Int = ??? + + /** + * Since execute is already overrided here, doExecute() can be empty. + */ + def doExecute(source: ChunkSource, + querySession: QuerySession) + (implicit sched: Scheduler): ExecResult = ??? + + override def execute(source: ChunkSource, + querySession: QuerySession) + (implicit sched: Scheduler): Task[QueryResponse] = { + if (queryEndpoint == null) { + throw new BadQueryException("Remote Query endpoint can not be null in RemoteExec.") + } + + // Please note that the following needs to be wrapped inside `runWithSpan` so that the context will be propagated + // across threads. Note that task/observable will not run on the thread where span is present since + // kamon uses thread-locals. + val span = Kamon.currentSpan() + // Dont finish span since this code didnt create it + Kamon.runWithSpan(span, false) { + Task.fromFuture(sendHttpRequest(span, requestTimeoutMs)) + } + } + + def sendHttpRequest(execPlan2Span: Span, httpTimeoutMs: Long) + (implicit sched: Scheduler): Future[QueryResponse] + + def getUrlParams(): Map[String, Any] = { + var finalUrlParams = urlParams ++ + Map("start" -> promQlQueryParams.startSecs, + "end" -> promQlQueryParams.endSecs, + "time" -> promQlQueryParams.endSecs, + "step" -> promQlQueryParams.stepSecs, + "processFailure" -> queryContext.plannerParams.processFailure, + "processMultiPartition" -> queryContext.plannerParams.processMultiPartition, + "histogramMap" -> "true", + "skipAggregatePresent" -> queryContext.plannerParams.skipAggregatePresent, + "verbose" -> promQlQueryParams.verbose) + if (queryContext.plannerParams.spread.isDefined) finalUrlParams = finalUrlParams + ("spread" -> queryContext. + plannerParams.spread.get) + logger.debug("URLParams for RemoteExec:" + finalUrlParams) + finalUrlParams + } + +} + +/** + * A trait for remoteExec GET Queries. + */ +trait RemoteExecHttpClient extends StrictLogging { + + def httpGet(applicationId: String, httpEndpoint: String, + httpTimeoutMs: Long, submitTime: Long, urlParams: Map[String, Any]) + (implicit scheduler: Scheduler): + Future[Response[scala.Either[DeserializationError[io.circe.Error], SuccessResponse]]] + + def httpMetadataGet(applicationId: String, httpEndpoint: String, + httpTimeoutMs: Long, submitTime: Long, urlParams: Map[String, Any]) + (implicit scheduler: Scheduler): + Future[Response[scala.Either[DeserializationError[io.circe.Error], MetadataSuccessResponse]]] + +} + +class RemoteHttpClient private(asyncHttpClientConfig: AsyncHttpClientConfig) extends RemoteExecHttpClient { + + import com.softwaremill.sttp._ + import io.circe.generic.auto._ + + // DO NOT REMOVE PromCirceSupport import below assuming it is unused - Intellij removes it in auto-imports :( . + // Needed to override Sampl case class Encoder. + import PromCirceSupport._ + private implicit val backend = AsyncHttpClientFutureBackend.usingConfig(asyncHttpClientConfig) + + ShutdownHookThread(shutdown()) + + def httpGet(applicationId: String, httpEndpoint: String, + httpTimeoutMs: Long, submitTime: Long, urlParams: Map[String, Any]) + (implicit scheduler: Scheduler): + Future[Response[scala.Either[DeserializationError[io.circe.Error], SuccessResponse]]] = { + val queryTimeElapsed = System.currentTimeMillis() - submitTime + val readTimeout = FiniteDuration(httpTimeoutMs - queryTimeElapsed, TimeUnit.MILLISECONDS) + val url = uri"$httpEndpoint?$urlParams" + logger.debug("promQlExec url={}", url) + sttp + .header(HeaderNames.UserAgent, applicationId) + .get(url) + .readTimeout(readTimeout) + .response(asJson[SuccessResponse]) + .send() + } + + def httpMetadataGet(applicationId: String, httpEndpoint: String, + httpTimeoutMs: Long, submitTime: Long, urlParams: Map[String, Any]) + (implicit scheduler: Scheduler): + Future[Response[scala.Either[DeserializationError[io.circe.Error], MetadataSuccessResponse]]] = { + val queryTimeElapsed = System.currentTimeMillis() - submitTime + val readTimeout = FiniteDuration(httpTimeoutMs - queryTimeElapsed, TimeUnit.MILLISECONDS) + val url = uri"$httpEndpoint?$urlParams" + logger.debug("promMetadataExec url={}", url) + sttp + .header(HeaderNames.UserAgent, applicationId) + .get(url) + .readTimeout(readTimeout) + .response(asJson[MetadataSuccessResponse]) + .send() + } + + def shutdown(): Unit = + { + logger.info("Shutting PromQlExec http") + backend.close() + } +} + +object RemoteHttpClient { + + import scala.collection.JavaConverters._ + + /** + * A default prom remote http client backend from DefaultPromRemoteHttpClientFactory. + */ + def configBuilder(): DefaultAsyncHttpClientConfig.Builder = { + // A copy of private AsyncHttpClientBackend.defaultClient. + var configBuilder = new DefaultAsyncHttpClientConfig.Builder() + .setConnectTimeout(SttpBackendOptions.Default.connectionTimeout.toMillis.toInt) + configBuilder = SttpBackendOptions.Default.proxy match { + case None => configBuilder + case Some(p) => + val proxyType: org.asynchttpclient.proxy.ProxyType = + p.proxyType match { + case Socks => org.asynchttpclient.proxy.ProxyType.SOCKS_V5 + case Http => org.asynchttpclient.proxy.ProxyType.HTTP + } + + configBuilder.setProxyServer( + new ProxyServer.Builder(p.host, p.port) + .setProxyType(proxyType) // Fix issue #145 + .setNonProxyHosts(p.nonProxyHosts.asJava) + .build()) + } + configBuilder + } + + val defaultClient = RemoteHttpClient(configBuilder().build()) + + def apply(asyncHttpClientConfig: AsyncHttpClientConfig): RemoteHttpClient = + new RemoteHttpClient(asyncHttpClientConfig) + +} diff --git a/query/src/main/scala/filodb/query/exec/ScalarBinaryOperationExec.scala b/query/src/main/scala/filodb/query/exec/ScalarBinaryOperationExec.scala index 012f2c92d1..f8d444cc4b 100644 --- a/query/src/main/scala/filodb/query/exec/ScalarBinaryOperationExec.scala +++ b/query/src/main/scala/filodb/query/exec/ScalarBinaryOperationExec.scala @@ -1,6 +1,5 @@ package filodb.query.exec -import kamon.Kamon import monix.eval.Task import monix.execution.Scheduler @@ -57,24 +56,9 @@ case class ScalarBinaryOperationExec(queryContext: QueryContext, override def execute(source: ChunkSource, querySession: QuerySession) (implicit sched: Scheduler): Task[QueryResponse] = { - val execPlan2Span = Kamon.spanBuilder(s"execute-${getClass.getSimpleName}") - .asChildOf(Kamon.currentSpan()) - .tag("query-id", queryContext.queryId) - .start() - val rangeVectors : Seq[RangeVector] = Seq(ScalarFixedDouble(params, evaluate)) - // Please note that the following needs to be wrapped inside `runWithSpan` so that the context will be propagated - // across threads. Note that task/observable will not run on the thread where span is present since - // kamon uses thread-locals. - Kamon.runWithSpan(execPlan2Span, true) { - Task { - val span = Kamon.spanBuilder(s"transform-${getClass.getSimpleName}") - .asChildOf(execPlan2Span) - .tag("query-id", queryContext.queryId) - .start() - span.finish() - QueryResult(queryContext.queryId, resultSchema, rangeVectors) - } + Task { + QueryResult(queryContext.queryId, resultSchema, rangeVectors) } } diff --git a/query/src/main/scala/filodb/query/exec/ScalarFixedDoubleExec.scala b/query/src/main/scala/filodb/query/exec/ScalarFixedDoubleExec.scala index a6dd0fa3c9..6991d0c795 100644 --- a/query/src/main/scala/filodb/query/exec/ScalarFixedDoubleExec.scala +++ b/query/src/main/scala/filodb/query/exec/ScalarFixedDoubleExec.scala @@ -46,27 +46,19 @@ case class ScalarFixedDoubleExec(queryContext: QueryContext, override def execute(source: ChunkSource, querySession: QuerySession) (implicit sched: Scheduler): Task[QueryResponse] = { - val execPlan2Span = Kamon.spanBuilder(s"execute-${getClass.getSimpleName}") - .asChildOf(Kamon.currentSpan()) - .tag("query-id", queryContext.queryId) - .start() val resultSchema = ResultSchema(columns, 1) val rangeVectors : Seq[RangeVector] = Seq(ScalarFixedDouble(params, value)) // Please note that the following needs to be wrapped inside `runWithSpan` so that the context will be propagated // across threads. Note that task/observable will not run on the thread where span is present since // kamon uses thread-locals. - Kamon.runWithSpan(execPlan2Span, true) { + // Dont finish span since this code didnt create it + Kamon.runWithSpan(Kamon.currentSpan(), false) { Task { - val span = Kamon.spanBuilder(s"transform-${getClass.getSimpleName}") - .asChildOf(execPlan2Span) - .tag("query-id", queryContext.queryId) - .start() rangeVectorTransformers.foldLeft((Observable.fromIterable(rangeVectors), resultSchema)) { (acc, transf) => val paramRangeVector: Seq[Observable[ScalarRangeVector]] = transf.funcParams.map(_.getResult) - (transf.apply(acc._1, querySession, queryContext.sampleLimit, acc._2, + (transf.apply(acc._1, querySession, queryContext.plannerParams.sampleLimit, acc._2, paramRangeVector), transf.schema(acc._2)) }._1.toListL.map({ - span.finish() QueryResult(queryContext.queryId, resultSchema, _) }) }.flatten diff --git a/query/src/main/scala/filodb/query/exec/SelectRawPartitionsExec.scala b/query/src/main/scala/filodb/query/exec/SelectRawPartitionsExec.scala index a18a668baa..b7e9867029 100644 --- a/query/src/main/scala/filodb/query/exec/SelectRawPartitionsExec.scala +++ b/query/src/main/scala/filodb/query/exec/SelectRawPartitionsExec.scala @@ -1,6 +1,5 @@ package filodb.query.exec -import kamon.Kamon import monix.eval.Task import monix.execution.Scheduler import monix.reactive.Observable @@ -124,16 +123,12 @@ final case class SelectRawPartitionsExec(queryContext: QueryContext, def doExecute(source: ChunkSource, querySession: QuerySession) (implicit sched: Scheduler): ExecResult = { - val span = Kamon.spanBuilder(s"execute-${getClass.getSimpleName}") - .asChildOf(Kamon.currentSpan()) - .start() Query.qLogger.debug(s"queryId=${queryContext.queryId} on dataset=$datasetRef shard=" + s"${lookupRes.map(_.shard).getOrElse("")} " + s"schema=" + s"${dataSchema.map(_.name)} is configured to use columnIDs=$colIds") val rvs = dataSchema.map { sch => source.rangeVectors(datasetRef, lookupRes.get, colIds, sch, filterSchemas, querySession) }.getOrElse(Observable.empty) - span.finish() ExecResult(rvs, Task.eval(schemaOfDoExecute())) } diff --git a/query/src/main/scala/filodb/query/exec/TimeScalarGeneratorExec.scala b/query/src/main/scala/filodb/query/exec/TimeScalarGeneratorExec.scala index f59026d62d..dea5ac001c 100644 --- a/query/src/main/scala/filodb/query/exec/TimeScalarGeneratorExec.scala +++ b/query/src/main/scala/filodb/query/exec/TimeScalarGeneratorExec.scala @@ -44,10 +44,7 @@ case class TimeScalarGeneratorExec(queryContext: QueryContext, override def execute(source: ChunkSource, querySession: QuerySession) (implicit sched: Scheduler): Task[QueryResponse] = { - val execPlan2Span = Kamon.spanBuilder(s"execute-${getClass.getSimpleName}") - .asChildOf(Kamon.currentSpan()) - .tag("query-id", queryContext.queryId) - .start() + val span = Kamon.currentSpan() val resultSchema = ResultSchema(columns, 1) val rangeVectors : Seq[RangeVector] = function match { case Time => Seq(TimeScalar(params)) @@ -63,18 +60,14 @@ case class TimeScalarGeneratorExec(queryContext: QueryContext, // Please note that the following needs to be wrapped inside `runWithSpan` so that the context will be propagated // across threads. Note that task/observable will not run on the thread where span is present since // kamon uses thread-locals. - Kamon.runWithSpan(execPlan2Span, true) { + // Dont finish span since this code didnt create it + Kamon.runWithSpan(span, false) { Task { - val span = Kamon.spanBuilder(s"transform-${getClass.getSimpleName}") - .asChildOf(execPlan2Span) - .tag("query-id", queryContext.queryId) - .start() rangeVectorTransformers.foldLeft((Observable.fromIterable(rangeVectors), resultSchema)) { (acc, transf) => val paramRangeVector: Seq[Observable[ScalarRangeVector]] = transf.funcParams.map(_.getResult) - (transf.apply(acc._1, querySession, queryContext.sampleLimit, acc._2, + (transf.apply(acc._1, querySession, queryContext.plannerParams.sampleLimit, acc._2, paramRangeVector), transf.schema(acc._2)) }._1.toListL.map({ - span.finish() QueryResult(queryContext.queryId, resultSchema, _) }) }.flatten diff --git a/query/src/main/scala/filodb/query/exec/aggregator/AvgRowAggregator.scala b/query/src/main/scala/filodb/query/exec/aggregator/AvgRowAggregator.scala index 499bae88bb..8ea6894f00 100644 --- a/query/src/main/scala/filodb/query/exec/aggregator/AvgRowAggregator.scala +++ b/query/src/main/scala/filodb/query/exec/aggregator/AvgRowAggregator.scala @@ -45,7 +45,7 @@ object AvgRowAggregator extends RowAggregator { acc } // ignore last count column. we rely on schema change - def present(aggRangeVector: RangeVector, limit: Int): Seq[RangeVector] = Seq(aggRangeVector) + def present(aggRangeVector: RangeVector, limit: Int, rangeParams: RangeParams): Seq[RangeVector] = Seq(aggRangeVector) def reductionSchema(source: ResultSchema): ResultSchema = { source.copy(columns = source.columns :+ ColumnInfo("count", ColumnType.LongColumn)) } diff --git a/query/src/main/scala/filodb/query/exec/aggregator/CountRowAggregator.scala b/query/src/main/scala/filodb/query/exec/aggregator/CountRowAggregator.scala index d684c0be47..9bb3899fd7 100644 --- a/query/src/main/scala/filodb/query/exec/aggregator/CountRowAggregator.scala +++ b/query/src/main/scala/filodb/query/exec/aggregator/CountRowAggregator.scala @@ -34,7 +34,7 @@ abstract class CountRowAggregator extends RowAggregator { acc.count += aggRes.getDouble(1) acc } - def present(aggRangeVector: RangeVector, limit: Int): Seq[RangeVector] = Seq(aggRangeVector) + def present(aggRangeVector: RangeVector, limit: Int, rangeParams: RangeParams): Seq[RangeVector] = Seq(aggRangeVector) def reductionSchema(source: ResultSchema): ResultSchema = source def presentationSchema(reductionSchema: ResultSchema): ResultSchema = reductionSchema } 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 cb74a54df8..1b64d62a49 100644 --- a/query/src/main/scala/filodb/query/exec/aggregator/CountValuesRowAggregator.scala +++ b/query/src/main/scala/filodb/query/exec/aggregator/CountValuesRowAggregator.scala @@ -2,8 +2,6 @@ package filodb.query.exec.aggregator import scala.collection.mutable -import kamon.Kamon - import filodb.core.binaryrecord2.RecordBuilder import filodb.core.memstore.FiloSchedulers import filodb.core.memstore.FiloSchedulers.QuerySchedName @@ -83,16 +81,15 @@ class CountValuesRowAggregator(label: String, limit: Int = 1000) extends RowAggr acc } - def present(aggRangeVector: RangeVector, limit: Int): Seq[RangeVector] = { + def present(aggRangeVector: RangeVector, limit: Int, rangeParams: RangeParams): 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 span = Kamon.spanBuilder(s"execplan-scan-latency-CountValues").start() try { FiloSchedulers.assertThreadName(QuerySchedName) // aggRangeVector.rows.take below triggers the ChunkInfoIterator which requires lock/release - ChunkMap.validateNoSharedLocks() + ChunkMap.validateNoSharedLocks(s"CountValues-$label") aggRangeVector.rows.take(limit).foreach { row => val rowMap = CountValuesSerDeser.deserialize(row.getBlobBase(1), row.getBlobNumBytes(1), row.getBlobOffset(1)) @@ -111,7 +108,6 @@ class CountValuesRowAggregator(label: String, limit: Int = 1000) extends RowAggr aggRangeVector.rows.close() ChunkMap.releaseAllSharedLocks() } - span.finish() resRvs.map { case (key, builder) => val numRows = builder.allContainers.map(_.countRecords()).sum new SerializedRangeVector(key, numRows, builder.allContainers, recSchema, 0) diff --git a/query/src/main/scala/filodb/query/exec/aggregator/HistMaxSumAggregator.scala b/query/src/main/scala/filodb/query/exec/aggregator/HistMaxSumAggregator.scala index 8d89465bfc..e81b9a143a 100644 --- a/query/src/main/scala/filodb/query/exec/aggregator/HistMaxSumAggregator.scala +++ b/query/src/main/scala/filodb/query/exec/aggregator/HistMaxSumAggregator.scala @@ -1,6 +1,6 @@ package filodb.query.exec.aggregator -import filodb.core.query.{MutableRowReader, RangeVector, RangeVectorKey, ResultSchema, TransientHistMaxRow} +import filodb.core.query.{MutableRowReader, RangeParams, RangeVector, RangeVectorKey, ResultSchema, TransientHistMaxRow} import filodb.memory.format.RowReader object HistMaxSumAggregator extends RowAggregator { @@ -29,7 +29,7 @@ object HistMaxSumAggregator extends RowAggregator { acc.m = if (acc.m.isNaN) aggRes.getDouble(2) else Math.max(acc.m, aggRes.getDouble(2)) acc } - def present(aggRangeVector: RangeVector, limit: Int): Seq[RangeVector] = Seq(aggRangeVector) + def present(aggRangeVector: RangeVector, limit: Int, rangeParams: RangeParams): Seq[RangeVector] = Seq(aggRangeVector) def reductionSchema(source: ResultSchema): ResultSchema = source def presentationSchema(reductionSchema: ResultSchema): ResultSchema = reductionSchema } diff --git a/query/src/main/scala/filodb/query/exec/aggregator/HistSumRowAggregator.scala b/query/src/main/scala/filodb/query/exec/aggregator/HistSumRowAggregator.scala index 307fe5380a..fb6920a7bc 100644 --- a/query/src/main/scala/filodb/query/exec/aggregator/HistSumRowAggregator.scala +++ b/query/src/main/scala/filodb/query/exec/aggregator/HistSumRowAggregator.scala @@ -1,6 +1,6 @@ package filodb.query.exec.aggregator -import filodb.core.query.{MutableRowReader, RangeVector, RangeVectorKey, ResultSchema, TransientHistRow} +import filodb.core.query.{MutableRowReader, RangeParams, RangeVector, RangeVectorKey, ResultSchema, TransientHistRow} import filodb.memory.format.RowReader object HistSumRowAggregator extends RowAggregator { @@ -27,7 +27,7 @@ object HistSumRowAggregator extends RowAggregator { } acc } - def present(aggRangeVector: RangeVector, limit: Int): Seq[RangeVector] = Seq(aggRangeVector) + def present(aggRangeVector: RangeVector, limit: Int, rangeParams: RangeParams): Seq[RangeVector] = Seq(aggRangeVector) def reductionSchema(source: ResultSchema): ResultSchema = source def presentationSchema(reductionSchema: ResultSchema): ResultSchema = reductionSchema } \ No newline at end of file diff --git a/query/src/main/scala/filodb/query/exec/aggregator/MaxRowAggregator.scala b/query/src/main/scala/filodb/query/exec/aggregator/MaxRowAggregator.scala index 2799066ef1..fae8f0b299 100644 --- a/query/src/main/scala/filodb/query/exec/aggregator/MaxRowAggregator.scala +++ b/query/src/main/scala/filodb/query/exec/aggregator/MaxRowAggregator.scala @@ -27,7 +27,7 @@ object MaxRowAggregator extends RowAggregator { } acc } - def present(aggRangeVector: RangeVector, limit: Int): Seq[RangeVector] = Seq(aggRangeVector) + def present(aggRangeVector: RangeVector, limit: Int, rangeParams: RangeParams): Seq[RangeVector] = Seq(aggRangeVector) def reductionSchema(source: ResultSchema): ResultSchema = source def presentationSchema(reductionSchema: ResultSchema): ResultSchema = reductionSchema } diff --git a/query/src/main/scala/filodb/query/exec/aggregator/MinRowAggregator.scala b/query/src/main/scala/filodb/query/exec/aggregator/MinRowAggregator.scala index d41764a01e..d5914aa6b6 100644 --- a/query/src/main/scala/filodb/query/exec/aggregator/MinRowAggregator.scala +++ b/query/src/main/scala/filodb/query/exec/aggregator/MinRowAggregator.scala @@ -28,7 +28,7 @@ object MinRowAggregator extends RowAggregator { } acc } - def present(aggRangeVector: RangeVector, limit: Int): Seq[RangeVector] = Seq(aggRangeVector) + def present(aggRangeVector: RangeVector, limit: Int, rangeParams: RangeParams): Seq[RangeVector] = Seq(aggRangeVector) def reductionSchema(source: ResultSchema): ResultSchema = source def presentationSchema(reductionSchema: ResultSchema): ResultSchema = reductionSchema } diff --git a/query/src/main/scala/filodb/query/exec/aggregator/QuantileRowAggregator.scala b/query/src/main/scala/filodb/query/exec/aggregator/QuantileRowAggregator.scala index df7fb687ee..a3ae73ffa7 100644 --- a/query/src/main/scala/filodb/query/exec/aggregator/QuantileRowAggregator.scala +++ b/query/src/main/scala/filodb/query/exec/aggregator/QuantileRowAggregator.scala @@ -63,7 +63,7 @@ class QuantileRowAggregator(q: Double) extends RowAggregator { acc } - def present(aggRangeVector: RangeVector, limit: Int): Seq[RangeVector] = { + def present(aggRangeVector: RangeVector, limit: Int, rangeParams: RangeParams): Seq[RangeVector] = { val mutRow = new TransientRow() val result = aggRangeVector.rows.mapRow { r => val qVal = ArrayDigest.fromBytes(r.getBuffer(1)).quantile(q) diff --git a/query/src/main/scala/filodb/query/exec/aggregator/RowAggregator.scala b/query/src/main/scala/filodb/query/exec/aggregator/RowAggregator.scala index 23c258fc74..470d3334fc 100644 --- a/query/src/main/scala/filodb/query/exec/aggregator/RowAggregator.scala +++ b/query/src/main/scala/filodb/query/exec/aggregator/RowAggregator.scala @@ -1,7 +1,7 @@ package filodb.query.exec.aggregator import filodb.core.metadata.Column.ColumnType -import filodb.core.query.{MutableRowReader, RangeVector, RangeVectorKey, ResultSchema} +import filodb.core.query.{MutableRowReader, RangeParams, RangeVector, RangeVectorKey, ResultSchema} import filodb.memory.format.RowReader import filodb.query.AggregationOperator import filodb.query.AggregationOperator._ @@ -97,7 +97,7 @@ trait RowAggregator { * Apply limit only on iterators that are NOT lazy and need to be * materialized. */ - def present(aggRangeVector: RangeVector, limit: Int): Seq[RangeVector] + def present(aggRangeVector: RangeVector, limit: Int, rangeParams: RangeParams): Seq[RangeVector] /** * Schema of the RowReader returned by toRowReader diff --git a/query/src/main/scala/filodb/query/exec/aggregator/StddevRowAggregator.scala b/query/src/main/scala/filodb/query/exec/aggregator/StddevRowAggregator.scala index c19f7ec10e..8cffe29ddc 100644 --- a/query/src/main/scala/filodb/query/exec/aggregator/StddevRowAggregator.scala +++ b/query/src/main/scala/filodb/query/exec/aggregator/StddevRowAggregator.scala @@ -57,7 +57,7 @@ object StddevRowAggregator extends RowAggregator { acc } // ignore last two column. we rely on schema change - def present(aggRangeVector: RangeVector, limit: Int): Seq[RangeVector] = Seq(aggRangeVector) + def present(aggRangeVector: RangeVector, limit: Int, rangeParams: RangeParams): Seq[RangeVector] = Seq(aggRangeVector) def reductionSchema(source: ResultSchema): ResultSchema = { source.copy(source.columns :+ ColumnInfo("mean", ColumnType.DoubleColumn) :+ ColumnInfo("count", ColumnType.LongColumn)) diff --git a/query/src/main/scala/filodb/query/exec/aggregator/StdvarRowAggregator.scala b/query/src/main/scala/filodb/query/exec/aggregator/StdvarRowAggregator.scala index 4360873964..43fa7476f1 100644 --- a/query/src/main/scala/filodb/query/exec/aggregator/StdvarRowAggregator.scala +++ b/query/src/main/scala/filodb/query/exec/aggregator/StdvarRowAggregator.scala @@ -71,7 +71,7 @@ object StdvarRowAggregator extends RowAggregator { acc } // ignore last two column. we rely on schema change - def present(aggRangeVector: RangeVector, limit: Int): Seq[RangeVector] = Seq(aggRangeVector) + def present(aggRangeVector: RangeVector, limit: Int, rangeParams: RangeParams): Seq[RangeVector] = Seq(aggRangeVector) def reductionSchema(source: ResultSchema): ResultSchema = { source.copy(source.columns :+ ColumnInfo("mean", ColumnType.DoubleColumn) :+ ColumnInfo("count", ColumnType.LongColumn)) 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 06ea90d089..617f9d99f6 100644 --- a/query/src/main/scala/filodb/query/exec/aggregator/SumRowAggregator.scala +++ b/query/src/main/scala/filodb/query/exec/aggregator/SumRowAggregator.scala @@ -1,6 +1,6 @@ package filodb.query.exec.aggregator -import filodb.core.query.{MutableRowReader, RangeVector, RangeVectorKey, ResultSchema, TransientRow} +import filodb.core.query.{MutableRowReader, RangeParams, RangeVector, RangeVectorKey, ResultSchema, TransientRow} import filodb.memory.format.RowReader /** * Map: Every sample is mapped to itself @@ -26,7 +26,7 @@ object SumRowAggregator extends RowAggregator { } acc } - def present(aggRangeVector: RangeVector, limit: Int): Seq[RangeVector] = Seq(aggRangeVector) + def present(aggRangeVector: RangeVector, limit: Int, rangeParams: RangeParams): Seq[RangeVector] = Seq(aggRangeVector) def reductionSchema(source: ResultSchema): ResultSchema = source def presentationSchema(reductionSchema: ResultSchema): ResultSchema = reductionSchema } \ No newline at end of file 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 43d37c5b0b..438b9c2e53 100644 --- a/query/src/main/scala/filodb/query/exec/aggregator/TopBottomKRowAggregator.scala +++ b/query/src/main/scala/filodb/query/exec/aggregator/TopBottomKRowAggregator.scala @@ -1,8 +1,9 @@ package filodb.query.exec.aggregator -import scala.collection.mutable +import java.util.concurrent.TimeUnit -import kamon.Kamon +import scala.collection.mutable +import scala.collection.mutable.ListBuffer import filodb.core.binaryrecord2.RecordBuilder import filodb.core.memstore.FiloSchedulers @@ -26,6 +27,10 @@ class TopBottomKRowAggregator(k: Int, bottomK: Boolean) extends RowAggregator { private val rvkStringCache = mutable.HashMap[RangeVectorKey, ZeroCopyUTF8String]() case class RVKeyAndValue(rvk: ZeroCopyUTF8String, value: Double) + val colSchema = Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), + ColumnInfo("value", ColumnType.DoubleColumn)) + val recSchema = SerializedRangeVector.toSchema(colSchema) + class TopKHolder(var timestamp: Long = 0L) extends AggregateHolder { val valueOrdering = Ordering.by[RVKeyAndValue, Double](kr => kr.value) implicit val ordering = if (bottomK) valueOrdering else valueOrdering.reverse @@ -58,7 +63,7 @@ class TopBottomKRowAggregator(k: Int, bottomK: Boolean) extends RowAggregator { def map(rvk: RangeVectorKey, item: RowReader, mapInto: MutableRowReader): RowReader = { val rvkString = rvkStringCache.getOrElseUpdate(rvk, CustomRangeVectorKey.toZcUtf8(rvk)) mapInto.setLong(0, item.getLong(0)) - // TODO: Use setBlob instead of setString once RowReeder has the support for blob + // TODO: Use setBlob instead of setString once RowReader has the support for blob mapInto.setString(1, rvkString) mapInto.setDouble(2, item.getDouble(1)) var i = 3 @@ -83,35 +88,52 @@ class TopBottomKRowAggregator(k: Int, bottomK: Boolean) extends RowAggregator { acc } - def present(aggRangeVector: RangeVector, limit: Int): Seq[RangeVector] = { - val colSchema = Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), - ColumnInfo("value", ColumnType.DoubleColumn)) - val recSchema = SerializedRangeVector.toSchema(colSchema) + private def addRecordToBuilder(builder: RecordBuilder, timeStampMs: Long, value: Double): Unit = { + builder.startNewRecord(recSchema) + builder.addLong(timeStampMs) + builder.addDouble(value) + builder.endRecord() + } + + /** + Create new builder and add NaN till current time + */ + private def createBuilder(rangeParams: RangeParams, currentTime: Long): RecordBuilder= { + val builder = SerializedRangeVector.newBuilder(); + for (t <- rangeParams.startSecs to (currentTime - rangeParams.stepSecs) by rangeParams.stepSecs) { + addRecordToBuilder(builder, t * 1000, Double.NaN) + } + builder + } + + def present(aggRangeVector: RangeVector, limit: Int, rangeParams: RangeParams): Seq[RangeVector] = { val resRvs = mutable.Map[RangeVectorKey, RecordBuilder]() - val span = Kamon.spanBuilder(s"execplan-scan-latency-TopBottomK").start() try { FiloSchedulers.assertThreadName(QuerySchedName) - ChunkMap.validateNoSharedLocks() + ChunkMap.validateNoSharedLocks(s"TopkQuery-$k-$bottomK") // We limit the results wherever it is materialized first. So it is done here. - aggRangeVector.rows.take(limit).foreach { row => + val rows = aggRangeVector.rows.take(limit) + for (t <- rangeParams.startSecs to rangeParams.endSecs by rangeParams.stepSecs) { + val rvkSeen = new ListBuffer[RangeVectorKey] + val row = rows.next() var i = 1 while (row.notNull(i)) { if (row.filoUTF8String(i) != CustomRangeVectorKey.emptyAsZcUtf8) { val rvk = CustomRangeVectorKey.fromZcUtf8(row.filoUTF8String(i)) - val builder = resRvs.getOrElseUpdate(rvk, SerializedRangeVector.newBuilder()) - builder.startNewRecord(recSchema) - builder.addLong(row.getLong(0)) - builder.addDouble(row.getDouble(i + 1)) - builder.endRecord() + rvkSeen += rvk + val builder = resRvs.getOrElseUpdate(rvk, createBuilder(rangeParams, t)) + addRecordToBuilder(builder, TimeUnit.SECONDS.toMillis(t), row.getDouble(i + 1)) } i += 2 } + resRvs.keySet.foreach { rvs => + if (!rvkSeen.contains(rvs)) addRecordToBuilder(resRvs.get(rvs).get, t * 1000, Double.NaN) + } } } finally { aggRangeVector.rows().close() ChunkMap.releaseAllSharedLocks() } - span.finish() resRvs.map { case (key, builder) => val numRows = builder.allContainers.map(_.countRecords).sum new SerializedRangeVector(key, numRows, builder.allContainers, recSchema, 0) diff --git a/query/src/main/scala/filodb/query/exec/rangefn/RangeFunction.scala b/query/src/main/scala/filodb/query/exec/rangefn/RangeFunction.scala index 356c03195c..cd155f6230 100644 --- a/query/src/main/scala/filodb/query/exec/rangefn/RangeFunction.scala +++ b/query/src/main/scala/filodb/query/exec/rangefn/RangeFunction.scala @@ -6,6 +6,7 @@ import filodb.core.query._ import filodb.core.store.ChunkSetInfoReader import filodb.memory.format.{vectors => bv, _} import filodb.memory.format.BinaryVector.BinaryVectorPtr +import filodb.query.Query import filodb.query.exec._ /** @@ -143,8 +144,13 @@ trait CounterChunkedRangeFunction[R <: MutableRowReader] extends ChunkedRangeFun // At least one sample is present if (startRowNum <= endRowNum) { - addTimeChunks(valueVectorAcc, valueVector, ccReader, startRowNum, endRowNum, - tsReader(tsVectorAcc, tsVector, startRowNum), tsReader(tsVectorAcc, tsVector, endRowNum)) + try { + addTimeChunks(valueVectorAcc, valueVector, ccReader, startRowNum, endRowNum, + tsReader(tsVectorAcc, tsVector, startRowNum), tsReader(tsVectorAcc, tsVector, endRowNum)) + } catch { case e: ArrayIndexOutOfBoundsException => + Query.qLogger.error(s"ArrayIndexOutOfBoundsException startRowNum=$startRowNum endRowNum=$endRowNum") + throw e + } } // Add any corrections from this chunk, pass on lastValue also to next chunk computation diff --git a/query/src/test/scala/filodb/query/PromCirceSupportSpec.scala b/query/src/test/scala/filodb/query/PromCirceSupportSpec.scala new file mode 100644 index 0000000000..4fd00031d5 --- /dev/null +++ b/query/src/test/scala/filodb/query/PromCirceSupportSpec.scala @@ -0,0 +1,164 @@ +package filodb.query + +import io.circe.parser +import io.circe.generic.auto._ +import org.scalatest.concurrent.ScalaFutures +import org.scalatest.funspec.AnyFunSpec +import org.scalatest.matchers.should.Matchers + +class PromCirceSupportSpec extends AnyFunSpec with Matchers with ScalaFutures { + + // DO NOT REMOVE PromCirceSupport import below assuming it is unused - Intellij removes it in auto-imports :( . + // Needed to override Sampl case class Encoder. + import PromCirceSupport._ + + it("should parse Sampl") { + val inputString = + """ + |[ + | [ + | 1600102672, + | "1.2" + | ], + | [ + | 1600102687, + | "3.1" + | ] + |] + |""".stripMargin + + parseAndValidate(inputString, List(Sampl(1600102672,1.2), Sampl(1600102687,3.1))) + } + + it("should parse aggregateResponse") { + val input = """[{ + | "status": "success", + | "data": { + | "resultType": "matrix", + | "result": [{ + | "metric": { + | + | }, + | "aggregateResponse": { + | "aggregateValues": [ + | [ + | 1601491649, + | "15.186417982460787", + | 5 + | ], + | [ + | 1601491679, + | "14.891293858511071", + | 6 + | ], + | [ + | 1601491709, + | "14.843819532173134", + | 7 + | ], + | [ + | 1601491719, + | "NaN", + | 7 + | ] + | + | ], + | "function": "avg" + | } + | }] + | }, + | "errorType": null, + | "error": null + |}]""".stripMargin + val expectedResult =List(AvgSampl(1601491649,15.186417982460787,5), + AvgSampl(1601491679,14.891293858511071,6), AvgSampl(1601491709,14.843819532173134,7), AvgSampl(1601491719, + Double.NaN, 7)) + + parser.decode[List[SuccessResponse]](input) match { + case Right(successResponse) => val aggregateResponse = successResponse.head.data.result.head.aggregateResponse.get + aggregateResponse.function shouldEqual("avg") + aggregateResponse.aggregateSampl.map(_.asInstanceOf[AvgSampl]).zip(expectedResult).foreach { + case (res, ex) => if (res.value.isNaN) { + ex.value.isNaN shouldEqual(true) + ex.count shouldEqual(res.count) + ex.timestamp shouldEqual(ex.timestamp) + } else ex shouldEqual(res) + } + case Left(ex) => println(ex) + } + } + + it("should parse sttdev aggregateResponse") { + val input = """[{ + | "status": "success", + | "data": { + | "resultType": "matrix", + | "result": [ + | { + | "metric": { + | + | }, + | "aggregateResponse": { + | "aggregateValues": [ + | [ + | 1603920650, + | "NaN", + | "NaN", + | 0 + | ], + | [ + | 1603920740, + | "0.0", + | "16.068496952984738", + | 1 + | ] + | ], + | "function": "stdval" + | } + | } + | ] + | }, + | "errorType": null, + | "error": null + |}]""".stripMargin + val expectedResult =List(StdValSampl(1603920650,Double.NaN, Double.NaN, 0), + StdValSampl(1603920740,0,16.068496952984738,1) + ) + + parser.decode[List[SuccessResponse]](input) match { + case Right(successResponse) => val aggregateResponse = successResponse.head.data.result.head.aggregateResponse.get + aggregateResponse.function shouldEqual("stdval") + aggregateResponse.aggregateSampl.map(_.asInstanceOf[StdValSampl]).zip(expectedResult).foreach { + case (res, ex) => if (res.mean.isNaN) { + ex.mean.isNaN shouldEqual(true) + ex.stddev.isNaN shouldEqual true + ex.count shouldEqual(res.count) + ex.timestamp shouldEqual(res.timestamp) + } else ex shouldEqual(res) + } + case Left(ex) => println(ex) + } + } + + + def parseAndValidate(input: String, expectedResult: List[DataSampl]): Unit = { + parser.decode[List[DataSampl]](input) match { + case Right(samples) => + samples.zip(expectedResult).foreach { + case (val1: Sampl, val2: Sampl) => { + val1.timestamp shouldEqual(val2.timestamp) + if (val1.value.isNaN) val2.value.isNaN shouldEqual true + else val1.value shouldEqual val2.value + } + + case (val1: HistSampl, val2: HistSampl) => { + val1.timestamp shouldEqual(val2.timestamp) + val1.buckets shouldEqual val2.buckets + } + + case _ => samples.sameElements(expectedResult) + } + case Left(ex) => throw ex + } + } +} diff --git a/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala b/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala index d33e86d497..a79cdacc7f 100644 --- a/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala +++ b/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala @@ -41,6 +41,8 @@ class AggrOverRangeVectorsSpec extends RawDataWindowingSpec with ScalaFutures { override def rows(): RangeVectorCursor = data.iterator }) + val rangeParams = RangeParams(0, 1, 0) + // Sum val agg1 = RowAggregator(AggregationOperator.Sum, Nil, tvSchema) val resultObs = RangeVectorAggregator.mapReduce(agg1, false, Observable.fromIterable(samples), noGrouping) @@ -111,7 +113,7 @@ class AggrOverRangeVectorsSpec extends RawDataWindowingSpec with ScalaFutures { val agg7 = RowAggregator(AggregationOperator.Quantile, Seq(0.70), tvSchema) val resultObs7a = RangeVectorAggregator.mapReduce(agg7, false, Observable.fromIterable(samples), noGrouping) val resultObs7 = RangeVectorAggregator.mapReduce(agg7, true, resultObs7a, rv=>rv.key) - val resultObs7b = RangeVectorAggregator.present(agg7, resultObs7, 1000) + val resultObs7b = RangeVectorAggregator.present(agg7, resultObs7, 1000, rangeParams) val result7 = resultObs7b.toListL.runAsync.futureValue result7.size shouldEqual 1 result7(0).key shouldEqual noKey @@ -166,17 +168,22 @@ class AggrOverRangeVectorsSpec extends RawDataWindowingSpec with ScalaFutures { val ignoreKey = CustomRangeVectorKey( Map("ignore".utf8 -> "ignore".utf8)) + val ignoreKey2 = CustomRangeVectorKey( + Map("ignore2".utf8 -> "ignore2".utf8)) + val noKey = CustomRangeVectorKey(Map.empty) def noGrouping(rv: RangeVector): RangeVectorKey = noKey it ("should ignore NaN while aggregating") { val samples: Array[RangeVector] = Array( - toRv(Seq((1L, Double.NaN), (2L, 5.6d))), - toRv(Seq((1L, 4.6d), (2L, 4.4d))), - toRv(Seq((1L, 2.1d), (2L, 5.4d))) + toRv(Seq((1000L, Double.NaN), (2000L, 5.6d))), + toRv(Seq((1000L, 4.6d), (2000L, 4.4d))), + toRv(Seq((1000L, 2.1d), (2000L, 5.4d))) ) + val rangeParams = RangeParams(1,1,2) + // Sum val agg1 = RowAggregator(AggregationOperator.Sum, Nil, tvSchema) val resultObs = RangeVectorAggregator.mapReduce(agg1, false, Observable.fromIterable(samples), noGrouping) @@ -214,8 +221,8 @@ class AggrOverRangeVectorsSpec extends RawDataWindowingSpec with ScalaFutures { // BottomK val agg5 = RowAggregator(AggregationOperator.BottomK, Seq(2.0), tvSchema) val resultObs5a = RangeVectorAggregator.mapReduce(agg5, false, Observable.fromIterable(samples), noGrouping) - val resultObs5 = RangeVectorAggregator.mapReduce(agg5, true, resultObs5a, rv=>rv.key) - val resultObs5b = RangeVectorAggregator.present(agg5, resultObs5, 1000) + val resultObs5 = RangeVectorAggregator.mapReduce(agg5,true, resultObs5a, rv=>rv.key) + val resultObs5b = RangeVectorAggregator.present(agg5, resultObs5, 1000, rangeParams) val result5 = resultObs5.toListL.runAsync.futureValue result5.size shouldEqual 1 result5(0).key shouldEqual noKey @@ -230,7 +237,7 @@ class AggrOverRangeVectorsSpec extends RawDataWindowingSpec with ScalaFutures { val agg6 = RowAggregator(AggregationOperator.TopK, Seq(2.0), tvSchema) val resultObs6a = RangeVectorAggregator.mapReduce(agg6, false, Observable.fromIterable(samples), noGrouping) val resultObs6 = RangeVectorAggregator.mapReduce(agg6, true, resultObs6a, rv=>rv.key) - val resultObs6b = RangeVectorAggregator.present(agg6, resultObs6, 1000) + val resultObs6b = RangeVectorAggregator.present(agg6, resultObs6, 1000, rangeParams) val result6 = resultObs6.toListL.runAsync.futureValue result6.size shouldEqual 1 result6(0).key shouldEqual noKey @@ -245,7 +252,7 @@ class AggrOverRangeVectorsSpec extends RawDataWindowingSpec with ScalaFutures { val agg7 = RowAggregator(AggregationOperator.Quantile, Seq(0.5), tvSchema) val resultObs7a = RangeVectorAggregator.mapReduce(agg7, false, Observable.fromIterable(samples), noGrouping) val resultObs7 = RangeVectorAggregator.mapReduce(agg7, true, resultObs7a, rv=>rv.key) - val resultObs7b = RangeVectorAggregator.present(agg7, resultObs7, 1000) + val resultObs7b = RangeVectorAggregator.present(agg7, resultObs7, 1000, rangeParams) val result7 = resultObs7b.toListL.runAsync.futureValue result7.size shouldEqual 1 result7(0).key shouldEqual noKey @@ -287,18 +294,18 @@ class AggrOverRangeVectorsSpec extends RawDataWindowingSpec with ScalaFutures { val recSchema = SerializedRangeVector.toSchema(Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), ColumnInfo("tdig", ColumnType.StringColumn))) val builder = SerializedRangeVector.newBuilder() - val srv = SerializedRangeVector(result7(0), builder, recSchema) + val srv = SerializedRangeVector(result7(0), builder, recSchema, "AggrOverRangeVectorsSpec") - val resultObs7b = RangeVectorAggregator.present(agg7, Observable.now(srv), 1000) + val resultObs7b = RangeVectorAggregator.present(agg7, Observable.now(srv), 1000, RangeParams(0,1,0)) val finalResult = resultObs7b.toListL.runAsync.futureValue compareIter(finalResult(0).rows.map(_.getDouble(1)), Seq(3.35d, 5.4d).iterator) } - private def toRv(samples: Seq[(Long, Double)]): RangeVector = { + private def toRv(samples: Seq[(Long, Double)], rangeVectorKey: RangeVectorKey = ignoreKey): RangeVector = { new RangeVector { import NoCloseCursor._ - override def key: RangeVectorKey = ignoreKey + override def key: RangeVectorKey = rangeVectorKey override def rows(): RangeVectorCursor = samples.map(r => new TransientRow(r._1, r._2)).iterator } } @@ -359,9 +366,9 @@ class AggrOverRangeVectorsSpec extends RawDataWindowingSpec with ScalaFutures { it("should return NaN when all values are NaN for a timestamp ") { val samples: Array[RangeVector] = Array( - toRv(Seq((1L, Double.NaN), (2L, 5.6d))), - toRv(Seq((1L, Double.NaN), (2L, 4.4d))), - toRv(Seq((1L, Double.NaN), (2L, 5.4d))) + toRv(Seq((1000L, Double.NaN), (2000L, 5.6d))), + toRv(Seq((1000L, Double.NaN), (2000L, 4.4d))), + toRv(Seq((1000L, Double.NaN), (2000L, 5.4d))) ) // Sum @@ -402,7 +409,7 @@ class AggrOverRangeVectorsSpec extends RawDataWindowingSpec with ScalaFutures { val agg5 = RowAggregator(AggregationOperator.BottomK, Seq(2.0), tvSchema) val resultObs5a = RangeVectorAggregator.mapReduce(agg5, false, Observable.fromIterable(samples), noGrouping) val resultObs5 = RangeVectorAggregator.mapReduce(agg5, true, resultObs5a, rv=>rv.key) - val resultObs5b = RangeVectorAggregator.present(agg5, resultObs5, 1000) + val resultObs5b = RangeVectorAggregator.present(agg5, resultObs5, 1000, RangeParams(1,1,2)) val result5 = resultObs5.toListL.runAsync.futureValue result5.size shouldEqual 1 result5(0).key shouldEqual noKey @@ -412,14 +419,14 @@ class AggrOverRangeVectorsSpec extends RawDataWindowingSpec with ScalaFutures { val result5b = resultObs5b.toListL.runAsync.futureValue result5b.size shouldEqual 1 result5b(0).key shouldEqual ignoreKey - // present removes the range vector which has all values as Double.Max - compareIter(result5b(0).rows.map(_.getDouble(1)), Seq(5.4d, 4.4d).iterator) + + compareIter(result5b(0).rows.map(_.getDouble(1)), Seq(Double.NaN, 5.4d, 4.4d).iterator) // TopK val agg6 = RowAggregator(AggregationOperator.TopK, Seq(2.0), tvSchema) val resultObs6a = RangeVectorAggregator.mapReduce(agg6, false, Observable.fromIterable(samples), noGrouping) val resultObs6 = RangeVectorAggregator.mapReduce(agg6, true, resultObs6a, rv=>rv.key) - val resultObs6b = RangeVectorAggregator.present(agg6, resultObs6, 1000) + val resultObs6b = RangeVectorAggregator.present(agg6, resultObs6, 1000, RangeParams(1,1,2)) val result6 = resultObs6.toListL.runAsync.futureValue result6.size shouldEqual 1 result6(0).key shouldEqual noKey @@ -428,7 +435,7 @@ class AggrOverRangeVectorsSpec extends RawDataWindowingSpec with ScalaFutures { val result6b = resultObs6b.toListL.runAsync.futureValue result6b.size shouldEqual 1 result6b(0).key shouldEqual ignoreKey - compareIter(result6b(0).rows.map(_.getDouble(1)), Seq(5.4d,5.6d).iterator) + compareIter(result6b(0).rows.map(_.getDouble(1)), Seq(Double.NaN, 5.4d, 5.6d).iterator) // Stdvar val agg8 = RowAggregator(AggregationOperator.Stdvar, Nil, tvSchema) @@ -453,14 +460,14 @@ class AggrOverRangeVectorsSpec extends RawDataWindowingSpec with ScalaFutures { // The value before NaN should not get carried over. Topk result for timestamp 1556744173L should have Double.NaN val samples: Array[RangeVector] = Array( - toRv(Seq((1556744143L, 42d), (1556744158L, 42d),(1556744173L, Double.NaN))) + toRv(Seq((1556744143L, 42d), (1556745158L, 42d),(1556745173L, Double.NaN))) ) val agg6 = RowAggregator(AggregationOperator.TopK, Seq(5.0), tvSchema) val resultObs6a = RangeVectorAggregator.mapReduce(agg6, false, Observable.fromIterable(samples), noGrouping) val resultObs6 = RangeVectorAggregator.mapReduce(agg6, true, resultObs6a, rv=>rv .key) - val resultObs6b = RangeVectorAggregator.present(agg6, resultObs6, 1000) + val resultObs6b = RangeVectorAggregator.present(agg6, resultObs6, 1000, RangeParams(1556744,1,1556745)) val result6 = resultObs6.toListL.runAsync.futureValue result6(0).key shouldEqual noKey val result6b = resultObs6b.toListL.runAsync.futureValue @@ -518,7 +525,39 @@ class AggrOverRangeVectorsSpec extends RawDataWindowingSpec with ScalaFutures { result(0).rows.map(_.getDouble(1)).toList shouldEqual counts } - it("should sum and compute max of histogram & max RVs") { + it ("should add NaN in topK") { + + val samples: Array[RangeVector] = Array( + toRv(Seq((1000L, Double.NaN), (2000L, 5.1), (3000L, Double.NaN), (4000L, 5.6d), (5000L, 4.0d), (6000L, + Double.NaN))), + toRv(Seq((1000L, Double.NaN), (2000L, 5.0), (3000L, Double.NaN), (4000L, 5.7d), (5000L, 4.4d), + (6000L, Double.NaN)), ignoreKey2) + ) + + val rangeParams = RangeParams(1,1,6) + + val agg = RowAggregator(AggregationOperator.TopK, Seq(1.0), tvSchema) + val resultObsa = RangeVectorAggregator.mapReduce(agg, false, Observable.fromIterable(samples), + noGrouping) + val resultObsb = RangeVectorAggregator.mapReduce(agg, true, resultObsa, rv=>rv.key) + val resultObsc = RangeVectorAggregator.present(agg, resultObsb, 1000, rangeParams) + val result = resultObsc.toListL.runAsync.futureValue + + result.size shouldEqual 2 + result(0).key shouldEqual ignoreKey2 + result(1).key shouldEqual ignoreKey + + result(0).rows.map(_.getLong(0)). + sameElements(Seq(1000L, 2000L, 3000L, 4000L, 5000L, 6000L).toIterator) shouldEqual true + result(1).rows.map(_.getLong(0)). + sameElements(Seq(1000L, 2000L, 3000L, 4000L, 5000L, 6000L).toIterator) shouldEqual true + compareIter(result(0).rows.map(_.getDouble(1)).toIterator, + Seq(Double.NaN, Double.NaN, Double.NaN, 5.7, 4.4, Double.NaN).toIterator) + compareIter(result(1).rows.map(_.getDouble(1)).toIterator, + Seq(Double.NaN, 5.1, Double.NaN, Double.NaN, Double.NaN, Double.NaN).toIterator) + } + + it("should sum and compute max of histogram & max RVs") { val (data1, rv1) = MMD.histMaxRV(100000L, numSamples = 5) val (data2, rv2) = MMD.histMaxRV(100000L, numSamples = 5) val samples: Array[RangeVector] = Array(rv1, rv2) @@ -560,7 +599,7 @@ class AggrOverRangeVectorsSpec extends RawDataWindowingSpec with ScalaFutures { val resultObs = RangeVectorAggregator.mapReduce(agg, false, Observable.fromIterable(samples), noGrouping) val resultObs1 = RangeVectorAggregator.mapReduce(agg, true, resultObs, rv=>rv.key) - val resultObs2 = RangeVectorAggregator.present(agg, resultObs1, 1000) + val resultObs2 = RangeVectorAggregator.present(agg, resultObs1, 1000, RangeParams(0,1,0) ) val result = resultObs2.toListL.runAsync.futureValue result.size.shouldEqual(4) result.map(_.key.labelValues).sameElements(expectedLabels) shouldEqual true diff --git a/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala b/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala index 82dcf6d8f3..066f50cfa0 100644 --- a/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala @@ -481,7 +481,7 @@ class BinaryJoinExecSpec extends AnyFunSpec with Matchers with ScalaFutures { } it("should throw BadQueryException - one-to-one with ignoring - cardinality limit 1") { - val queryContext = QueryContext(joinQueryCardLimit = 1) // set join card limit to 1 + val queryContext = QueryContext(plannerParams= PlannerParams(joinQueryCardLimit = 1)) // set join card limit to 1 val execPlan = BinaryJoinExec(queryContext, dummyDispatcher, Array(dummyPlan), // cannot be empty as some compose's rely on the schema new Array[ExecPlan](1), // empty since we test compose, not execute or doExecute @@ -506,7 +506,7 @@ class BinaryJoinExecSpec extends AnyFunSpec with Matchers with ScalaFutures { } it("should throw BadQueryException - one-to-one with on - cardinality limit 1") { - val queryContext = QueryContext(joinQueryCardLimit = 1) // set join card limit to 1 + val queryContext = QueryContext(plannerParams = PlannerParams(joinQueryCardLimit = 1)) // set join card limit to 1 val execPlan = BinaryJoinExec(queryContext, dummyDispatcher, Array(dummyPlan), // cannot be empty as some compose's rely on the schema new Array[ExecPlan](1), // empty since we test compose, not execute or doExecute diff --git a/query/src/test/scala/filodb/query/exec/BinaryJoinGroupingSpec.scala b/query/src/test/scala/filodb/query/exec/BinaryJoinGroupingSpec.scala index e946486123..72385d668b 100644 --- a/query/src/test/scala/filodb/query/exec/BinaryJoinGroupingSpec.scala +++ b/query/src/test/scala/filodb/query/exec/BinaryJoinGroupingSpec.scala @@ -391,7 +391,7 @@ class BinaryJoinGroupingSpec extends AnyFunSpec with Matchers with ScalaFutures } it("should throw BadQueryException - many-to-one with on - cardinality limit 1") { - val queryContext = QueryContext(joinQueryCardLimit = 1) // set join card limit to 1 + val queryContext = QueryContext(plannerParams= PlannerParams(joinQueryCardLimit = 1)) // set join card limit to 1 val samplesRhs2 = scala.util.Random.shuffle(sampleNodeRole.toList) // they may come out of order val execPlan = BinaryJoinExec(queryContext, dummyDispatcher, @@ -418,7 +418,7 @@ class BinaryJoinGroupingSpec extends AnyFunSpec with Matchers with ScalaFutures } it("should throw BadQueryException - many-to-one with ignoring - cardinality limit 1") { - val queryContext = QueryContext(joinQueryCardLimit = 1) // set join card limit to 1 + val queryContext = QueryContext(plannerParams= PlannerParams(joinQueryCardLimit = 1)) // set join card limit to 1 val samplesRhs2 = scala.util.Random.shuffle(sampleNodeRole.toList) // they may come out of order val execPlan = BinaryJoinExec(queryContext, dummyDispatcher, @@ -445,7 +445,7 @@ class BinaryJoinGroupingSpec extends AnyFunSpec with Matchers with ScalaFutures } it("should throw BadQueryException - many-to-one with by and grouping without arguments - cardinality limit 1") { - val queryContext = QueryContext(joinQueryCardLimit = 3) // set join card limit to 3 + val queryContext = QueryContext(plannerParams= PlannerParams(joinQueryCardLimit = 3)) // set join card limit to 3 val agg = RowAggregator(AggregationOperator.Sum, Nil, tvSchema) val aggMR = AggregateMapReduce(AggregationOperator.Sum, Nil, Nil, Seq("instance", "job")) val mapped = aggMR(Observable.fromIterable(sampleNodeCpu), querySession, 1000, tvSchema) diff --git a/query/src/test/scala/filodb/query/exec/InProcessPlanDispatcherSpec.scala b/query/src/test/scala/filodb/query/exec/InProcessPlanDispatcherSpec.scala index 0e27d05d09..6cd596d924 100644 --- a/query/src/test/scala/filodb/query/exec/InProcessPlanDispatcherSpec.scala +++ b/query/src/test/scala/filodb/query/exec/InProcessPlanDispatcherSpec.scala @@ -50,7 +50,7 @@ class InProcessPlanDispatcherSpec extends AnyFunSpec } after { - ChunkMap.validateNoSharedLocks(true) + ChunkMap.validateNoSharedLocks("InProcessPlanDispatcherSpec", true) } override def afterAll(): Unit = { diff --git a/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala b/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala index e5803b17b6..915742765b 100644 --- a/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala @@ -2,7 +2,6 @@ package filodb.query.exec import scala.collection.mutable.ArrayBuffer import scala.concurrent.duration._ - import com.typesafe.config.ConfigFactory import monix.eval.Task import monix.execution.Scheduler @@ -10,13 +9,12 @@ import monix.execution.Scheduler.Implicits.global import org.scalatest.BeforeAndAfterAll import org.scalatest.concurrent.ScalaFutures import org.scalatest.time.{Millis, Seconds, Span} - import filodb.core.MetricsTestData._ import filodb.core.TestData import filodb.core.binaryrecord2.BinaryRecordRowReader import filodb.core.memstore.{FixedMaxPartitionsEvictionPolicy, SomeData, TimeSeriesMemStore} import filodb.core.metadata.Schemas -import filodb.core.query.{ColumnFilter, Filter, QueryConfig, QueryContext, QuerySession, SerializedRangeVector} +import filodb.core.query.{ColumnFilter, Filter, PlannerParams, QueryConfig, QueryContext, QuerySession, SerializedRangeVector} import filodb.core.store.{InMemoryMetaStore, NullColumnStore} import filodb.memory.format.{SeqRowReader, ZeroCopyUTF8String} import filodb.query._ @@ -142,7 +140,7 @@ class MetadataExecSpec extends AnyFunSpec with Matchers with ScalaFutures with B val filters = Seq (ColumnFilter("job", Filter.Equals("myCoolService".utf8))) //Reducing limit results in truncated metadata response - val execPlan = PartKeysExec(QueryContext(sampleLimit = limit-1), dummyDispatcher, + val execPlan = PartKeysExec(QueryContext(plannerParams= PlannerParams(sampleLimit = limit - 1)), dummyDispatcher, timeseriesDataset.ref, 0, filters, false, now-5000, now) val resp = execPlan.execute(memStore, querySession).runAsync.futureValue diff --git a/query/src/test/scala/filodb/query/exec/MultiSchemaPartitionsExecSpec.scala b/query/src/test/scala/filodb/query/exec/MultiSchemaPartitionsExecSpec.scala index 63a7bcbbec..99c33dc0f9 100644 --- a/query/src/test/scala/filodb/query/exec/MultiSchemaPartitionsExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/MultiSchemaPartitionsExecSpec.scala @@ -448,8 +448,8 @@ class MultiSchemaPartitionsExecSpec extends AnyFunSpec with Matchers with ScalaF // Query returns n ("numRawSamples") samples - Applying Limit (n-1) to fail the query execution // with ResponseTooLargeException - val execPlan = MultiSchemaPartitionsExec(QueryContext(sampleLimit = 999), dummyDispatcher, - dsRef, 0, filters, AllChunkScan) + val execPlan = MultiSchemaPartitionsExec(QueryContext(plannerParams= PlannerParams(sampleLimit = 999)), + dummyDispatcher, dsRef, 0, filters, AllChunkScan) val resp = execPlan.execute(memStore, querySession).runAsync.futureValue val result = resp.asInstanceOf[QueryError] diff --git a/query/src/test/scala/filodb/query/exec/PromQlRemoteExecSpec.scala b/query/src/test/scala/filodb/query/exec/PromQlRemoteExecSpec.scala index 180f01015c..8c1c4f94cd 100644 --- a/query/src/test/scala/filodb/query/exec/PromQlRemoteExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/PromQlRemoteExecSpec.scala @@ -9,7 +9,7 @@ import org.scalatest.matchers.should.Matchers import filodb.core.metadata.{Dataset, DatasetOptions} import filodb.core.query.{PromQlQueryParams, QueryContext} -import filodb.memory.format.vectors.{MutableHistogram} +import filodb.memory.format.vectors.MutableHistogram import filodb.query import filodb.query.{Data, HistSampl, QueryResponse, QueryResult, Sampl} @@ -25,11 +25,11 @@ class PromQlRemoteExecSpec extends AnyFunSpec with Matchers with ScalaFutures { (implicit sched: Scheduler): Task[QueryResponse] = ??? } - val queryContext = QueryContext() val params = PromQlQueryParams("", 0, 0 , 0) + val queryContext = QueryContext(origQueryParams = params) it ("should convert matrix Data to QueryResponse ") { val expectedResult = List((1000000, 1.0), (2000000, 2.0), (3000000, 3.0)) - val exec = PromQlRemoteExec("", 60000, queryContext, dummyDispatcher, timeseriesDataset.ref, params) + val exec = PromQlRemoteExec("", 60000, queryContext, dummyDispatcher, timeseriesDataset.ref, RemoteHttpClient.defaultClient) val result = query.Result (Map("instance" -> "inst1"), Some(Seq(Sampl(1000, 1), Sampl(2000, 2), Sampl(3000, 3))), None) val res = exec.toQueryResponse(Data("vector", Seq(result)), "id", Kamon.currentSpan()) @@ -42,7 +42,7 @@ class PromQlRemoteExecSpec extends AnyFunSpec with Matchers with ScalaFutures { it ("should convert vector Data to QueryResponse ") { val expectedResult = List((1000000, 1.0)) - val exec = PromQlRemoteExec("", 60000, queryContext, dummyDispatcher, timeseriesDataset.ref, params) + val exec = PromQlRemoteExec("", 60000, queryContext, dummyDispatcher, timeseriesDataset.ref, RemoteHttpClient.defaultClient) val result = query.Result (Map("instance" -> "inst1"), None, Some(Sampl(1000, 1))) val res = exec.toQueryResponse(Data("vector", Seq(result)), "id", Kamon.currentSpan()) res.isInstanceOf[QueryResult] shouldEqual true @@ -55,7 +55,7 @@ class PromQlRemoteExecSpec extends AnyFunSpec with Matchers with ScalaFutures { it ("should convert vector Data to QueryResponse for MetadataQuery") { val exec = MetadataRemoteExec("", 60000, Map.empty, - queryContext, dummyDispatcher, timeseriesDataset.ref, params) + queryContext, dummyDispatcher, timeseriesDataset.ref, RemoteHttpClient.defaultClient) val map1 = Map("instance" -> "inst-1", "last-sample" -> "6377838" ) val map2 = Map("instance" -> "inst-2", "last-sample" -> "6377834" ) val res = exec.toQueryResponse(Seq(map1, map2), "id", Kamon.currentSpan()) @@ -67,7 +67,8 @@ class PromQlRemoteExecSpec extends AnyFunSpec with Matchers with ScalaFutures { } it ("should convert vector Data to QueryResponse for Metadata series query") { - val exec = MetadataRemoteExec("", 60000, Map.empty, queryContext, dummyDispatcher, timeseriesDataset.ref, params) + val exec = MetadataRemoteExec("", 60000, Map.empty, queryContext, + dummyDispatcher, timeseriesDataset.ref, RemoteHttpClient.defaultClient) val map1 = Map("instance" -> "inst-1", "last-sample" -> "6377838" ) val map2 = Map("instance" -> "inst-2", "last-sample" -> "6377834" ) val res = exec.toQueryResponse(Seq(map1, map2), "id", Kamon.currentSpan()) @@ -79,7 +80,7 @@ class PromQlRemoteExecSpec extends AnyFunSpec with Matchers with ScalaFutures { } it ("should convert histogram to QueryResponse ") { - val exec = PromQlRemoteExec("", 60000, queryContext, dummyDispatcher, timeseriesDataset.ref, params) + val exec = PromQlRemoteExec("", 60000, queryContext, dummyDispatcher, timeseriesDataset.ref, RemoteHttpClient.defaultClient) val result = query.Result (Map("instance" -> "inst1"), None, Some(HistSampl(1000, Map("1" -> 2, "+Inf" -> 3)))) val res = exec.toQueryResponse(Data("vector", Seq(result)), "id", Kamon.currentSpan()) res.isInstanceOf[QueryResult] shouldEqual true diff --git a/query/src/test/scala/filodb/query/exec/rangefn/AbsentFunctionSpec.scala b/query/src/test/scala/filodb/query/exec/rangefn/AbsentFunctionSpec.scala index 70262e2327..0ba50d31b1 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/AbsentFunctionSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/AbsentFunctionSpec.scala @@ -18,7 +18,7 @@ import org.scalatest.matchers.should.Matchers class AbsentFunctionSpec extends AnyFunSpec with Matchers with ScalaFutures with BeforeAndAfter { after { - ChunkMap.validateNoSharedLocks(true) + ChunkMap.validateNoSharedLocks("AbsentFunctionSpec", true) } val config: Config = ConfigFactory.load("application_test.conf").getConfig("filodb") diff --git a/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala b/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala index ce74dbb4d0..48a2c18612 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala @@ -28,17 +28,17 @@ trait RawDataWindowingSpec extends AnyFunSpec with Matchers with BeforeAndAfter private val blockStore = new PageAlignedBlockManager(100 * 1024 * 1024, new MemoryStats(Map("test"-> "test")), null, 16) val storeConf = TestData.storeConf.copy(maxChunksSize = 200) - protected val ingestBlockHolder = new BlockMemFactory(blockStore, None, timeseriesSchema.data.blockMetaSize, + protected val ingestBlockHolder = new BlockMemFactory(blockStore, timeseriesSchema.data.blockMetaSize, MMD.dummyContext, true) protected val tsBufferPool = new WriteBufferPool(TestData.nativeMem, timeseriesDatasetWithMetric.schema.data, storeConf) - protected val ingestBlockHolder2 = new BlockMemFactory(blockStore, None, downsampleSchema.data.blockMetaSize, + protected val ingestBlockHolder2 = new BlockMemFactory(blockStore, downsampleSchema.data.blockMetaSize, MMD.dummyContext, true) protected val tsBufferPool2 = new WriteBufferPool(TestData.nativeMem, downsampleSchema.data, storeConf) after { - ChunkMap.validateNoSharedLocks(true) + ChunkMap.validateNoSharedLocks(getClass().toString(), true) } override def afterAll(): Unit = { @@ -154,7 +154,8 @@ trait RawDataWindowingSpec extends AnyFunSpec with Matchers with BeforeAndAfter partKey: NativePointer = defaultPartKey): RawDataRangeVector = { val part = TimeSeriesPartitionSpec.makePart(0, timeseriesDatasetWithMetric, partKey, bufferPool = tsBufferPool) val readers = tuples.map { case (ts, d) => TupleRowReader((Some(ts), Some(d))) } - readers.foreach { row => part.ingest(0, row, ingestBlockHolder) } + readers.foreach { row => part.ingest(0, row, ingestBlockHolder, createChunkAtFlushBoundary = false, + flushIntervalMillis = Option.empty) } // Now flush and ingest the rest to ensure two separate chunks part.switchBuffers(ingestBlockHolder, encode = true) // part.encodeAndReleaseBuffers(ingestBlockHolder) @@ -167,7 +168,8 @@ trait RawDataWindowingSpec extends AnyFunSpec with Matchers with BeforeAndAfter val readers = tuples.map { case (ts, d1, d2, d3, d4, d5) => TupleRowReader((Some(ts), Some(d1), Some(d2), Some(d3), Some(d4), Some(d5))) } - readers.foreach { row => part.ingest(0, row, ingestBlockHolder2) } + readers.foreach { row => part.ingest(0, row, ingestBlockHolder2, createChunkAtFlushBoundary = false, + flushIntervalMillis = Option.empty) } // Now flush and ingest the rest to ensure two separate chunks part.switchBuffers(ingestBlockHolder2, encode = true) // part.encodeAndReleaseBuffers(ingestBlockHolder) @@ -184,7 +186,8 @@ trait RawDataWindowingSpec extends AnyFunSpec with Matchers with BeforeAndAfter val part = rv.partition.asInstanceOf[TimeSeriesPartition] val startingNumChunks = part.numChunks val readers = tuples.map { case (ts, d) => TupleRowReader((Some(ts), Some(d))) } - readers.foreach { row => part.ingest(0, row, ingestBlockHolder) } + readers.foreach { row => part.ingest(0, row, ingestBlockHolder, createChunkAtFlushBoundary = false, + flushIntervalMillis = Option.empty) } part.switchBuffers(ingestBlockHolder, encode = true) part.numChunks shouldEqual (startingNumChunks + 1) } diff --git a/query/src/test/scala/filodb/query/exec/rangefn/RateFunctionsSpec.scala b/query/src/test/scala/filodb/query/exec/rangefn/RateFunctionsSpec.scala index 4f4013e0b3..e6d7c7a269 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/RateFunctionsSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/RateFunctionsSpec.scala @@ -227,7 +227,8 @@ class RateFunctionsSpec extends RawDataWindowingSpec { val dropData = data.map(d => (d.head.asInstanceOf[Long] + 70000L) +: d.drop(1)) val container = MachineMetricsData.records(promHistDS, dropData).records val bh = MachineMetricsData.histIngestBH - container.iterate(promHistDS.ingestionSchema).foreach { row => part.ingest(0, row, bh) } + container.iterate(promHistDS.ingestionSchema).foreach { row => part.ingest(0, row, bh, + createChunkAtFlushBoundary = false, flushIntervalMillis = Option.empty) } part.switchBuffers(bh, encode = true) diff --git a/spark-jobs/src/main/scala/filodb/downsampler/chunk/BatchDownsampler.scala b/spark-jobs/src/main/scala/filodb/downsampler/chunk/BatchDownsampler.scala index 7a7ffb970f..acc992aad1 100644 --- a/spark-jobs/src/main/scala/filodb/downsampler/chunk/BatchDownsampler.scala +++ b/spark-jobs/src/main/scala/filodb/downsampler/chunk/BatchDownsampler.scala @@ -5,6 +5,7 @@ import scala.concurrent.Await import scala.concurrent.duration.FiniteDuration import kamon.Kamon +import kamon.metric.MeasurementUnit import monix.reactive.Observable import spire.syntax.cfor._ @@ -49,6 +50,13 @@ class BatchDownsampler(settings: DownsamplerSettings) extends Instance with Seri @transient lazy val numRawChunksDownsampled = Kamon.counter("num-raw-chunks-downsampled").withoutTags() @transient lazy val numDownsampledChunksWritten = Kamon.counter("num-downsampled-chunks-written").withoutTags() + @transient lazy val downsampleBatchLatency = Kamon.histogram("downsample-batch-latency", + MeasurementUnit.time.milliseconds).withoutTags() + @transient lazy val downsampleSinglePartLatency = Kamon.histogram("downsample-single-partition-latency", + MeasurementUnit.time.milliseconds).withoutTags() + @transient lazy val downsampleBatchPersistLatency = Kamon.histogram("cassandra-downsample-batch-persist-latency", + MeasurementUnit.time.milliseconds).withoutTags() + @transient lazy private val session = DownsamplerContext.getOrCreateCassandraSession(settings.cassandraConfig) @transient lazy private[downsampler] val downsampleCassandraColStore = @@ -110,7 +118,6 @@ class BatchDownsampler(settings: DownsamplerSettings) extends Instance with Seri def downsampleBatch(rawPartsBatch: Seq[RawPartData], userTimeStart: Long, userTimeEndExclusive: Long): Unit = { - val batchSpan = Kamon.spanBuilder("downsample-batch-latency").start() DownsamplerContext.dsLogger.info(s"Starting to downsample batchSize=${rawPartsBatch.size} partitions " + s"rawDataset=${settings.rawDatasetName} for " + s"userTimeStart=${java.time.Instant.ofEpochMilli(userTimeStart)} " + @@ -158,7 +165,7 @@ class BatchDownsampler(settings: DownsamplerSettings) extends Instance with Seri numBatchesFailed.increment() throw e // will be logged by spark } finally { - batchSpan.finish() + downsampleBatchLatency.record(System.currentTimeMillis() - startedAt) offHeapMem.free() // free offheap mem pagedPartsToFree.clear() downsampledPartsToFree.clear() @@ -221,7 +228,7 @@ class BatchDownsampler(settings: DownsamplerSettings) extends Instance with Seri res -> part }.toMap - val downsamplePartSpan = Kamon.spanBuilder("downsample-single-partition-latency").start() + val downsamplePartStart = System.currentTimeMillis() downsampleChunks(offHeapMem, rawReadablePart, downsamplers, periodMarker, downsampledParts, userTimeStart, userTimeEndExclusive, dsRecordBuilder, shouldTrace) @@ -233,7 +240,7 @@ class BatchDownsampler(settings: DownsamplerSettings) extends Instance with Seri val newIt = downsampledChunksToPersist(res) ++ dsPartition.makeFlushChunks(offHeapMem.blockMemFactory) downsampledChunksToPersist(res) = newIt } - downsamplePartSpan.finish() + downsampleSinglePartLatency.record(System.currentTimeMillis() - downsamplePartStart) case None => numPartitionsNoDownsampleSchema.increment() DownsamplerContext.dsLogger.debug(s"Skipping downsampling of partition " + @@ -328,7 +335,8 @@ class BatchDownsampler(settings: DownsamplerSettings) extends Instance with Seri for {c <- dsRecordBuilder.allContainers row <- c.iterate(part.schema.ingestionSchema) } { - part.ingest(userTimeEndExclusive, row, offHeapMem.blockMemFactory) + part.ingest(userTimeEndExclusive, row, offHeapMem.blockMemFactory, createChunkAtFlushBoundary = false, + flushIntervalMillis = Option.empty) } } catch { case e: Exception => @@ -362,7 +370,7 @@ class BatchDownsampler(settings: DownsamplerSettings) extends Instance with Seri * Persist chunks in `downsampledChunksToPersist` to Cassandra. */ private def persistDownsampledChunks(downsampledChunksToPersist: MMap[FiniteDuration, Iterator[ChunkSet]]): Int = { - val batchWriteSpan = Kamon.spanBuilder("cassandra-downsample-batch-persist-latency").start() + val start = System.currentTimeMillis() @volatile var numChunks = 0 // write all chunks to cassandra val writeFut = downsampledChunksToPersist.map { case (res, chunks) => @@ -384,7 +392,7 @@ class BatchDownsampler(settings: DownsamplerSettings) extends Instance with Seri DownsamplerContext.dsLogger.error(s"Got response $response when writing to Cassandra") } numDownsampledChunksWritten.increment(numChunks) - batchWriteSpan.finish() + downsampleBatchPersistLatency.record(System.currentTimeMillis() - start) numChunks } diff --git a/spark-jobs/src/main/scala/filodb/downsampler/index/DSIndexJob.scala b/spark-jobs/src/main/scala/filodb/downsampler/index/DSIndexJob.scala index 61ff5a43ca..41bffa740c 100644 --- a/spark-jobs/src/main/scala/filodb/downsampler/index/DSIndexJob.scala +++ b/spark-jobs/src/main/scala/filodb/downsampler/index/DSIndexJob.scala @@ -3,6 +3,7 @@ package filodb.downsampler.index import scala.concurrent.Await import kamon.Kamon +import kamon.metric.MeasurementUnit import monix.reactive.Observable import filodb.cassandra.columnstore.CassandraColumnStore @@ -24,6 +25,8 @@ class DSIndexJob(dsSettings: DownsamplerSettings, @transient lazy private val numPartKeysNoDownsampleSchema = Kamon.counter("num-partkeys-no-downsample").withoutTags() @transient lazy private val numPartKeysMigrated = Kamon.counter("num-partkeys-migrated").withoutTags() @transient lazy private val numPartKeysBlocked = Kamon.counter("num-partkeys-blocked").withoutTags() + @transient lazy val perShardIndexMigrationLatency = Kamon.histogram("per-shard-index-migration-latency", + MeasurementUnit.time.milliseconds).withoutTags() @transient lazy private[downsampler] val schemas = Schemas.fromConfig(dsSettings.filodbConfig).get @@ -59,10 +62,7 @@ class DSIndexJob(dsSettings: DownsamplerSettings, val rawDataSource = rawCassandraColStore @volatile var count = 0 try { - val span = Kamon.spanBuilder("per-shard-index-migration-latency") - .asChildOf(Kamon.currentSpan()) - .tag("shard", shard) - .start + val start = System.currentTimeMillis() if (fullIndexMigration) { DownsamplerContext.dsLogger.info(s"Starting Full PartKey Migration for shard=$shard") val partKeys = rawDataSource.scanPartKeys(ref = rawDatasetRef, @@ -80,7 +80,7 @@ class DSIndexJob(dsSettings: DownsamplerSettings, s"count=$count fromHour=$fromHour toHourExcl=$toHourExcl") } sparkForeachTasksCompleted.increment() - span.finish() + perShardIndexMigrationLatency.record(System.currentTimeMillis() - start) } catch { case e: Exception => DownsamplerContext.dsLogger.error(s"Exception in task count=$count " + s"shard=$shard fromHour=$fromHour toHourExcl=$toHourExcl fullIndexMigration=$fullIndexMigration", e) diff --git a/spark-jobs/src/test/scala/filodb/downsampler/DownsamplerMainSpec.scala b/spark-jobs/src/test/scala/filodb/downsampler/DownsamplerMainSpec.scala index 2c9bae0476..6a749b9ced 100644 --- a/spark-jobs/src/test/scala/filodb/downsampler/DownsamplerMainSpec.scala +++ b/spark-jobs/src/test/scala/filodb/downsampler/DownsamplerMainSpec.scala @@ -132,7 +132,8 @@ class DownsamplerMainSpec extends AnyFunSpec with Matchers with BeforeAndAfterAl MachineMetricsData.records(rawDataset, rawSamples).records.foreach { case (base, offset) => val rr = new BinaryRecordRowReader(Schemas.untyped.ingestionSchema, base, offset) - part.ingest( lastSampleTime, rr, offheapMem.blockMemFactory) + part.ingest(lastSampleTime, rr, offheapMem.blockMemFactory, createChunkAtFlushBoundary = false, + flushIntervalMillis = Option.empty) } part.switchBuffers(offheapMem.blockMemFactory, true) val chunks = part.makeFlushChunks(offheapMem.blockMemFactory) @@ -174,7 +175,8 @@ class DownsamplerMainSpec extends AnyFunSpec with Matchers with BeforeAndAfterAl MachineMetricsData.records(rawDataset, rawSamples).records.foreach { case (base, offset) => val rr = new BinaryRecordRowReader(Schemas.gauge.ingestionSchema, base, offset) - part.ingest( lastSampleTime, rr, offheapMem.blockMemFactory) + part.ingest( lastSampleTime, rr, offheapMem.blockMemFactory, createChunkAtFlushBoundary = false, + flushIntervalMillis = Option.empty) } part.switchBuffers(offheapMem.blockMemFactory, true) val chunks = part.makeFlushChunks(offheapMem.blockMemFactory) @@ -214,7 +216,8 @@ class DownsamplerMainSpec extends AnyFunSpec with Matchers with BeforeAndAfterAl MachineMetricsData.records(rawDataset, rawSamples).records.foreach { case (base, offset) => val rr = new BinaryRecordRowReader(Schemas.gauge.ingestionSchema, base, offset) - part.ingest( lastSampleTime, rr, offheapMem.blockMemFactory) + part.ingest( lastSampleTime, rr, offheapMem.blockMemFactory, createChunkAtFlushBoundary = false, + flushIntervalMillis = Option.empty) } part.switchBuffers(offheapMem.blockMemFactory, true) val chunks = part.makeFlushChunks(offheapMem.blockMemFactory) @@ -260,7 +263,8 @@ class DownsamplerMainSpec extends AnyFunSpec with Matchers with BeforeAndAfterAl MachineMetricsData.records(rawDataset, rawSamples).records.foreach { case (base, offset) => val rr = new BinaryRecordRowReader(Schemas.promCounter.ingestionSchema, base, offset) - part.ingest( lastSampleTime, rr, offheapMem.blockMemFactory) + part.ingest( lastSampleTime, rr, offheapMem.blockMemFactory, createChunkAtFlushBoundary = false, + flushIntervalMillis = Option.empty) } part.switchBuffers(offheapMem.blockMemFactory, true) val chunks = part.makeFlushChunks(offheapMem.blockMemFactory) @@ -307,7 +311,8 @@ class DownsamplerMainSpec extends AnyFunSpec with Matchers with BeforeAndAfterAl MachineMetricsData.records(rawDataset, rawSamples).records.foreach { case (base, offset) => val rr = new BinaryRecordRowReader(Schemas.promHistogram.ingestionSchema, base, offset) - part.ingest( lastSampleTime, rr, offheapMem.blockMemFactory) + part.ingest( lastSampleTime, rr, offheapMem.blockMemFactory, createChunkAtFlushBoundary = false, + flushIntervalMillis = Option.empty) } part.switchBuffers(offheapMem.blockMemFactory, true) val chunks = part.makeFlushChunks(offheapMem.blockMemFactory) @@ -665,8 +670,8 @@ class DownsamplerMainSpec extends AnyFunSpec with Matchers with BeforeAndAfterAl Seq(gaugeName, gaugeLowFreqName, counterName, histName).foreach { metricName => val queryFilters = colFilters :+ ColumnFilter("_metric_", Equals(metricName)) - val exec = MultiSchemaPartitionsExec(QueryContext(sampleLimit = 1000), InProcessPlanDispatcher, - batchDownsampler.rawDatasetRef, 0, queryFilters, AllChunkScan) + val exec = MultiSchemaPartitionsExec(QueryContext(plannerParams = PlannerParams(sampleLimit = 1000)), + InProcessPlanDispatcher, batchDownsampler.rawDatasetRef, 0, queryFilters, AllChunkScan) val querySession = QuerySession(QueryContext(), queryConfig) val queryScheduler = Scheduler.fixedPool(s"$QuerySchedName", 3) @@ -693,8 +698,9 @@ class DownsamplerMainSpec extends AnyFunSpec with Matchers with BeforeAndAfterAl val colFilters = seriesTags.map { case (t, v) => ColumnFilter(t.toString, Equals(v.toString)) }.toSeq val queryFilters = colFilters :+ ColumnFilter("_metric_", Equals(untypedName)) - val exec = MultiSchemaPartitionsExec(QueryContext(sampleLimit = 1000), InProcessPlanDispatcher, - batchDownsampler.rawDatasetRef, 0, queryFilters, AllChunkScan) + val exec = MultiSchemaPartitionsExec(QueryContext(plannerParams + = PlannerParams(sampleLimit = 1000)), InProcessPlanDispatcher, batchDownsampler.rawDatasetRef, 0, + queryFilters, AllChunkScan) val querySession = QuerySession(QueryContext(), queryConfig) val queryScheduler = Scheduler.fixedPool(s"$QuerySchedName", 3) @@ -715,9 +721,8 @@ class DownsamplerMainSpec extends AnyFunSpec with Matchers with BeforeAndAfterAl 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)) - val exec = MultiSchemaPartitionsExec(QueryContext(sampleLimit = 1000), InProcessPlanDispatcher, - batchDownsampler.rawDatasetRef, 0, queryFilters, AllChunkScan, - colName = Option("sum")) + val exec = MultiSchemaPartitionsExec(QueryContext(plannerParams = PlannerParams(sampleLimit = 1000)), + InProcessPlanDispatcher, batchDownsampler.rawDatasetRef, 0, queryFilters, AllChunkScan, colName = Option("sum")) val querySession = QuerySession(QueryContext(), queryConfig) val queryScheduler = Scheduler.fixedPool(s"$QuerySchedName", 3) val res = exec.execute(downsampleTSStore, querySession)(queryScheduler) diff --git a/version.sbt b/version.sbt index c0bd680cb2..06316d5a23 100644 --- a/version.sbt +++ b/version.sbt @@ -1 +1 @@ -version in ThisBuild := "0.9.10" \ No newline at end of file +version in ThisBuild := "0.9.11" \ No newline at end of file