diff --git a/README.md b/README.md index 1192836ccc..c7f9230c04 100644 --- a/README.md +++ b/README.md @@ -340,7 +340,7 @@ If you had run the unit test `DownsamplerMainSpec` which populates data into the dataset, you can query downsample results by visiting the following URL: ``` -curl "http://localhost:8080/promql/prometheus/api/v1/query_range?query=my_counter\{_ws_='my_ws',_ns_='my_ns'\}&start=1574272801&end=1574273042&step=10&verbose=true&spread=2" +curl "http://localhost:9080/promql/prometheus/api/v1/query_range?query=my_counter\{_ws_='my_ws',_ns_='my_ns'\}&start=74372801&end=74373042&step=10&verbose=true&spread=2" ``` #### Local Scale Testing diff --git a/conf/timeseries-dev-source.conf b/conf/timeseries-dev-source.conf index 50a532b89a..17cfd9ea8a 100644 --- a/conf/timeseries-dev-source.conf +++ b/conf/timeseries-dev-source.conf @@ -95,9 +95,8 @@ # metric = "bad-metric-to-log" # } - # Maximum number of partitions per shard scanned per query. This is necessary - # to ensure no run-away query hogs memory and destabilizes the server. - # max-query-matches = 250000 + # Limits maximum amount of data a single leaf query can scan + max-data-per-shard-query = 50 MB } downsample { # can be disabled by setting this flag to false diff --git a/coordinator/src/main/scala/filodb.coordinator/IngestionActor.scala b/coordinator/src/main/scala/filodb.coordinator/IngestionActor.scala index 4ecb672c8d..7a9f21a412 100644 --- a/coordinator/src/main/scala/filodb.coordinator/IngestionActor.scala +++ b/coordinator/src/main/scala/filodb.coordinator/IngestionActor.scala @@ -109,6 +109,7 @@ private[filodb] final class IngestionActor(ref: DatasetRef, * reconciles any differences. It does so by stopping ingestion for shards that aren't mapped * to this node, and it starts ingestion for those that are. */ + // scalastyle:off method.length private def resync(state: ShardIngestionState, origin: ActorRef): Unit = { if (invalid(state.ref)) { logger.error(s"$state is invalid for this ingester '$ref'.") @@ -144,7 +145,13 @@ private[filodb] final class IngestionActor(ref: DatasetRef, } } else { val status = state.map.statuses(shard) - logger.info(s"Will stop ingestion of for dataset=$ref shard=$shard due to status ${status}") + if (shardsToStop.contains(shard)) { + logger.info(s"Will stop ingestion for dataset=$ref shard=$shard due to status ${status}") + } else { + // Already stopped. Send the message again in case it got dropped. + logger.info(s"Stopping ingestion again for dataset=$ref shard=$shard due to status ${status}") + sendStopMessage(shard) + } } } } @@ -237,9 +244,7 @@ private[filodb] final class IngestionActor(ref: DatasetRef, // Define a cancel task to run when ingestion is stopped. val onCancel = Task { logger.info(s"Ingestion cancel task invoked for dataset=$ref shard=$shard") - val stopped = IngestionStopped(ref, shard) - self ! stopped - statusActor ! stopped + sendStopMessage(shard) } val shardIngestionEnd = memStore.ingestStream(ref, @@ -268,6 +273,12 @@ private[filodb] final class IngestionActor(ref: DatasetRef, } } + private def sendStopMessage(shard: Int): Unit = { + val stopped = IngestionStopped(ref, shard) + self ! stopped + statusActor ! stopped + } + import Iterators._ /** diff --git a/coordinator/src/main/scala/filodb.coordinator/KamonLogger.scala b/coordinator/src/main/scala/filodb.coordinator/KamonLogger.scala index c86b873c13..6e919c09ba 100644 --- a/coordinator/src/main/scala/filodb.coordinator/KamonLogger.scala +++ b/coordinator/src/main/scala/filodb.coordinator/KamonLogger.scala @@ -1,7 +1,10 @@ package filodb.coordinator +import scala.concurrent.Await + import com.typesafe.config.Config import com.typesafe.scalalogging.StrictLogging +import kamon.Kamon import kamon.metric.{MeasurementUnit, MetricSnapshot, PeriodSnapshot} import kamon.metric.MeasurementUnit.{information, time} import kamon.metric.MeasurementUnit.Dimension.{Information, Time} @@ -119,3 +122,26 @@ object KamonLogger { new KamonSpanLogReporter } } + +object KamonShutdownHook extends StrictLogging { + + import scala.concurrent.duration._ + + private val shutdownHookAdded = new java.util.concurrent.atomic.AtomicBoolean(false) + def registerShutdownHook(): Unit = { + if (shutdownHookAdded.compareAndSet(false, true)) { + logger.info(s"Registering Kamon Shutdown Hook...") + Runtime.getRuntime.addShutdownHook(new Thread() { + override def run(): Unit = { + logger.info(s"Stopping Kamon modules - this will ensure that last few metrics are drained") + try { + Await.result(Kamon.stopModules(), 5.minutes) + logger.info(s"Finished stopping Kamon modules") + } catch { case e: Exception => + logger.error(s"Exception when stopping Kamon Modules", e) + } + } + }) + } + } +} \ No newline at end of file diff --git a/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala b/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala index c5db07aafa..400a7fc0ab 100644 --- a/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala +++ b/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala @@ -1,5 +1,7 @@ package filodb.coordinator +import java.lang.Thread.UncaughtExceptionHandler +import java.util.concurrent.{ScheduledThreadPoolExecutor, ThreadFactory} import java.util.concurrent.atomic.AtomicLong import scala.util.control.NonFatal @@ -8,8 +10,10 @@ import akka.actor.{ActorRef, ActorSystem, Props} import akka.dispatch.{Envelope, UnboundedStablePriorityMailbox} import com.typesafe.config.Config import kamon.Kamon +import kamon.instrumentation.executor.ExecutorInstrumentation import kamon.tag.TagSet import monix.execution.Scheduler +import monix.execution.schedulers.SchedulerService import net.ceedubs.ficus.Ficus._ import net.ceedubs.ficus.readers.ValueReader @@ -17,7 +21,7 @@ import filodb.coordinator.queryplanner.SingleClusterPlanner import filodb.core._ import filodb.core.memstore.{FiloSchedulers, MemStore, TermInfo} import filodb.core.metadata.Schemas -import filodb.core.query.QueryContext +import filodb.core.query.{QueryConfig, QueryContext, QuerySession} import filodb.core.store.CorruptVectorException import filodb.query._ import filodb.query.exec.ExecPlan @@ -88,8 +92,7 @@ final class QueryActor(memStore: MemStore, val queryConfig = new QueryConfig(config.getConfig("filodb.query")) val queryPlanner = new SingleClusterPlanner(dsRef, schemas, shardMapFunc, earliestRawTimestampFn, queryConfig, functionalSpreadProvider) - val numSchedThreads = Math.ceil(config.getDouble("filodb.query.threads-factor") * sys.runtime.availableProcessors) - val queryScheduler = Scheduler.fixedPool(s"$QuerySchedName-$dsRef", numSchedThreads.toInt) + val queryScheduler = createInstrumentedQueryScheduler() private val tags = Map("dataset" -> dsRef.toString) private val lpRequests = Kamon.counter("queryactor-logicalPlan-requests").withTags(TagSet.from(tags)) @@ -97,54 +100,99 @@ final class QueryActor(memStore: MemStore, private val resultVectors = Kamon.histogram("queryactor-result-num-rvs").withTags(TagSet.from(tags)) private val queryErrors = Kamon.counter("queryactor-query-errors").withTags(TagSet.from(tags)) + /** + * Instrumentation adds following metrics on the Query Scheduler + * + * # Counter + * executor_tasks_submitted_total{type="ThreadPoolExecutor",name="query-sched-prometheus"} + * # Counter + * executor_tasks_completed_total{type="ThreadPoolExecutor",name="query-sched-prometheus"} + * # Histogram + * executor_threads_active{type="ThreadPoolExecutor",name="query-sched-prometheus"} + * # Histogram + * executor_queue_size_count{type="ThreadPoolExecutor",name="query-sched-prometheus"} + * + */ + private def createInstrumentedQueryScheduler(): SchedulerService = { + val numSchedThreads = Math.ceil(config.getDouble("filodb.query.threads-factor") + * sys.runtime.availableProcessors).toInt + val schedName = s"$QuerySchedName-$dsRef" + + val thFactory = new ThreadFactory { + def newThread(r: Runnable) = { + val thread = new Thread(r) + thread.setName(s"$schedName-${thread.getId}") + thread.setDaemon(true) + thread.setUncaughtExceptionHandler( + new UncaughtExceptionHandler { + override def uncaughtException(t: Thread, e: Throwable): Unit = + logger.error("Uncaught Exception in Query Scheduler", e) + }) + thread + } + } + // TODO retaining old fixed size pool for now - later change to fork join pool. + val executor = new ScheduledThreadPoolExecutor(numSchedThreads, thFactory) + Scheduler.apply(ExecutorInstrumentation.instrument(executor, schedName)) + } + def execPhysicalPlan2(q: ExecPlan, replyTo: ActorRef): Unit = { - epRequests.increment() - Kamon.currentSpan().tag("query", q.getClass.getSimpleName) - Kamon.currentSpan().tag("query-id", q.queryContext.queryId) - q.execute(memStore, queryConfig)(queryScheduler) - .foreach { res => - FiloSchedulers.assertThreadName(QuerySchedName) - 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 => - // 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) + 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) + } } private def processLogicalPlan2Query(q: LogicalPlan2Query, replyTo: ActorRef) = { - // This is for CLI use only. Always prefer clients to materialize logical plan - lpRequests.increment() - try { - val execPlan = queryPlanner.materialize(q.logicalPlan, q.qContext) - self forward execPlan - } catch { - case NonFatal(ex) => - if (!ex.isInstanceOf[BadQueryException]) // dont log user errors - logger.error(s"Exception while materializing logical plan", ex) - replyTo ! QueryError("unknown", ex) + if (checkTimeout(q.qContext, replyTo)) { + // This is for CLI use only. Always prefer clients to materialize logical plan + lpRequests.increment() + try { + val execPlan = queryPlanner.materialize(q.logicalPlan, q.qContext) + self forward execPlan + } catch { + case NonFatal(ex) => + if (!ex.isInstanceOf[BadQueryException]) // dont log user errors + logger.error(s"Exception while materializing logical plan", ex) + replyTo ! QueryError("unknown", ex) + } } } private def processExplainPlanQuery(q: ExplainPlan2Query, replyTo: ActorRef): Unit = { - try { - val execPlan = queryPlanner.materialize(q.logicalPlan, q.qContext) - replyTo ! execPlan - } catch { - case NonFatal(ex) => - if (!ex.isInstanceOf[BadQueryException]) // dont log user errors - logger.error(s"Exception while materializing logical plan", ex) - replyTo ! QueryError("unknown", ex) + if (checkTimeout(q.qContext, replyTo)) { + try { + val execPlan = queryPlanner.materialize(q.logicalPlan, q.qContext) + replyTo ! execPlan + } catch { + case NonFatal(ex) => + if (!ex.isInstanceOf[BadQueryException]) // dont log user errors + logger.error(s"Exception while materializing logical plan", ex) + replyTo ! QueryError("unknown", ex) + } } } @@ -160,6 +208,16 @@ final class QueryActor(memStore: MemStore, } } + 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}", + QueryTimeoutException(queryTimeElapsed, this.getClass.getName)) + false + } else true + } + def receive: Receive = { case q: LogicalPlan2Query => val replyTo = sender() processLogicalPlan2Query(q, replyTo) diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/CompositePlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/CompositePlanner.scala deleted file mode 100644 index 8264ab6a24..0000000000 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/CompositePlanner.scala +++ /dev/null @@ -1,41 +0,0 @@ -package filodb.coordinator.queryplanner - -import com.typesafe.scalalogging.StrictLogging - -import filodb.coordinator.ShardMapper -import filodb.coordinator.client.QueryCommands.StaticSpreadProvider -import filodb.core.{DatasetRef, SpreadProvider} -import filodb.core.metadata.Schemas -import filodb.core.query.QueryContext -import filodb.query._ -import filodb.query.exec._ - -/** - * Query Planner implementation that composes other planners to provide overall capability - * of high availability, downsampling and (later) multi-cluster partitioning. - */ -class CompositePlanner(dsRef: DatasetRef, - schemas: Schemas, - shardMapperFunc: => ShardMapper, - downsampleMapperFunc: => ShardMapper, - failureProvider: FailureProvider, - earliestRawTimestampFn: => Long, - earliestDownsampleTimestampFn: => Long, - queryConfig: QueryConfig, - spreadProvider: SpreadProvider = StaticSpreadProvider(), - stitchDispatcher: => PlanDispatcher = { InProcessPlanDispatcher }) extends QueryPlanner - with StrictLogging { - // Note the composition of query planners below using decorator pattern - val rawClusterPlanner = new SingleClusterPlanner(dsRef, schemas, shardMapperFunc, - earliestRawTimestampFn, queryConfig, spreadProvider) - val downsampleClusterPlanner = new SingleClusterPlanner(dsRef, schemas, downsampleMapperFunc, - earliestDownsampleTimestampFn, queryConfig, spreadProvider) - val longTimeRangePlanner = new LongTimeRangePlanner(rawClusterPlanner, downsampleClusterPlanner, - earliestRawTimestampFn, stitchDispatcher) - val haPlanner = new HighAvailabilityPlanner(dsRef, longTimeRangePlanner, failureProvider, queryConfig) - //val multiPodPlanner = new MultiClusterPlanner(podLocalityProvider, haPlanner) - - def materialize(rootLogicalPlan: LogicalPlan, options: QueryContext): ExecPlan = { - haPlanner.materialize(rootLogicalPlan, options) - } -} diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/HighAvailabilityPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/HighAvailabilityPlanner.scala index ae9a8e8028..faa3b0074a 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/HighAvailabilityPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/HighAvailabilityPlanner.scala @@ -3,8 +3,8 @@ package filodb.coordinator.queryplanner import com.typesafe.scalalogging.StrictLogging import filodb.core.DatasetRef -import filodb.core.query.{PromQlQueryParams, QueryContext} -import filodb.query.{LogicalPlan, QueryConfig} +import filodb.core.query.{PromQlQueryParams, QueryConfig, QueryContext} +import filodb.query.LogicalPlan import filodb.query.exec.{ExecPlan, InProcessPlanDispatcher, PromQlExec, StitchRvsExec} /** diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala index e0c05dcf90..92b2cd9642 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala @@ -1,5 +1,6 @@ package filodb.coordinator.queryplanner +import filodb.prometheus.ast.Vectors.PromMetricLabel import filodb.prometheus.ast.WindowConstants import filodb.query._ @@ -29,7 +30,7 @@ object LogicalPlanUtils { case lp: ApplyInstantFunction => getPeriodicSeriesTimeFromLogicalPlan(lp.vectors) case lp: Aggregate => getPeriodicSeriesTimeFromLogicalPlan(lp.vectors) case lp: BinaryJoin => // can assume lhs & rhs have same time - getPeriodicSeriesTimeFromLogicalPlan(lp.lhs) + getPeriodicSeriesTimeFromLogicalPlan(lp.lhs) case lp: ScalarVectorBinaryOperation => getPeriodicSeriesTimeFromLogicalPlan(lp.vector) case lp: ApplyMiscellaneousFunction => getPeriodicSeriesTimeFromLogicalPlan(lp.vectors) case lp: ApplySortFunction => getPeriodicSeriesTimeFromLogicalPlan(lp.vectors) @@ -121,4 +122,24 @@ object LogicalPlanUtils { case _ => 0 } } + + def getMetricName(logicalPlan: LogicalPlan, datasetMetricColumn: String): Option[Seq[String]] = { + val metricName = LogicalPlan.getLabelValueFromLogicalPlan(logicalPlan, PromMetricLabel) + if (metricName.isEmpty) LogicalPlan.getLabelValueFromLogicalPlan(logicalPlan, datasetMetricColumn) + else metricName + } + + /** + * Renames Prom AST __name__ label to one based on the actual metric column of the dataset, + * if it is not the prometheus standard + */ + def renameLabels(labels: Seq[String], datasetMetricColumn: String): Seq[String] = + if (datasetMetricColumn != PromMetricLabel) { + labels map { + case PromMetricLabel => datasetMetricColumn + case other: String => other + } + } else { + labels + } } diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LongTimeRangePlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LongTimeRangePlanner.scala index 1795fbd3b3..8d34a14146 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LongTimeRangePlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LongTimeRangePlanner.scala @@ -16,11 +16,18 @@ import filodb.query.exec.{ExecPlan, PlanDispatcher, StitchRvsExec} * abstracts planning for downsample cluster data * @param earliestRawTimestampFn the function that will provide millis timestamp of earliest sample that * would be available in the raw cluster + * @param latestDownsampleTimestampFn the function that will provide millis timestamp of newest sample + * that would be available in the downsample cluster. This typically + * is not "now" because of the delay in population of downsampled data + * via spark job. If job is run every 6 hours, + * `(System.currentTimeMillis - 12.hours.toMillis)` + * may a function that could be passed. 12 hours to account for failures/reruns etc. * @param stitchDispatcher function to get the dispatcher for the stitch exec plan node */ class LongTimeRangePlanner(rawClusterPlanner: QueryPlanner, downsampleClusterPlanner: QueryPlanner, earliestRawTimestampFn: => Long, + latestDownsampleTimestampFn: => Long, stitchDispatcher: => PlanDispatcher) extends QueryPlanner { def materialize(logicalPlan: LogicalPlan, qContext: QueryContext): ExecPlan = { @@ -35,10 +42,19 @@ class LongTimeRangePlanner(rawClusterPlanner: QueryPlanner, else if (endWithOffsetMs < earliestRawTime) downsampleClusterPlanner.materialize(logicalPlan, qContext) else if (startWithOffsetMs - lookbackMs >= earliestRawTime) rawClusterPlanner.materialize(logicalPlan, qContext) - else { + else if (endWithOffsetMs - lookbackMs < earliestRawTime) { + val lastDownsampleSampleTime = latestDownsampleTimestampFn + val downsampleLp = if (endWithOffsetMs < lastDownsampleSampleTime) { + logicalPlan + } else { + copyWithUpdatedTimeRange(logicalPlan, + TimeRange(p.startMs, latestDownsampleTimestampFn + offsetMillis), lookbackMs) + } + downsampleClusterPlanner.materialize(downsampleLp, qContext) + } else { // Split the query between raw and downsample planners val numStepsInDownsample = (earliestRawTime - startWithOffsetMs + lookbackMs) / p.stepMs - val lastDownsampleInstant = startWithOffsetMs + numStepsInDownsample * p.stepMs + val lastDownsampleInstant = p.startMs + numStepsInDownsample * p.stepMs val firstInstantInRaw = lastDownsampleInstant + p.stepMs val downsampleLp = copyWithUpdatedTimeRange(logicalPlan, @@ -50,9 +66,7 @@ class LongTimeRangePlanner(rawClusterPlanner: QueryPlanner, val rawEp = rawClusterPlanner.materialize(rawLp, qContext) StitchRvsExec(qContext, stitchDispatcher, Seq(rawEp, downsampleEp)) } - case _ => - // for now send everything else to raw cluster. Metadata queries are TODO - rawClusterPlanner.materialize(logicalPlan, qContext) + case _ => rawClusterPlanner.materialize(logicalPlan, qContext) } } } diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiClusterPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiClusterPlanner.scala deleted file mode 100644 index 54cef39f8e..0000000000 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiClusterPlanner.scala +++ /dev/null @@ -1,19 +0,0 @@ -package filodb.coordinator.queryplanner - -import filodb.core.query.QueryContext -import filodb.query.LogicalPlan -import filodb.query.exec.ExecPlan - -/** - * MultiClusterPlanner is responsible for planning in situations where time series data is - * distributed across multiple clusters. - * - * This is TBD. - */ -class MultiClusterPlanner extends QueryPlanner { - - def materialize(logicalPlan: LogicalPlan, qContext: QueryContext): ExecPlan = { - ??? - } - -} diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala index 05f54b6bdb..b75bbaaaae 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala @@ -11,7 +11,7 @@ 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, QueryContext, RangeParams} +import filodb.core.query.{ColumnFilter, Filter, QueryConfig, QueryContext, RangeParams} import filodb.core.store.{AllChunkScan, ChunkScanMethod, InMemoryChunkScan, TimeRangeChunkScan, WriteBufferChunkScan} import filodb.prometheus.ast.Vectors.{PromMetricLabel, TypeLabel} import filodb.prometheus.ast.WindowConstants @@ -73,6 +73,7 @@ class SingleClusterPlanner(dsRef: DatasetRef, def materialize(logicalPlan: LogicalPlan, qContext: QueryContext): ExecPlan = { + if (shardMapperFunc.numShards == 0) throw new IllegalStateException("No shards available") val materialized = walkLogicalPlanTree(logicalPlan, qContext) match { case PlanResult(Seq(justOne), stitch) => @@ -153,20 +154,6 @@ class SingleClusterPlanner(dsRef: DatasetRef, filters } - /** - * Renames Prom AST __name__ label to one based on the actual metric column of the dataset, - * if it is not the prometheus standard - */ - private def renameLabels(labels: Seq[String]): Seq[String] = - if (dsOptions.metricColumn != PromMetricLabel) { - labels map { - case PromMetricLabel => dsOptions.metricColumn - case other: String => other - } - } else { - labels - } - /** * Walk logical plan tree depth-first and generate execution plans starting from the bottom * @@ -225,10 +212,13 @@ class SingleClusterPlanner(dsRef: DatasetRef, val targetActor = pickDispatcher(stitchedLhs ++ stitchedRhs) val joined = if (lp.operator.isInstanceOf[SetOperator]) Seq(exec.SetOperatorExec(qContext, targetActor, stitchedLhs, stitchedRhs, lp.operator, - renameLabels(lp.on), renameLabels(lp.ignoring), dsOptions.metricColumn)) + LogicalPlanUtils.renameLabels(lp.on, dsOptions.metricColumn), + LogicalPlanUtils.renameLabels(lp.ignoring, dsOptions.metricColumn), dsOptions.metricColumn)) else Seq(BinaryJoinExec(qContext, targetActor, stitchedLhs, stitchedRhs, lp.operator, lp.cardinality, - renameLabels(lp.on), renameLabels(lp.ignoring), renameLabels(lp.include), dsOptions.metricColumn)) + LogicalPlanUtils.renameLabels(lp.on, dsOptions.metricColumn), + LogicalPlanUtils.renameLabels(lp.ignoring, dsOptions.metricColumn), + LogicalPlanUtils.renameLabels(lp.include, dsOptions.metricColumn), dsOptions.metricColumn)) PlanResult(joined, false) } @@ -248,8 +238,9 @@ class SingleClusterPlanner(dsRef: DatasetRef, * Starting off with solution 1 first until (2) or some other approach is decided on. */ toReduceLevel1.plans.foreach { - _.addRangeVectorTransformer(AggregateMapReduce(lp.operator, lp.params, renameLabels(lp.without), - renameLabels(lp.by))) + _.addRangeVectorTransformer(AggregateMapReduce(lp.operator, lp.params, + LogicalPlanUtils.renameLabels(lp.without, dsOptions.metricColumn), + LogicalPlanUtils.renameLabels(lp.by, dsOptions.metricColumn))) } val toReduceLevel2 = diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SinglePartitionPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SinglePartitionPlanner.scala new file mode 100644 index 0000000000..b00f7904f2 --- /dev/null +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SinglePartitionPlanner.scala @@ -0,0 +1,78 @@ +package filodb.coordinator.queryplanner + +import filodb.core.query.QueryContext +import filodb.query.{BinaryJoin, LabelValues, LogicalPlan, SeriesKeysByFilters, SetOperator} +import filodb.query.exec._ + +/** + * SinglePartitionPlanner is responsible for planning in situations where time series data is + * distributed across multiple clusters. + * + * @param planners map of clusters names in the local partition to their Planner objects + * @param plannerSelector a function that selects the planner name given the metric name + * + */ +class SinglePartitionPlanner(planners: Map[String, QueryPlanner], plannerSelector: String => String, + datasetMetricColumn: String) + extends QueryPlanner { + + def materialize(logicalPlan: LogicalPlan, qContext: QueryContext): ExecPlan = { + + logicalPlan match { + case lp: BinaryJoin => materializeBinaryJoin(lp, qContext) + case lp: LabelValues => materializeLabelValues(lp, qContext) + case lp: SeriesKeysByFilters => materializeSeriesKeysFilters(lp, qContext) + case _ => materializeSimpleQuery(logicalPlan, qContext) + + } + } + + /** + * Returns planner for first metric in logical plan + * If logical plan does not have metric, first planner present in planners is returned + */ + private def getPlanner(logicalPlan: LogicalPlan): QueryPlanner = { + LogicalPlanUtils.getMetricName(logicalPlan, datasetMetricColumn). + map(x => planners.get(plannerSelector(x.head)).get).getOrElse(planners.values.head) + } + + private def materializeSimpleQuery(logicalPlan: LogicalPlan, qContext: QueryContext): ExecPlan = { + getPlanner(logicalPlan).materialize(logicalPlan, qContext) + } + + private def materializeBinaryJoin(logicalPlan: BinaryJoin, qContext: QueryContext): ExecPlan = { + + val lhsExec = logicalPlan.lhs match { + case b: BinaryJoin => materializeBinaryJoin(b, qContext) + case _ => getPlanner(logicalPlan.lhs).materialize(logicalPlan.lhs, qContext) + } + + val rhsExec = logicalPlan.rhs match { + case b: BinaryJoin => materializeBinaryJoin(b, qContext) + case _ => getPlanner(logicalPlan.rhs).materialize(logicalPlan.rhs, qContext) + } + + if (logicalPlan.operator.isInstanceOf[SetOperator]) + SetOperatorExec(qContext, InProcessPlanDispatcher, Seq(lhsExec), Seq(rhsExec), logicalPlan.operator, + LogicalPlanUtils.renameLabels(logicalPlan.on, datasetMetricColumn), + LogicalPlanUtils.renameLabels(logicalPlan.ignoring, datasetMetricColumn), datasetMetricColumn) + else + BinaryJoinExec(qContext, InProcessPlanDispatcher, Seq(lhsExec), Seq(rhsExec), logicalPlan.operator, + logicalPlan.cardinality, LogicalPlanUtils.renameLabels(logicalPlan.on, datasetMetricColumn), + LogicalPlanUtils.renameLabels(logicalPlan.ignoring, datasetMetricColumn), + LogicalPlanUtils.renameLabels(logicalPlan.include, datasetMetricColumn), datasetMetricColumn) + } + + private def materializeLabelValues(logicalPlan: LogicalPlan, qContext: QueryContext) = { + val execPlans = planners.values.toList.distinct.map(_.materialize(logicalPlan, qContext)) + if (execPlans.size == 1) execPlans.head + else LabelValuesDistConcatExec(qContext, InProcessPlanDispatcher, execPlans) + } + + private def materializeSeriesKeysFilters(logicalPlan: LogicalPlan, qContext: QueryContext) = { + val execPlans = planners.values.toList.distinct.map(_.materialize(logicalPlan, qContext)) + if (execPlans.size == 1) execPlans.head + else PartKeysDistConcatExec(qContext, InProcessPlanDispatcher, execPlans) + } +} + diff --git a/coordinator/src/test/scala/filodb.coordinator/client/SerializationSpec.scala b/coordinator/src/test/scala/filodb.coordinator/client/SerializationSpec.scala index 117a8b8faf..952bc3142a 100644 --- a/coordinator/src/test/scala/filodb.coordinator/client/SerializationSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/client/SerializationSpec.scala @@ -5,12 +5,14 @@ import akka.serialization.SerializationExtension import akka.testkit.TestProbe import com.typesafe.config.ConfigFactory import org.scalatest.concurrent.ScalaFutures + import filodb.coordinator.{ActorSpecConfig, ActorTest, ShardMapper} import filodb.coordinator.queryplanner.SingleClusterPlanner -import filodb.core.{MachineMetricsData, SpreadChange, query} +import filodb.core.{query, MachineMetricsData, SpreadChange} import filodb.core.binaryrecord2.BinaryRecordRowReader import filodb.core.metadata.{Dataset, Schemas} import filodb.core.metadata.Column.ColumnType +import filodb.core.query.QueryConfig import filodb.core.store.IngestionConfig import filodb.memory.format.{RowReader, SeqRowReader, UTF8MapIteratorRowReader, ZeroCopyUTF8String => UTF8Str} import filodb.prometheus.ast.TimeStepParams diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/HighAvailabilityPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/HighAvailabilityPlannerSpec.scala index a20de35328..cd88071a23 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/HighAvailabilityPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/HighAvailabilityPlannerSpec.scala @@ -4,10 +4,11 @@ import akka.actor.ActorSystem import akka.testkit.TestProbe import com.typesafe.config.ConfigFactory import org.scalatest.{FunSpec, Matchers} + import filodb.coordinator.ShardMapper import filodb.core.{DatasetRef, MetricsTestData} import filodb.core.metadata.Schemas -import filodb.core.query.{ColumnFilter, Filter, PromQlQueryParams, QueryContext} +import filodb.core.query.{ColumnFilter, Filter, PromQlQueryParams, QueryConfig, QueryContext} import filodb.core.store.TimeRangeChunkScan import filodb.prometheus.ast.TimeStepParams import filodb.prometheus.parse.Parser diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/LongTimeRangePlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/LongTimeRangePlannerSpec.scala index b31f3d7d33..955c541460 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/LongTimeRangePlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/LongTimeRangePlannerSpec.scala @@ -1,14 +1,16 @@ package filodb.coordinator.queryplanner import scala.concurrent.duration._ + import monix.execution.Scheduler import org.scalatest.{FunSpec, Matchers} + import filodb.core.DatasetRef -import filodb.core.query.QueryContext +import filodb.core.query.{QueryContext, QuerySession} import filodb.core.store.ChunkSource import filodb.prometheus.ast.TimeStepParams import filodb.prometheus.parse.Parser -import filodb.query.{LogicalPlan, PeriodicSeries, PeriodicSeriesPlan, QueryConfig} +import filodb.query.{LogicalPlan, PeriodicSeriesPlan, PeriodicSeriesWithWindowing} import filodb.query.exec._ class LongTimeRangePlannerSpec extends FunSpec with Matchers { @@ -19,7 +21,7 @@ class LongTimeRangePlannerSpec extends FunSpec with Matchers { override def submitTime: Long = ??? override def dataset: DatasetRef = ??? override def dispatcher: PlanDispatcher = ??? - override def doExecute(source: ChunkSource, queryConfig: QueryConfig) + override def doExecute(source: ChunkSource, querySession: QuerySession) (implicit sched: Scheduler): ExecResult = ??? override protected def args: String = ??? } @@ -39,9 +41,11 @@ class LongTimeRangePlannerSpec extends FunSpec with Matchers { val rawRetention = 10.minutes val now = System.currentTimeMillis() / 1000 * 1000 val earliestRawTime = now - rawRetention.toMillis + val latestDownsampleTime = now - 4.minutes.toMillis // say it takes 4 minutes to downsample private def disp = InProcessPlanDispatcher - val longTermPlanner = new LongTimeRangePlanner(rawPlanner, downsamplePlanner, earliestRawTime, disp) + val longTermPlanner = new LongTimeRangePlanner(rawPlanner, downsamplePlanner, + earliestRawTime, latestDownsampleTime, disp) it("should direct raw-cluster-only queries to raw planner") { val logicalPlan = Parser.queryRangeToLogicalPlan("rate(foo[2m])", @@ -92,7 +96,42 @@ class LongTimeRangePlannerSpec extends FunSpec with Matchers { downsampleLp.startMs shouldEqual logicalPlan.startMs downsampleLp.endMs shouldEqual rawStart - 1.minute.toMillis + } + + it("should delegate to downsample cluster and omit recent instants when there is a long lookback") { + + val start = now/1000 - 30.minutes.toSeconds + val step = 1.minute.toSeconds + val end = now/1000 + // notice raw data retention is 10m but lookback is 20m + val logicalPlan = Parser.queryRangeToLogicalPlan("rate(foo[20m])", + TimeStepParams(start, step, end)) + .asInstanceOf[PeriodicSeriesPlan] + + val ep = longTermPlanner.materialize(logicalPlan, QueryContext()) + val downsampleLp = ep.asInstanceOf[MockExecPlan] + downsampleLp.lp.asInstanceOf[PeriodicSeriesPlan].startMs shouldEqual logicalPlan.startMs + downsampleLp.lp.asInstanceOf[PeriodicSeriesPlan].endMs shouldEqual latestDownsampleTime + + } + + it("should delegate to downsample cluster and retain endTime when there is a long lookback with offset that causes " + + "recent data to not be used") { + + val start = now/1000 - 30.minutes.toSeconds + val step = 1.minute.toSeconds + val end = now/1000 + // notice raw data retention is 10m but lookback is 20m + val logicalPlan = Parser.queryRangeToLogicalPlan("rate(foo[20m] offset 5m)", + TimeStepParams(start, step, end)) + .asInstanceOf[PeriodicSeriesPlan] + val ep = longTermPlanner.materialize(logicalPlan, QueryContext()) + val downsampleLp = ep.asInstanceOf[MockExecPlan] + downsampleLp.lp.asInstanceOf[PeriodicSeriesPlan].startMs shouldEqual logicalPlan.startMs + // endTime is retained even with long lookback because 5m offset compensates + // for 4m delay in downsample data population + downsampleLp.lp.asInstanceOf[PeriodicSeriesPlan].endMs shouldEqual logicalPlan.endMs } @@ -120,7 +159,7 @@ class LongTimeRangePlannerSpec extends FunSpec with Matchers { val start = now/1000 - 30.minutes.toSeconds val step = 1.minute.toSeconds val end = now/1000 - 2.minutes.toSeconds - val logicalPlan = Parser.queryRangeToLogicalPlan("foo offset 5m", + val logicalPlan = Parser.queryRangeToLogicalPlan("rate(foo[5m] offset 2m)", TimeStepParams(start, step, end)) .asInstanceOf[PeriodicSeriesPlan] @@ -138,15 +177,15 @@ class LongTimeRangePlannerSpec extends FunSpec with Matchers { // find first instant with range available within raw data val rawStart = ((start*1000) to (end*1000) by (step*1000)).find { instant => - instant - 5.minutes.toMillis > earliestRawTime // subtract offset + instant - (5 + 2).minutes.toMillis > earliestRawTime // subtract lookback & offset }.get rawLp.startMs shouldEqual rawStart rawLp.endMs shouldEqual logicalPlan.endMs - rawLp.asInstanceOf[PeriodicSeries].offsetMs.get shouldEqual(300000) + rawLp.asInstanceOf[PeriodicSeriesWithWindowing].offsetMs.get shouldEqual(120000) downsampleLp.startMs shouldEqual logicalPlan.startMs downsampleLp.endMs shouldEqual rawStart - (step * 1000) - downsampleLp.asInstanceOf[PeriodicSeries].offsetMs.get shouldEqual(300000) + downsampleLp.asInstanceOf[PeriodicSeriesWithWindowing].offsetMs.get shouldEqual(120000) } } diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/ScalarQueriesSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/ScalarQueriesSpec.scala index 49f0f94413..609bbc1186 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/ScalarQueriesSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/ScalarQueriesSpec.scala @@ -8,7 +8,7 @@ import org.scalatest.{FunSpec, Matchers} import filodb.coordinator.ShardMapper import filodb.core.MetricsTestData import filodb.core.metadata.Schemas -import filodb.core.query.{ColumnFilter, Filter, PromQlQueryParams, QueryContext, RangeParams} +import filodb.core.query.{ColumnFilter, Filter, PromQlQueryParams, QueryConfig, QueryContext, RangeParams} import filodb.prometheus.parse.Parser import filodb.query._ import filodb.query.ScalarFunctionId.Time @@ -69,9 +69,9 @@ class ScalarQueriesSpec extends FunSpec with Matchers { """T~ScalarFunctionMapper(function=Scalar, funcParams=List()) |-E~DistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1153666897]) |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=3, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1153666897]) + |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=3, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1153666897]) |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=19, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1153666897]) + |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=19, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1153666897]) """.stripMargin maskDispatcher(execPlan.printTree()) shouldEqual (maskDispatcher(expected)) } @@ -99,11 +99,11 @@ class ScalarQueriesSpec extends FunSpec with Matchers { |-T~ScalarOperationMapper(operator=ADD, scalarOnLhs=false) |--FA1~TimeFuncArgs(RangeParams(1000,1000,1000)) |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1510751596]) + |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1510751596]) |-T~ScalarOperationMapper(operator=ADD, scalarOnLhs=false) |--FA1~TimeFuncArgs(RangeParams(1000,1000,1000)) |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None ) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1510751596])""".stripMargin + |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1510751596])""".stripMargin maskDispatcher(execPlan.printTree()) shouldEqual (maskDispatcher(expected)) } @@ -124,21 +124,21 @@ class ScalarQueriesSpec extends FunSpec with Matchers { |--T~ScalarFunctionMapper(function=Scalar, funcParams=List()) |---E~DistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1643642770]) |----T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1643642770]) + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1643642770]) |----T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1643642770]) + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1643642770]) |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=15, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(node_info))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1643642770]) + |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=15, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(node_info))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1643642770]) |-T~ScalarOperationMapper(operator=ADD, scalarOnLhs=true) |--FA1~ |--T~ScalarFunctionMapper(function=Scalar, funcParams=List()) |---E~DistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1643642770]) |----T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1643642770]) + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1643642770]) |----T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1643642770]) + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1643642770]) |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=31, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(node_info))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1643642770])""".stripMargin + |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=31, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(node_info))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1643642770])""".stripMargin maskDispatcher(execPlan.printTree()) shouldEqual (maskDispatcher(expected)) } @@ -154,11 +154,11 @@ class ScalarQueriesSpec extends FunSpec with Matchers { |-T~ScalarOperationMapper(operator=ADD, scalarOnLhs=true) |--FA1~StaticFuncArgs(10.0,RangeParams(1000,1000,1000)) |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1110105620]) + |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1110105620]) |-T~ScalarOperationMapper(operator=ADD, scalarOnLhs=true) |--FA1~StaticFuncArgs(10.0,RangeParams(1000,1000,1000)) |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1110105620]) + |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1110105620]) |""".stripMargin maskDispatcher(execPlan.printTree()) shouldEqual (maskDispatcher(expected)) } @@ -187,11 +187,11 @@ class ScalarQueriesSpec extends FunSpec with Matchers { |-T~ScalarOperationMapper(operator=ADD, scalarOnLhs=false) |--FA1~StaticFuncArgs(10.0,RangeParams(1000,1000,1000)) |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1245070935]) + |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1245070935]) |-T~ScalarOperationMapper(operator=ADD, scalarOnLhs=false) |--FA1~StaticFuncArgs(10.0,RangeParams(1000,1000,1000)) |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1245070935]) + |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1245070935]) """.stripMargin maskDispatcher(execPlan.printTree()) shouldEqual (maskDispatcher(expected)) } @@ -206,9 +206,9 @@ class ScalarQueriesSpec extends FunSpec with Matchers { |-T~ScalarFunctionMapper(function=Scalar, funcParams=List()) |--E~DistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-2114470773]) |---T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-2114470773]) + |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-2114470773]) |---T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-2114470773]) + |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-2114470773]) |-E~TimeScalarGeneratorExec(params = RangeParams(1000,1000,1000), function = Time) on InProcessPlanDispatcher""".stripMargin maskDispatcher(execPlan.printTree()) shouldEqual (maskDispatcher(expected)) } @@ -223,15 +223,15 @@ class ScalarQueriesSpec extends FunSpec with Matchers { |-T~ScalarFunctionMapper(function=Scalar, funcParams=List()) |--E~DistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1081611650]) |---T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1081611650]) + |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1081611650]) |---T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1081611650]) + |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1081611650]) |-T~ScalarFunctionMapper(function=Scalar, funcParams=List()) |--E~DistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1081611650]) |---T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=15, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(node_info))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1081611650]) + |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=15, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(node_info))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1081611650]) |---T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=31, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(node_info))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1081611650])""".stripMargin + |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=31, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(node_info))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1081611650])""".stripMargin maskDispatcher(execPlan.printTree()) shouldEqual (maskDispatcher(expected)) } @@ -245,9 +245,9 @@ class ScalarQueriesSpec extends FunSpec with Matchers { |-T~ScalarFunctionMapper(function=Scalar, funcParams=List()) |--E~DistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#856852588]) |---T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#856852588]) + |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#856852588]) |---T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#856852588]) + |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#856852588]) |""".stripMargin maskDispatcher(execPlan.printTree()) shouldEqual (maskDispatcher(expected)) } @@ -315,21 +315,21 @@ class ScalarQueriesSpec extends FunSpec with Matchers { |--T~ScalarFunctionMapper(function=Scalar, funcParams=List()) |---E~DistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#79055924]) |----T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#79055924]) + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#79055924]) |----T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#79055924]) + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#79055924]) |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=15, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(node_info))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#79055924]) + |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=15, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(node_info))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#79055924]) |-T~InstantVectorFunctionMapper(function=ClampMax) |--FA1~ |--T~ScalarFunctionMapper(function=Scalar, funcParams=List()) |---E~DistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#79055924]) |----T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#79055924]) + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#79055924]) |----T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#79055924]) + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#79055924]) |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=31, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(node_info))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#79055924])""".stripMargin + |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=31, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(node_info))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#79055924])""".stripMargin maskDispatcher(execPlan.printTree()) shouldEqual (maskDispatcher(expected)) } @@ -365,13 +365,13 @@ class ScalarQueriesSpec extends FunSpec with Matchers { |--T~ScalarOperationMapper(operator=ADD, scalarOnLhs=true) |---FA1~StaticFuncArgs(2.0,RangeParams(1000,1000,1000)) |---T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1098511474]) + |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1098511474]) |-T~ScalarOperationMapper(operator=LSS_BOOL, scalarOnLhs=true) |--FA1~StaticFuncArgs(1.0,RangeParams(1000,1000,1000)) |--T~ScalarOperationMapper(operator=ADD, scalarOnLhs=true) |---FA1~StaticFuncArgs(2.0,RangeParams(1000,1000,1000)) |---T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1098511474]) + |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1098511474]) |""".stripMargin maskDispatcher(execPlan.printTree()) shouldEqual (maskDispatcher(expected)) } @@ -389,21 +389,21 @@ class ScalarQueriesSpec extends FunSpec with Matchers { |--T~ScalarFunctionMapper(function=Scalar, funcParams=List()) |---E~DistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-879546200]) |----T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=15, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(node_info))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-879546200]) + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=15, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(node_info))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-879546200]) |----T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=31, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(node_info))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-879546200]) + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=31, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(node_info))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-879546200]) |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-879546200]) + |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-879546200]) |-T~ScalarOperationMapper(operator=GTR_BOOL, scalarOnLhs=true) |--FA1~ |--T~ScalarFunctionMapper(function=Scalar, funcParams=List()) |---E~DistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-879546200]) |----T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=15, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(node_info))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-879546200]) + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=15, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(node_info))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-879546200]) |----T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=31, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(node_info))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-879546200]) + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=31, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(node_info))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-879546200]) |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-879546200]) + |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-879546200]) |""".stripMargin maskDispatcher(execPlan.printTree()) shouldEqual (maskDispatcher(expected)) } @@ -417,13 +417,13 @@ class ScalarQueriesSpec extends FunSpec with Matchers { val expected = """E~BinaryJoinExec(binaryOp=GTR_BOOL, on=List(), ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1392317349]) |-T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |--E~MultiSchemaPartitionsExec(dataset=timeseries, shard=15, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(node_info))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1392317349]) + |--E~MultiSchemaPartitionsExec(dataset=timeseries, shard=15, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(node_info))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1392317349]) |-T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |--E~MultiSchemaPartitionsExec(dataset=timeseries, shard=31, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(node_info))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1392317349]) + |--E~MultiSchemaPartitionsExec(dataset=timeseries, shard=31, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(node_info))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1392317349]) |-T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |--E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1392317349]) + |--E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1392317349]) |-T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |--E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1392317349]) + |--E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1392317349]) |""".stripMargin maskDispatcher(execPlan.printTree()) shouldEqual (maskDispatcher(expected)) } diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/SingleClusterPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/SingleClusterPlannerSpec.scala index 03e8931f3e..6d0e4eca14 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/SingleClusterPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/SingleClusterPlannerSpec.scala @@ -12,7 +12,7 @@ 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, QueryContext} +import filodb.core.query.{ColumnFilter, Filter, PromQlQueryParams, QueryConfig, QueryContext} import filodb.core.store.TimeRangeChunkScan import filodb.prometheus.ast.{TimeStepParams, WindowConstants} import filodb.prometheus.parse.Parser diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/SinglePartitionPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/SinglePartitionPlannerSpec.scala new file mode 100644 index 0000000000..35e0c7d9d4 --- /dev/null +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/SinglePartitionPlannerSpec.scala @@ -0,0 +1,160 @@ +package filodb.coordinator.queryplanner + +import akka.actor.ActorSystem +import akka.testkit.TestProbe +import com.typesafe.config.ConfigFactory +import monix.execution.Scheduler +import org.scalatest.{FunSpec, Matchers} + +import filodb.coordinator.ShardMapper +import filodb.core.{DatasetRef, MetricsTestData} +import filodb.core.metadata.Schemas +import filodb.core.query.{PromQlQueryParams, QueryConfig, QueryContext, QuerySession} +import filodb.core.store.ChunkSource +import filodb.prometheus.ast.TimeStepParams +import filodb.prometheus.parse.Parser +import filodb.query._ +import filodb.query.exec._ + +class SinglePartitionPlannerSpec extends FunSpec with Matchers{ + private implicit val system = ActorSystem() + private val node = TestProbe().ref + + private val localMapper = new ShardMapper(32) + for {i <- 0 until 32} localMapper.registerNode(Seq(i), node) + + private val remoteMapper = new ShardMapper(16) + for {i <- 0 until 32} localMapper.registerNode(Seq(i), node) + + private val dataset = MetricsTestData.timeseriesDataset + private val dsRef = dataset.ref + private val schemas = Schemas(dataset.schema) + + private val routingConfigString = "routing {\n buddy {\n http {\n timeout = 10.seconds\n }\n }\n}" + private val routingConfig = ConfigFactory.parseString(routingConfigString) + private val config = ConfigFactory.load("application_test.conf").getConfig("filodb.query"). + withFallback(routingConfig) + private val queryConfig = new QueryConfig(config) + + private val promQlQueryParams = PromQlQueryParams(ConfigFactory.empty, "sum(heap_usage)", 100, 1, 1000, None) + + val localPlanner = new SingleClusterPlanner(dsRef, schemas, localMapper, earliestRetainedTimestampFn = 0, queryConfig) + val remotePlanner = new SingleClusterPlanner(dsRef, schemas, remoteMapper, earliestRetainedTimestampFn = 0, + queryConfig) + + val failureProvider = new FailureProvider { + override def getFailures(datasetRef: DatasetRef, queryTimeRange: TimeRange): Seq[FailureTimeRange] = { + Seq(FailureTimeRange("local", datasetRef, + TimeRange(100, 10000), false)) + } + } + + val highAvailabilityPlanner = new HighAvailabilityPlanner(dsRef, localPlanner, failureProvider, queryConfig) + + class MockExecPlan(val name: String, val lp: LogicalPlan) extends ExecPlan { + override def queryContext: QueryContext = QueryContext() + override def children: Seq[ExecPlan] = Nil + override def submitTime: Long = 1000 + override def dataset: DatasetRef = ??? + override def dispatcher: PlanDispatcher = InProcessPlanDispatcher + override def doExecute(source: ChunkSource, querySession: QuerySession) + (implicit sched: Scheduler): ExecResult = ??? + override protected def args: String = "mock-args" + } + + val rrPlanner1 = new QueryPlanner { + override def materialize(logicalPlan: LogicalPlan, qContext: QueryContext): ExecPlan = { + new MockExecPlan("rules1", logicalPlan) + } + } + + val rrPlanner2 = new QueryPlanner { + override def materialize(logicalPlan: LogicalPlan, qContext: QueryContext): ExecPlan = { + new MockExecPlan("rules2", logicalPlan) + } + } + + val planners = Map("local" -> highAvailabilityPlanner, "rules1" -> rrPlanner1, "rules2" -> rrPlanner2) + val plannerSelector = (metricName: String) => { if (metricName.equals("rr1")) "rules1" + else if (metricName.equals("rr2")) "rules2" else "local" } + + val engine = new SinglePartitionPlanner(planners, plannerSelector, "_metric_") + + it("should generate Exec plan for simple query") { + val lp = Parser.queryToLogicalPlan("test{job = \"app\"}", 1000) + + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + execPlan.isInstanceOf[DistConcatExec] shouldEqual (true) + execPlan.children.length shouldEqual 2 + execPlan.children.head.isInstanceOf[MultiSchemaPartitionsExec] shouldEqual true + execPlan.children.head.rangeVectorTransformers.head.isInstanceOf[PeriodicSamplesMapper] shouldEqual true + } + + it("should generate BinaryJoin Exec plan") { + val lp = Parser.queryToLogicalPlan("test1{job = \"app\"} + test2{job = \"app\"}", 1000) + + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + execPlan.printTree() + execPlan.isInstanceOf[BinaryJoinExec] shouldEqual (true) + execPlan.children.foreach { l1 => + l1.isInstanceOf[DistConcatExec] shouldEqual true + l1.children.foreach { l2 => + l2.isInstanceOf[MultiSchemaPartitionsExec] shouldEqual true + l2.rangeVectorTransformers.size shouldEqual 1 + l2.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) + + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + execPlan.isInstanceOf[BinaryJoinExec] shouldEqual (true) + execPlan.asInstanceOf[BinaryJoinExec].lhs.head.isInstanceOf[BinaryJoinExec] shouldEqual true + execPlan.asInstanceOf[BinaryJoinExec].rhs.head.isInstanceOf[DistConcatExec] 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) + + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + execPlan.printTree() + execPlan.isInstanceOf[BinaryJoinExec] shouldEqual (true) + execPlan.asInstanceOf[BinaryJoinExec].rhs.head.asInstanceOf[MockExecPlan].name shouldEqual ("rules1") + execPlan.asInstanceOf[BinaryJoinExec].lhs.head.isInstanceOf[DistConcatExec] shouldEqual true + } + + it("should generate BinaryJoin Exec plan with remote cluster metrics") { + val lp = Parser.queryToLogicalPlan("rr1{job = \"app\"} + rr2{job = \"app\"}", 1000) + + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + 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") + } + + it("should generate Exec plan for Metadata query") { + val lp = Parser.metadataQueryToLogicalPlan("http_requests_total{job=\"prometheus\", method=\"GET\"}", + TimeStepParams(1000, 10, 2000)) + + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + execPlan.printTree() + execPlan.isInstanceOf[PartKeysDistConcatExec] shouldEqual (true) + execPlan.asInstanceOf[PartKeysDistConcatExec].children.length shouldEqual(3) + + // For Raw and Downsample + execPlan.asInstanceOf[PartKeysDistConcatExec].children(0).isInstanceOf[PartKeysDistConcatExec] shouldEqual true + + execPlan.asInstanceOf[PartKeysDistConcatExec].children(1).asInstanceOf[MockExecPlan].name shouldEqual ("rules1") + execPlan.asInstanceOf[PartKeysDistConcatExec].children(2).asInstanceOf[MockExecPlan].name shouldEqual ("rules2") + } + + it("should generate Exec plan for Scalar query which does not have any metric") { + val lp = Parser.queryToLogicalPlan("time()", 1000) + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + execPlan.isInstanceOf[TimeScalarGeneratorExec] shouldEqual true + } + +} + diff --git a/core/src/main/resources/filodb-defaults.conf b/core/src/main/resources/filodb-defaults.conf index b56dbfad9f..b74fc6e635 100644 --- a/core/src/main/resources/filodb-defaults.conf +++ b/core/src/main/resources/filodb-defaults.conf @@ -163,6 +163,7 @@ filodb { # Enable faster rate/increase/delta calculations. Depends on drop detection in chunks (detectDrops=true) faster-rate = true + } shard-manager { @@ -259,8 +260,8 @@ filodb { # Read parallelism in downsample cluster demand-paging-parallelism = 30 - # Maximum number of TS partitions paged in in downsample cluster queries - max-query-matches = 100000 + # Limits maximum amount of data a single leaf query can scan per shard + max-data-per-shard-query = 100 MB # Write buffer size, in bytes, for blob columns (histograms, UTF8Strings). Since these are variable data types, # we need a maximum size, not a maximum number of items. diff --git a/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesShard.scala b/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesShard.scala index f3faadf9d5..9f32528a9e 100644 --- a/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesShard.scala @@ -14,11 +14,11 @@ import monix.eval.Task import monix.execution.{CancelableFuture, Scheduler, UncaughtExceptionReporter} import monix.reactive.Observable -import filodb.core.DatasetRef +import filodb.core.{DatasetRef, Types} import filodb.core.binaryrecord2.RecordSchema import filodb.core.memstore._ import filodb.core.metadata.Schemas -import filodb.core.query.ColumnFilter +import filodb.core.query.{ColumnFilter, QuerySession} import filodb.core.store._ import filodb.memory.format.{UnsafeUtils, ZeroCopyUTF8String} @@ -54,9 +54,6 @@ class DownsampledTimeSeriesShard(rawDatasetRef: DatasetRef, private val downsampleStoreConfig = StoreConfig(filodbConfig.getConfig("downsampler.downsample-store-config")) - // since all partitions are paged from store, this would be much lower than what is configured for raw data - private val maxQueryMatches = downsampleStoreConfig.maxQueryMatches - private val nextPartitionID = new AtomicInteger(0) private val stats = new DownsampledTimeSeriesShardStats(rawDatasetRef, shardNum) @@ -206,7 +203,8 @@ class DownsampledTimeSeriesShard(rawDatasetRef: DatasetRef, def refreshPartKeyIndexBlocking(): Unit = {} def lookupPartitions(partMethod: PartitionScanMethod, - chunkMethod: ChunkScanMethod): PartLookupResult = { + chunkMethod: ChunkScanMethod, + querySession: QuerySession): PartLookupResult = { partMethod match { case SinglePartitionScan(partition, _) => throw new UnsupportedOperationException case MultiPartitionScan(partKeys, _) => throw new UnsupportedOperationException @@ -230,21 +228,20 @@ class DownsampledTimeSeriesShard(rawDatasetRef: DatasetRef, } } - def scanPartitions(lookup: PartLookupResult): Observable[ReadablePartition] = { + def scanPartitions(lookup: PartLookupResult, + colIds: Seq[Types.ColumnId], + querySession: QuerySession): Observable[ReadablePartition] = { // Step 1: Choose the downsample level depending on the range requested - val downsampledDataset = chooseDownsampleResolution(lookup.chunkMethod) + val (resolution, downsampledDataset) = chooseDownsampleResolution(lookup.chunkMethod) logger.debug(s"Chose resolution $downsampledDataset for chunk method ${lookup.chunkMethod}") + + capDataScannedPerShardCheck(lookup, resolution) + // Step 2: Query Cassandra table for that downsample level using downsampleColStore // Create a ReadablePartition objects that contain the time series data. This can be either a // PagedReadablePartitionOnHeap or PagedReadablePartitionOffHeap. This will be garbage collected/freed // when query is complete. - - if (lookup.partsInMemory.length > maxQueryMatches) - throw new IllegalArgumentException(s"Seeing ${lookup.partsInMemory.length} matching time series per shard. Try " + - s"to narrow your query by adding more filters so there is less than $maxQueryMatches matches " + - s"or request for increasing number of shards this metric lives in") - val partKeys = lookup.partsInMemory.iterator().map(partKeyFromPartId) Observable.fromIterator(partKeys) // 3 times value configured for raw dataset since expected throughput for downsampled cluster is much lower @@ -260,17 +257,29 @@ class DownsampledTimeSeriesShard(rawDatasetRef: DatasetRef, SinglePartitionScan(partBytes, shardNum), lookup.chunkMethod) .map { pd => - val part = makePagedPartition(pd, lookup.firstSchemaId.get) + val part = makePagedPartition(pd, lookup.firstSchemaId.get, colIds) stats.partitionsQueried.increment() stats.chunksQueried.increment(part.numChunks) partLoadSpan.finish() part } - .defaultIfEmpty(makePagedPartition(RawPartData(partBytes, Seq.empty), lookup.firstSchemaId.get)) + .defaultIfEmpty(makePagedPartition(RawPartData(partBytes, Seq.empty), lookup.firstSchemaId.get, colIds)) .headL } } + private def capDataScannedPerShardCheck(lookup: PartLookupResult, resolution: Long) = { + lookup.firstSchemaId.foreach { schId => + lookup.chunkMethod match { + case TimeRangeChunkScan(st, end) => + schemas.ensureQueriedDataSizeWithinLimit(schId, lookup.partsInMemory.length, + downsampleStoreConfig.flushInterval.toMillis, + resolution, end - st, downsampleStoreConfig.maxDataPerShardQuery) + case _ => + } + } + } + protected def schemaIDFromPartID(partID: Int): Int = { partKeyIndex.partKeyFromPartId(partID).map { pkBytesRef => val unsafeKeyOffset = PartKeyLuceneIndex.bytesRefToUnsafeOffset(pkBytesRef.offset) @@ -278,23 +287,28 @@ class DownsampledTimeSeriesShard(rawDatasetRef: DatasetRef, }.getOrElse(throw new IllegalStateException(s"PartId $partID returned by lucene, but partKey not found")) } - private def chooseDownsampleResolution(chunkScanMethod: ChunkScanMethod): DatasetRef = { + private def chooseDownsampleResolution(chunkScanMethod: ChunkScanMethod): (Long, DatasetRef) = { chunkScanMethod match { - case AllChunkScan => downsampledDatasetRefs.last // since it is the highest resolution/ttl + case AllChunkScan => + // since it is the highest resolution/ttl + downsampleTtls.last.toMillis -> downsampledDatasetRefs.last case TimeRangeChunkScan(startTime, _) => - val ttlIndex = downsampleTtls.indexWhere(t => startTime > System.currentTimeMillis() - t.toMillis) - downsampledDatasetRefs(ttlIndex) + var ttlIndex = downsampleTtls.indexWhere(t => startTime > System.currentTimeMillis() - t.toMillis) + // -1 return value means query startTime is before the earliest retention. Just pick the highest resolution + if (ttlIndex == -1) ttlIndex = downsampleTtls.size - 1 + downsampleConfig.resolutions(ttlIndex).toMillis -> downsampledDatasetRefs(ttlIndex) case _ => ??? } } - private def makePagedPartition(part: RawPartData, firstSchemaId: Int): ReadablePartition = { + private def makePagedPartition(part: RawPartData, firstSchemaId: Int, + colIds: Seq[Types.ColumnId]): ReadablePartition = { val schemaId = RecordSchema.schemaID(part.partitionKey, UnsafeUtils.arayOffset) if (schemaId != firstSchemaId) { throw SchemaMismatch(schemas.schemaName(firstSchemaId), schemas.schemaName(schemaId)) } // FIXME It'd be nice to pass in the correct partId here instead of -1 - new PagedReadablePartition(schemas(schemaId), shardNum, -1, part) + new PagedReadablePartition(schemas(schemaId), shardNum, -1, part, colIds) } /** diff --git a/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesStore.scala b/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesStore.scala index 3fe1736375..0c546965b6 100644 --- a/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesStore.scala +++ b/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesStore.scala @@ -10,10 +10,10 @@ import monix.execution.{CancelableFuture, Scheduler} import monix.reactive.Observable import org.jctools.maps.NonBlockingHashMapLong -import filodb.core.{DatasetRef, Response} +import filodb.core.{DatasetRef, Response, Types} import filodb.core.memstore._ import filodb.core.metadata.Schemas -import filodb.core.query.ColumnFilter +import filodb.core.query.{ColumnFilter, QuerySession} import filodb.core.store._ import filodb.memory.format.{UnsafeUtils, ZeroCopyUTF8String} @@ -88,25 +88,28 @@ extends MemStore with StrictLogging { def lookupPartitions(ref: DatasetRef, partMethod: PartitionScanMethod, - chunkMethod: ChunkScanMethod): PartLookupResult = { + chunkMethod: ChunkScanMethod, + querySession: QuerySession): PartLookupResult = { val shard = datasets(ref).get(partMethod.shard) if (shard == UnsafeUtils.ZeroPointer) { throw new IllegalArgumentException(s"Shard $shard of dataset $ref is not assigned to " + s"this node. Was it was recently reassigned to another node? Prolonged occurrence indicates an issue.") } - shard.lookupPartitions(partMethod, chunkMethod) + shard.lookupPartitions(partMethod, chunkMethod, querySession) } def scanPartitions(ref: DatasetRef, - lookupRes: PartLookupResult): Observable[ReadablePartition] = { + lookupRes: PartLookupResult, + colIds: Seq[Types.ColumnId], + querySession: QuerySession): Observable[ReadablePartition] = { val shard = datasets(ref).get(lookupRes.shard) if (shard == UnsafeUtils.ZeroPointer) { throw new IllegalArgumentException(s"Shard $shard of dataset $ref is not assigned to " + s"this node. Was it was recently reassigned to another node? Prolonged occurrence indicates an issue.") } - shard.scanPartitions(lookupRes) + shard.scanPartitions(lookupRes, colIds, querySession) } def activeShards(dataset: DatasetRef): Seq[Int] = diff --git a/core/src/main/scala/filodb.core/memstore/DemandPagedChunkStore.scala b/core/src/main/scala/filodb.core/memstore/DemandPagedChunkStore.scala index dcdd3f75a1..9c268c77af 100644 --- a/core/src/main/scala/filodb.core/memstore/DemandPagedChunkStore.scala +++ b/core/src/main/scala/filodb.core/memstore/DemandPagedChunkStore.scala @@ -69,6 +69,7 @@ extends RawToPartitionMaker with StrictLogging { FiloSchedulers.assertThreadName(FiloSchedulers.PopulateChunksSched) // Find the right partition given the partition key tsShard.getPartition(rawPartition.partitionKey).map { tsPart => + logger.debug(s"Populating paged chunks for shard=${tsShard.shardNum} partId=${tsPart.partID}") tsShard.shardStats.partitionsPagedFromColStore.increment() tsShard.shardStats.numChunksPagedIn.increment(rawPartition.chunkSets.size) // One chunkset at a time, load them into offheap and populate the partition diff --git a/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala b/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala index ced379ce90..28d32780bf 100644 --- a/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala +++ b/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala @@ -1,5 +1,7 @@ package filodb.core.memstore +import java.util + import scala.collection.mutable.ArrayBuffer import scala.concurrent.ExecutionContext @@ -10,14 +12,13 @@ import monix.eval.Task import monix.execution.Scheduler import monix.reactive.{Observable, OverflowStrategy} -import filodb.core.DatasetRef +import filodb.core.{DatasetRef, Types} import filodb.core.binaryrecord2.RecordSchema import filodb.core.downsample.{DownsampleConfig, DownsamplePublisher} import filodb.core.metadata.Schemas +import filodb.core.query.QuerySession import filodb.core.store._ -import filodb.memory.BinaryRegionLarge import filodb.memory.MemFactory -import filodb.memory.format.UnsafeUtils /** * Extends TimeSeriesShard with on-demand paging functionality by populating in-memory partitions with chunks from @@ -50,6 +51,21 @@ TimeSeriesShard(ref, schemas, storeConfig, shardNum, bufferMemoryManager, rawSto .tag("shard", shardNum) .start() + val assumedResolution = 20000 // for now hard-code and assume 30ms as reporting interval + + private def capDataScannedPerShardCheck(lookup: PartLookupResult): Unit = { + lookup.firstSchemaId.foreach { schId => + lookup.chunkMethod match { + case TimeRangeChunkScan(st, end) => + val numMatches = lookup.partsInMemory.length + lookup.partIdsNotInMemory.length + schemas.ensureQueriedDataSizeWithinLimit(schId, numMatches, + storeConfig.flushInterval.toMillis, + assumedResolution, end - st, storeConfig.maxDataPerShardQuery) + case _ => + } + } + } + // NOTE: the current implementation is as follows // 1. Fetch partitions from memStore // 2. Determine, one at a time, what chunks are missing and could be fetched from disk @@ -57,7 +73,12 @@ TimeSeriesShard(ref, schemas, storeConfig, shardNum, bufferMemoryManager, rawSto // 4. upload to memory and return partition // Definitely room for improvement, such as fetching multiple partitions at once, more parallelism, etc. //scalastyle:off - override def scanPartitions(partLookupRes: PartLookupResult): Observable[ReadablePartition] = { + override def scanPartitions(partLookupRes: PartLookupResult, + colIds: Seq[Types.ColumnId], + querySession: QuerySession): Observable[ReadablePartition] = { + + capDataScannedPerShardCheck(partLookupRes) + // For now, always read every data column. // 1. We don't have a good way to update just some columns of a chunkset for ODP // 2. Timestamp column almost always needed @@ -113,6 +134,13 @@ TimeSeriesShard(ref, schemas, storeConfig, shardNum, bufferMemoryManager, rawSto } else { Observable.fromTask(odpPartTask(partIdsNotInMemory, partKeyBytesToPage, pagingMethods, partLookupRes.chunkMethod)).flatMap { odpParts => + assertThreadName(QuerySchedName) + logger.debug(s"Finished creating full ODP partitions ${odpParts.map(_.partID)}") + if(logger.underlying.isDebugEnabled) { + partKeyBytesToPage.zip(pagingMethods).foreach { case (pk, method) => + logger.debug(s"Paging in chunks for partId=${getPartition(pk).get.partID} chunkMethod=$method") + } + } if (partKeyBytesToPage.nonEmpty) { val span = startODPSpan() Observable.fromIterable(partKeyBytesToPage.zip(pagingMethods)) @@ -122,6 +150,7 @@ TimeSeriesShard(ref, schemas, storeConfig, shardNum, bufferMemoryManager, rawSto .asyncBoundary(strategy) // This is needed so future computations happen in a different thread .defaultIfEmpty(getPartition(partBytes).get) .headL + // headL since we are fetching a SinglePartition above } .doOnTerminate(ex => span.finish()) } else { @@ -139,13 +168,12 @@ TimeSeriesShard(ref, schemas, storeConfig, shardNum, bufferMemoryManager, rawSto // 3. Deal with partitions no longer in memory but still indexed in Lucene. // Basically we need to create TSPartitions for them in the ingest thread -- if there's enough memory private def odpPartTask(partIdsNotInMemory: Buffer[Int], partKeyBytesToPage: ArrayBuffer[Array[Byte]], - methods: ArrayBuffer[ChunkScanMethod], chunkMethod: ChunkScanMethod) = + pagingMethods: ArrayBuffer[ChunkScanMethod], chunkMethod: ChunkScanMethod) = if (partIdsNotInMemory.nonEmpty) { - createODPPartitionsTask(partIdsNotInMemory, { case (bytes, offset) => - val partKeyBytes = if (offset == UnsafeUtils.arayOffset) bytes - else BinaryRegionLarge.asNewByteArray(bytes, offset) - partKeyBytesToPage += partKeyBytes - methods += chunkMethod + createODPPartitionsTask(partIdsNotInMemory, { case (pId, pkBytes) => + partKeyBytesToPage += pkBytes + pagingMethods += chunkMethod + logger.debug(s"Finished creating part for full odp. Now need to page partId=$pId chunkMethod=$chunkMethod") shardStats.partitionsRestored.increment() }).executeOn(ingestSched).asyncBoundary // asyncBoundary above will cause subsequent map operations to run on designated scheduler for task or observable @@ -159,7 +187,7 @@ TimeSeriesShard(ref, schemas, storeConfig, shardNum, bufferMemoryManager, rawSto * to create TSPartitions for partIDs found in Lucene but not in in-memory data structures * It runs in ingestion thread so it can correctly verify which ones to actually create or not */ - private def createODPPartitionsTask(partIDs: Buffer[Int], callback: (Array[Byte], Int) => Unit): + private def createODPPartitionsTask(partIDs: Buffer[Int], callback: (Int, Array[Byte]) => Unit): Task[Seq[TimeSeriesPartition]] = Task { assertThreadName(IngestSchedName) require(partIDs.nonEmpty) @@ -182,7 +210,9 @@ TimeSeriesShard(ref, schemas, storeConfig, shardNum, bufferMemoryManager, rawSto } finally { partSetLock.unlockWrite(stamp) } - callback(partKeyBytesRef.bytes, unsafeKeyOffset) + val pkBytes = util.Arrays.copyOfRange(partKeyBytesRef.bytes, partKeyBytesRef.offset, + partKeyBytesRef.offset + partKeyBytesRef.length) + callback(part.partID, pkBytes) part } // create the partition and update data structures (but no need to add to Lucene!) diff --git a/core/src/main/scala/filodb.core/memstore/PagedReadablePartition.scala b/core/src/main/scala/filodb.core/memstore/PagedReadablePartition.scala index 6ce238e04b..82f7852e71 100644 --- a/core/src/main/scala/filodb.core/memstore/PagedReadablePartition.scala +++ b/core/src/main/scala/filodb.core/memstore/PagedReadablePartition.scala @@ -1,7 +1,10 @@ package filodb.core.memstore +import java.nio.ByteBuffer + import com.typesafe.scalalogging.StrictLogging +import filodb.core.Types import filodb.core.Types.ChunkID import filodb.core.metadata.Schema import filodb.core.store._ @@ -9,6 +12,7 @@ import filodb.memory.format.UnsafeUtils object PagedReadablePartition extends StrictLogging { val _log = logger + val emptyByteBuffer = ByteBuffer.allocate(0) } /** @@ -20,11 +24,24 @@ object PagedReadablePartition extends StrictLogging { * * Any ChunkScanMethod will return results from all available chunks. This optimization * is done since that check would already done and does not need to be repeated. + * + * @param colIds the colIds that need to be retained. Leave empty if all are needed. */ class PagedReadablePartition(override val schema: Schema, override val shard: Int, override val partID: Int, - partData: RawPartData) extends ReadablePartition { + partData: RawPartData, + colIds: Seq[Types.ColumnId] = Seq.empty) extends ReadablePartition { + + import PagedReadablePartition._ + val notNeededColIds = if (colIds.nonEmpty) schema.dataInfos.indices.toSet -- colIds.toSet + else Set.empty + partData.chunkSets.foreach { vectors => + // release vectors that are not needed so they can be GCed quickly before scans + // finish. This is a temporary workaround since we dont have ability to fetch + // specific columns from Cassandra + notNeededColIds.foreach(i => vectors.vectors(i) = emptyByteBuffer) + } override def numChunks: Int = partData.chunkSets.length diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala index 1b1af9b861..c9166de1da 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala @@ -10,10 +10,10 @@ import monix.execution.{CancelableFuture, Scheduler} import monix.reactive.Observable import org.jctools.maps.NonBlockingHashMapLong -import filodb.core.{DatasetRef, Response} +import filodb.core.{DatasetRef, Response, Types} import filodb.core.downsample.{DownsampleConfig, DownsamplePublisher} import filodb.core.metadata.Schemas -import filodb.core.query.ColumnFilter +import filodb.core.query.{ColumnFilter, QuerySession} import filodb.core.store._ import filodb.memory.MemFactory import filodb.memory.NativeMemoryManager @@ -199,26 +199,29 @@ extends MemStore with StrictLogging { chunkMethod: ChunkScanMethod = AllChunkScan): Observable[RawPartData] = Observable.empty def scanPartitions(ref: DatasetRef, - iter: PartLookupResult): Observable[ReadablePartition] = { + iter: PartLookupResult, + colIds: Seq[Types.ColumnId], + querySession: QuerySession): Observable[ReadablePartition] = { val shard = datasets(ref).get(iter.shard) if (shard == UnsafeUtils.ZeroPointer) { throw new IllegalArgumentException(s"Shard ${iter.shard} of dataset $ref is not assigned to " + s"this node. Was it was recently reassigned to another node? Prolonged occurrence indicates an issue.") } - shard.scanPartitions(iter) + shard.scanPartitions(iter, colIds, querySession) } def lookupPartitions(ref: DatasetRef, partMethod: PartitionScanMethod, - chunkMethod: ChunkScanMethod): PartLookupResult = { + chunkMethod: ChunkScanMethod, + querySession: QuerySession): PartLookupResult = { val shard = datasets(ref).get(partMethod.shard) if (shard == UnsafeUtils.ZeroPointer) { throw new IllegalArgumentException(s"Shard ${partMethod.shard} of dataset $ref is not assigned to " + s"this node. Was it was recently reassigned to another node? Prolonged occurrence indicates an issue.") } - shard.lookupPartitions(partMethod, chunkMethod) + shard.lookupPartitions(partMethod, chunkMethod, querySession) } def numRowsIngested(dataset: DatasetRef, shard: Int): Long = diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala index 3eb65513ce..dd8e5ee884 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala @@ -387,8 +387,10 @@ extends ChunkMap(memFactory, initMapSize) with ReadablePartition { //def dataChunkPointer(id: ChunkID, columnID: Int): BinaryVector.BinaryVectorPtr = infoGet(id).vectorPtr(columnID) final def removeChunksAt(id: ChunkID): Unit = { - chunkmapWithExclusive(chunkmapDoRemove(id)) - shardStats.chunkIdsEvicted.increment() + // Remove all chunks at and lower than the given chunk. Doing so prevents a hole from + // emerging in the middle which ODP can't easily cope with. + val amt = chunkmapWithExclusive(chunkmapDoRemoveFloor(id)) + shardStats.chunkIdsEvicted.increment(amt) } final def hasChunksAt(id: ChunkID): Boolean = chunkmapContains(id) diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala index 4a947b1992..aad7a5fe2d 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala @@ -1,5 +1,6 @@ package filodb.core.memstore +import java.util.concurrent.TimeUnit import java.util.concurrent.locks.StampedLock import scala.collection.mutable.ArrayBuffer @@ -26,7 +27,7 @@ import filodb.core.{ErrorResponse, _} import filodb.core.binaryrecord2._ import filodb.core.downsample.{DownsampleConfig, DownsamplePublisher, ShardDownsampler} import filodb.core.metadata.{Schema, Schemas} -import filodb.core.query.ColumnFilter +import filodb.core.query.{ColumnFilter, QuerySession} import filodb.core.store._ import filodb.memory._ import filodb.memory.format.{UnsafeUtils, ZeroCopyUTF8String} @@ -312,6 +313,15 @@ class TimeSeriesShard(val ref: DatasetRef, storeConfig.numPagesPerBlock) private val blockFactoryPool = new BlockMemFactoryPool(blockStore, maxMetaSize, shardTags) + /** + * Lock that protects chunks from being reclaimed from Memstore. + * This is needed to prevent races between ODP queries and reclaims. + */ + private[memstore] final val reclaimLock = blockStore.reclaimLock + + // Requires blockStore. + startHeadroomTask(ingestSched) + // 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, @@ -1405,52 +1415,58 @@ class TimeSeriesShard(val ref: DatasetRef, * Also returns detailed information about what is in memory and not, and does schema discovery. */ def lookupPartitions(partMethod: PartitionScanMethod, - chunkMethod: ChunkScanMethod): PartLookupResult = partMethod match { - case SinglePartitionScan(partition, _) => - val partIds = debox.Buffer.empty[Int] - getPartition(partition).foreach(p => partIds += p.partID) - PartLookupResult(shardNum, chunkMethod, partIds, Some(RecordSchema.schemaID(partition))) - case MultiPartitionScan(partKeys, _) => - val partIds = debox.Buffer.empty[Int] - partKeys.flatMap(getPartition).foreach(p => partIds += p.partID) - PartLookupResult(shardNum, chunkMethod, partIds, partKeys.headOption.map(RecordSchema.schemaID)) - case FilteredPartitionScan(split, filters) => - // No matter if there are filters or not, need to run things through Lucene so we can discover potential - // TSPartitions to read back from disk - val matches = partKeyIndex.partIdsFromFilters(filters, chunkMethod.startTime, chunkMethod.endTime) - shardStats.queryTimeRangeMins.record((chunkMethod.endTime - chunkMethod.startTime) / 60000 ) - - Kamon.currentSpan().tag(s"num-partitions-from-index-$shardNum", matches.length) - if (matches.length > storeConfig.maxQueryMatches) - throw new IllegalArgumentException(s"Seeing ${matches.length} matching time series per shard. Try " + - s"to narrow your query by adding more filters so there is less than ${storeConfig.maxQueryMatches} matches " + - s"or request that number of shards for the metric be increased") - - // first find out which partitions are being queried for data not in memory - val firstPartId = if (matches.isEmpty) None else Some(matches(0)) - val _schema = firstPartId.map(schemaIDFromPartID) - val it1 = InMemPartitionIterator2(matches) - val partIdsToPage = it1.filter(_.earliestTime > chunkMethod.startTime).map(_.partID) - val partIdsNotInMem = it1.skippedPartIDs - Kamon.currentSpan().tag(s"num-partitions-not-in-memory-$shardNum", partIdsNotInMem.length) - val startTimes = if (partIdsToPage.nonEmpty) { - val st = partKeyIndex.startTimeFromPartIds(partIdsToPage) - logger.debug(s"Some partitions have earliestTime > queryStartTime(${chunkMethod.startTime}); " + - s"startTime lookup for query in dataset=$ref shard=$shardNum " + - s"resulted in startTimes=$st") - st - } - else { - logger.debug(s"StartTime lookup was not needed. All partition's data for query in dataset=$ref " + - s"shard=$shardNum are in memory") - debox.Map.empty[Int, Long] - } - // now provide an iterator that additionally supplies the startTimes for - // those partitions that may need to be paged - PartLookupResult(shardNum, chunkMethod, matches, _schema, startTimes, partIdsNotInMem) + chunkMethod: ChunkScanMethod, + querySession: QuerySession): PartLookupResult = { + querySession.lock = Some(reclaimLock) + reclaimLock.lock() + // any exceptions thrown here should be caught by a wrapped Task. + // At the end, MultiSchemaPartitionsExec.execute releases the lock when the task is complete + partMethod match { + case SinglePartitionScan(partition, _) => + val partIds = debox.Buffer.empty[Int] + getPartition(partition).foreach(p => partIds += p.partID) + PartLookupResult(shardNum, chunkMethod, partIds, Some(RecordSchema.schemaID(partition))) + case MultiPartitionScan(partKeys, _) => + val partIds = debox.Buffer.empty[Int] + partKeys.flatMap(getPartition).foreach(p => partIds += p.partID) + PartLookupResult(shardNum, chunkMethod, partIds, partKeys.headOption.map(RecordSchema.schemaID)) + case FilteredPartitionScan(_, filters) => + // No matter if there are filters or not, need to run things through Lucene so we can discover potential + // TSPartitions to read back from disk + val matches = partKeyIndex.partIdsFromFilters(filters, chunkMethod.startTime, chunkMethod.endTime) + shardStats.queryTimeRangeMins.record((chunkMethod.endTime - chunkMethod.startTime) / 60000 ) + + Kamon.currentSpan().tag(s"num-partitions-from-index-$shardNum", matches.length) + + // first find out which partitions are being queried for data not in memory + val firstPartId = if (matches.isEmpty) None else Some(matches(0)) + val _schema = firstPartId.map(schemaIDFromPartID) + val it1 = InMemPartitionIterator2(matches) + val partIdsToPage = it1.filter(_.earliestTime > chunkMethod.startTime).map(_.partID) + val partIdsNotInMem = it1.skippedPartIDs + Kamon.currentSpan().tag(s"num-partitions-not-in-memory-$shardNum", partIdsNotInMem.length) + val startTimes = if (partIdsToPage.nonEmpty) { + val st = partKeyIndex.startTimeFromPartIds(partIdsToPage) + logger.debug(s"Some partitions have earliestTime > queryStartTime(${chunkMethod.startTime}); " + + s"startTime lookup for query in dataset=$ref shard=$shardNum " + + s"resulted in startTimes=$st") + st + } + else { + logger.debug(s"StartTime lookup was not needed. All partition's data for query in dataset=$ref " + + s"shard=$shardNum are in memory") + debox.Map.empty[Int, Long] + } + // now provide an iterator that additionally supplies the startTimes for + // those partitions that may need to be paged + PartLookupResult(shardNum, chunkMethod, matches, _schema, startTimes, partIdsNotInMem) + } } - def scanPartitions(iterResult: PartLookupResult): Observable[ReadablePartition] = { + def scanPartitions(iterResult: PartLookupResult, + colIds: Seq[Types.ColumnId], + querySession: QuerySession): Observable[ReadablePartition] = { + val partIter = new InMemPartitionIterator2(iterResult.partsInMemory) Observable.fromIterator(partIter.map { p => shardStats.partitionsQueried.increment() @@ -1458,6 +1474,12 @@ class TimeSeriesShard(val ref: DatasetRef, }) } + private def startHeadroomTask(sched: Scheduler): Unit = { + sched.scheduleWithFixedDelay(1, 1, TimeUnit.MINUTES, new Runnable { + def run() = blockStore.ensureHeadroom(storeConfig.ensureHeadroomPercent) + }) + } + /** * Please use this for testing only - reclaims ALL used offheap blocks. Maybe you are trying to test * on demand paging. @@ -1495,5 +1517,6 @@ class TimeSeriesShard(val ref: DatasetRef, method to ensure that no threads are accessing the memory before it's freed. blockStore.releaseBlocks() */ + ingestSched.shutdown() } } diff --git a/core/src/main/scala/filodb.core/metadata/Schemas.scala b/core/src/main/scala/filodb.core/metadata/Schemas.scala index 199e63a2df..cfb245bb1a 100644 --- a/core/src/main/scala/filodb.core/metadata/Schemas.scala +++ b/core/src/main/scala/filodb.core/metadata/Schemas.scala @@ -1,6 +1,7 @@ package filodb.core.metadata import com.typesafe.config.Config +import com.typesafe.scalalogging.StrictLogging import net.ceedubs.ficus.Ficus._ import org.scalactic._ @@ -8,6 +9,7 @@ import filodb.core.GlobalConfig import filodb.core.Types._ import filodb.core.binaryrecord2._ import filodb.core.downsample.{ChunkDownsampler, DownsamplePeriodMarker} +import filodb.core.metadata.Column.ColumnType import filodb.core.query.ColumnInfo import filodb.core.store.ChunkSetInfo import filodb.memory.BinaryRegion @@ -260,6 +262,57 @@ final case class Schemas(part: PartitionSchema, schemas.values.foreach { s => _schemas(s.schemaHash) = s } + /** + * This is purely a SWAG to be used for query size estimation. Do not rely for other use cases. + */ + private val bytesPerSampleSwag: Map[Int, Double] = { + + val allSchemas = schemas.values ++ schemas.values.flatMap(_.downsample) + allSchemas.map { s => + val est = s.data.columns.map(_.columnType).map { + case ColumnType.LongColumn => 1 + case ColumnType.IntColumn => 1 + case ColumnType.TimestampColumn => 0.5 + case ColumnType.HistogramColumn => 20 + case ColumnType.DoubleColumn => 2 + case _ => 0 // TODO allow without sizing for now + }.sum + s.schemaHash -> est + }.toMap + } + + private def bytesPerSampleSwagString = bytesPerSampleSwag.map { case e => + schemaName(e._1) + ": " + e._2 + } + + Schemas._log.info(s"bytesPerSampleSwag: $bytesPerSampleSwagString") + + /** + * Note this approach below assumes the following for quick size estimation. The sizing is more + * a swag than reality: + * (a) every matched time series ingests at all query times. Looking up start/end times and more + * precise size estimation is costly + * (b) it also assigns bytes per sample based on schema which is much of a swag. In reality, it would depend on + * number of histogram buckets, samples per chunk etc. + */ + def ensureQueriedDataSizeWithinLimit(schemaId: Int, + numTsPartitions: Int, + chunkDurationMillis: Long, + resolutionMs: Long, + queryDurationMs: Long, + dataSizeLimit: Long): Unit = { + val numSamplesPerChunk = chunkDurationMillis / resolutionMs + // find number of chunks to be scanned. Ceil division needed here + val numChunksPerTs = (queryDurationMs + chunkDurationMillis - 1) / chunkDurationMillis + val bytesPerSample = bytesPerSampleSwag(schemaId) + val estDataSize = bytesPerSample * numTsPartitions * numSamplesPerChunk * numChunksPerTs + require(estDataSize < dataSizeLimit, + s"With match of $numTsPartitions time series, estimate of $estDataSize bytes exceeds limit of " + + s"$dataSizeLimit bytes queried per shard for ${_schemas(schemaId).name} schema. Try one or more of these: " + + s"(a) narrow your query filters to reduce to fewer than the current $numTsPartitions matches " + + s"(b) reduce query time range, currently at ${queryDurationMs / 1000 / 60 } minutes") + } + /** * Returns the Schema for a given schemaID, or UnknownSchema if not found */ @@ -289,11 +342,13 @@ final case class Schemas(part: PartitionSchema, * } * }}} */ -object Schemas { +object Schemas extends StrictLogging { import java.nio.charset.StandardCharsets.UTF_8 import Accumulation._ import Dataset._ + val _log = logger + val rowKeyIDs = Seq(0) // First or timestamp column is always the row keys val UnknownSchema = UnsafeUtils.ZeroPointer.asInstanceOf[Schema] diff --git a/core/src/main/scala/filodb.core/query/KeyFilter.scala b/core/src/main/scala/filodb.core/query/KeyFilter.scala index 4c07934cff..b10a59533f 100644 --- a/core/src/main/scala/filodb.core/query/KeyFilter.scala +++ b/core/src/main/scala/filodb.core/query/KeyFilter.scala @@ -7,17 +7,20 @@ import filodb.memory.format.{SingleValueRowReader, UTF8Wrapper, ZeroCopyUTF8Stri sealed trait Filter { def filterFunc: Any => Boolean + def operatorString: String def valuesStrings : Set[Any] } object Filter { final case class Equals(value: Any) extends Filter { val filterFunc: Any => Boolean = (item: Any) => value.equals(item) + val operatorString: String = "=" def valuesStrings: Set[Any] = Set(value) } final case class In(values: Set[Any]) extends Filter { val filterFunc: (Any) => Boolean = (item: Any) => values.contains(item) + val operatorString: String = "in" def valuesStrings: Set[Any] = values } @@ -25,23 +28,27 @@ object Filter { private val leftFunc = left.filterFunc private val rightFunc = right.filterFunc val filterFunc: (Any) => Boolean = (item: Any) => leftFunc(item) && rightFunc(item) + val operatorString: String = "&&" def valuesStrings: Set[Any] = left.valuesStrings.union(right.valuesStrings) } final case class NotEquals(value: Any) extends Filter { val filterFunc: (Any) => Boolean = (item: Any) => !value.equals(item) + val operatorString: String = "!=" def valuesStrings: Set[Any] = Set(value) } final case class EqualsRegex(value: Any) extends Filter { val pattern = Pattern.compile(value.toString, Pattern.DOTALL) val filterFunc: (Any) => Boolean = (item: Any) => pattern.matcher(item.toString).matches() + val operatorString: String = "=~" def valuesStrings: Set[Any] = Set(value) } final case class NotEqualsRegex(value: Any) extends Filter { val pattern = Pattern.compile(value.toString, Pattern.DOTALL) val filterFunc: (Any) => Boolean = (item: Any) => !pattern.matcher(item.toString).matches() + val operatorString: String = "!~" def valuesStrings: Set[Any] = Set(value) } diff --git a/query/src/main/scala/filodb/query/QueryConfig.scala b/core/src/main/scala/filodb.core/query/QueryConfig.scala similarity index 97% rename from query/src/main/scala/filodb/query/QueryConfig.scala rename to core/src/main/scala/filodb.core/query/QueryConfig.scala index d54c006ebe..08f14e29c4 100644 --- a/query/src/main/scala/filodb/query/QueryConfig.scala +++ b/core/src/main/scala/filodb.core/query/QueryConfig.scala @@ -1,4 +1,4 @@ -package filodb.query +package filodb.core.query import scala.concurrent.duration.FiniteDuration diff --git a/core/src/main/scala/filodb.core/query/QueryContext.scala b/core/src/main/scala/filodb.core/query/QueryContext.scala index e1b0df05a3..e0a0192e4a 100644 --- a/core/src/main/scala/filodb.core/query/QueryContext.scala +++ b/core/src/main/scala/filodb.core/query/QueryContext.scala @@ -1,6 +1,7 @@ package filodb.core.query import java.util.UUID +import java.util.concurrent.locks.Lock import com.typesafe.config.Config @@ -9,7 +10,7 @@ import filodb.core.{SpreadChange, SpreadProvider} trait TsdbQueryParams /** - * This class provides PromQl query paramaters + * This class provides PromQl query parameters * Config has routing parameters */ case class PromQlQueryParams(config: Config, promQl: String, startSecs: Long, stepSecs: Long, endSecs: Long, @@ -57,3 +58,19 @@ object QueryContext { simpleMapSpreadFunc(shardKeyNames.asScala, spreadAssignment, defaultSpread) } } + +/** + * Placeholder for query related information. Typically passed along query execution path. + */ +case class QuerySession(qContext: QueryContext, + queryConfig: QueryConfig, + var lock: Option[Lock] = None) { + def close(): Unit = { + lock.foreach(_.unlock()) + lock = None + } +} + +object QuerySession { + def forTestingOnly: QuerySession = QuerySession(QueryContext(), EmptyQueryConfig) +} \ No newline at end of file diff --git a/core/src/main/scala/filodb.core/store/ChunkSource.scala b/core/src/main/scala/filodb.core/store/ChunkSource.scala index e04284dc73..35efdcc104 100644 --- a/core/src/main/scala/filodb.core/store/ChunkSource.scala +++ b/core/src/main/scala/filodb.core/store/ChunkSource.scala @@ -78,16 +78,19 @@ trait ChunkSource extends RawChunkSource with StrictLogging { def scanPartitions(ref: DatasetRef, columnIDs: Seq[Types.ColumnId], partMethod: PartitionScanMethod, - chunkMethod: ChunkScanMethod = AllChunkScan): Observable[ReadablePartition] = { + chunkMethod: ChunkScanMethod = AllChunkScan, + querySession: QuerySession): Observable[ReadablePartition] = { logger.debug(s"scanPartitions dataset=$ref shard=${partMethod.shard} " + s"partMethod=$partMethod chunkMethod=$chunkMethod") - scanPartitions(ref, lookupPartitions(ref, partMethod, chunkMethod)) + scanPartitions(ref, lookupPartitions(ref, partMethod, chunkMethod, querySession), columnIDs, querySession) } // Internal API that needs to actually be implemented def scanPartitions(ref: DatasetRef, - lookupRes: PartLookupResult): Observable[ReadablePartition] + lookupRes: PartLookupResult, + colIds: Seq[Types.ColumnId], + querySession: QuerySession): Observable[ReadablePartition] // internal method to find # of groups in a dataset def groupsInDataset(ref: DatasetRef): Int @@ -108,7 +111,8 @@ trait ChunkSource extends RawChunkSource with StrictLogging { */ def lookupPartitions(ref: DatasetRef, partMethod: PartitionScanMethod, - chunkMethod: ChunkScanMethod): PartLookupResult + chunkMethod: ChunkScanMethod, + querySession: QuerySession): PartLookupResult /** * Returns a stream of RangeVectors's. Good for per-partition (or time series) processing. @@ -124,18 +128,19 @@ trait ChunkSource extends RawChunkSource with StrictLogging { lookupRes: PartLookupResult, columnIDs: Seq[Types.ColumnId], schema: Schema, - filterSchemas: Boolean): Observable[RangeVector] = { + filterSchemas: Boolean, + querySession: QuerySession): Observable[RangeVector] = { val ids = columnIDs.toArray val partCols = schema.infosFromIDs(schema.partition.columns.map(_.id)) val numGroups = groupsInDataset(ref) val filteredParts = if (filterSchemas) { - scanPartitions(ref, lookupRes) + scanPartitions(ref, lookupRes, columnIDs, querySession) .filter { p => p.schema.schemaHash == schema.schemaHash && p.hasChunks(lookupRes.chunkMethod) } } else { lookupRes.firstSchemaId match { case Some(reqSchemaId) => - scanPartitions(ref, lookupRes).filter { p => + scanPartitions(ref, lookupRes, columnIDs, querySession).filter { p => if (p.schema.schemaHash != reqSchemaId) throw SchemaMismatch(Schemas.global.schemaName(reqSchemaId), p.schema.name) p.hasChunks(lookupRes.chunkMethod) diff --git a/core/src/main/scala/filodb.core/store/IngestionConfig.scala b/core/src/main/scala/filodb.core/store/IngestionConfig.scala index 6130c1a3c8..6deedbd3d2 100644 --- a/core/src/main/scala/filodb.core/store/IngestionConfig.scala +++ b/core/src/main/scala/filodb.core/store/IngestionConfig.scala @@ -33,9 +33,11 @@ final case class StoreConfig(flushInterval: FiniteDuration, demandPagingParallelism: Int, demandPagingEnabled: Boolean, evictedPkBfCapacity: Int, + // Amount of free blocks to periodically reclaim, as a percent of total number of blocks + ensureHeadroomPercent: Double, // filters on ingested records to log in detail traceFilters: Map[String, String], - maxQueryMatches: Int) { + maxDataPerShardQuery: Long) { import collection.JavaConverters._ def toConfig: Config = ConfigFactory.parseMap(Map("flush-interval" -> (flushInterval.toSeconds + "s"), @@ -57,8 +59,9 @@ final case class StoreConfig(flushInterval: FiniteDuration, "multi-partition-odp" -> multiPartitionODP, "demand-paging-parallelism" -> demandPagingParallelism, "demand-paging-enabled" -> demandPagingEnabled, - "max-query-matches" -> maxQueryMatches, - "evicted-pk-bloom-filter-capacity" -> evictedPkBfCapacity).asJava) + "max-data-per-shard-query" -> maxDataPerShardQuery, + "evicted-pk-bloom-filter-capacity" -> evictedPkBfCapacity, + "ensure-headroom-percent" -> ensureHeadroomPercent).asJava) } final case class AssignShardConfig(address: String, shardList: Seq[Int]) @@ -67,17 +70,23 @@ final case class UnassignShardConfig(shardList: Seq[Int]) object StoreConfig { // NOTE: there are no defaults for flush interval and shard memory, those should be explicitly calculated + // default max-data-per-shard-query was calculated as follows: + // 750k TsPartitions * 48 chunksets queried * 2kb per chunkset / 256 shards = 280MB + + // The num-block-pages setting when multiplied by the page size (4KB) defines the + // BlockManager block size. When num-block-pages is 100, the effective block size is 400KB. + val defaults = ConfigFactory.parseString(""" |disk-time-to-live = 3 days |demand-paged-chunk-retention-period = 72 hours |max-chunks-size = 400 - |max-query-matches = 250000 + |max-data-per-shard-query = 300 MB |max-blob-buffer-size = 15000 |ingestion-buffer-mem-size = 10M |max-buffer-pool-size = 10000 |num-partitions-to-evict = 1000 |groups-per-shard = 60 - |num-block-pages = 1000 + |num-block-pages = 100 |failure-retries = 3 |retry-delay = 15 seconds |part-index-flush-max-delay = 60 seconds @@ -86,6 +95,7 @@ object StoreConfig { |demand-paging-parallelism = 10 |demand-paging-enabled = true |evicted-pk-bloom-filter-capacity = 5000000 + |ensure-headroom-percent = 5.0 |trace-filters = {} |""".stripMargin) /** Pass in the config inside the store {} */ @@ -116,8 +126,9 @@ object StoreConfig { config.getInt("demand-paging-parallelism"), config.getBoolean("demand-paging-enabled"), config.getInt("evicted-pk-bloom-filter-capacity"), + config.getDouble("ensure-headroom-percent"), config.as[Map[String, String]]("trace-filters"), - config.getInt("max-query-matches")) + config.getMemorySize("max-data-per-shard-query").toBytes) } } diff --git a/core/src/main/scala/filodb.core/store/package.scala b/core/src/main/scala/filodb.core/store/package.scala index 666e7f6d40..a7ab5cef7e 100644 --- a/core/src/main/scala/filodb.core/store/package.scala +++ b/core/src/main/scala/filodb.core/store/package.scala @@ -6,6 +6,7 @@ import net.jpountz.lz4.{LZ4Compressor, LZ4Factory, LZ4FastDecompressor} import filodb.core.Types._ import filodb.core.metadata.Dataset +import filodb.core.query.QuerySession import filodb.memory.format.{RowReader, UnsafeUtils} package object store { @@ -138,6 +139,8 @@ package object store { * Convenience method to scan/iterate over all rows of given selection of source data. You must iterate * through all the elements. * + * Used for testing only. + * * @param dataset the Dataset to read from * @param columnIDs the set of column IDs to read back. Order determines the order of columns read back * in each row. These are the IDs from the Column instances. @@ -145,8 +148,9 @@ package object store { def scanRows(dataset: Dataset, columnIDs: Seq[ColumnId], partMethod: PartitionScanMethod, - chunkMethod: ChunkScanMethod = AllChunkScan): Iterator[RowReader] = - source.scanPartitions(dataset.ref, columnIDs, partMethod, chunkMethod) + chunkMethod: ChunkScanMethod = AllChunkScan, + querySession: QuerySession = QuerySession.forTestingOnly): Iterator[RowReader] = + source.scanPartitions(dataset.ref, columnIDs, partMethod, chunkMethod, querySession) .toIterator() .flatMap(_.timeRangeRows(chunkMethod, columnIDs.toArray)) } diff --git a/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala b/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala index 46f70c2153..7252773c99 100644 --- a/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala +++ b/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala @@ -13,7 +13,7 @@ import org.scalatest.time.{Millis, Seconds, Span} import filodb.core._ import filodb.core.binaryrecord2.RecordBuilder import filodb.core.metadata.Schemas -import filodb.core.query.{ColumnFilter, Filter} +import filodb.core.query.{ColumnFilter, Filter, QuerySession} import filodb.core.store._ import filodb.memory.format.{UnsafeUtils, ZeroCopyUTF8String} import filodb.memory.format.vectors.LongHistogram @@ -46,6 +46,7 @@ class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter w } } + // Look mama! Real-time time series ingestion and querying across multiple partitions! it("should ingest into multiple series and be able to query across all partitions in real time") { memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf) @@ -345,7 +346,8 @@ class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter w val filter = ColumnFilter("n", Filter.Equals("2".utf8)) val range = TimeRangeChunkScan(105000L, 2000000L) - val res = memStore.lookupPartitions(dataset2.ref, FilteredPartitionScan(split, Seq(filter)), range) + val res = memStore.lookupPartitions(dataset2.ref, FilteredPartitionScan(split, Seq(filter)), range, + QuerySession.forTestingOnly) res.firstSchemaId shouldEqual Some(schema2.schemaHash) res.partsInMemory.length shouldEqual 2 // two partitions should match res.shard shouldEqual 0 @@ -451,7 +453,8 @@ class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter w memStore.getShardE(dataset1.ref, 0).partKeyIndex.indexNumEntries shouldEqual 22 val split = memStore.getScanSplits(dataset1.ref, 1).head - val parts = memStore.scanPartitions(dataset1.ref, Seq(0, 1), FilteredPartitionScan(split)) + val parts = memStore.scanPartitions(dataset1.ref, Seq(0, 1), FilteredPartitionScan(split), + querySession = QuerySession.forTestingOnly) .toListL.runAsync .futureValue .asInstanceOf[Seq[TimeSeriesPartition]] @@ -487,7 +490,8 @@ class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter w // one more part val split = memStore.getScanSplits(dataset1.ref, 1).head val filter = ColumnFilter("series", Filter.Equals("Series 0".utf8)) - val parts = memStore.scanPartitions(dataset1.ref, Seq(0, 1), FilteredPartitionScan(split, Seq(filter))) + val parts = memStore.scanPartitions(dataset1.ref, Seq(0, 1), FilteredPartitionScan(split, Seq(filter)), + querySession = QuerySession.forTestingOnly) .toListL.runAsync .futureValue .asInstanceOf[Seq[TimeSeriesPartition]] @@ -564,7 +568,8 @@ class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter w memStore.refreshIndexForTesting(dataset1.ref) // Check partitions are now 0 to 20, 21/22 did not get added val split = memStore.getScanSplits(dataset1.ref, 1).head - val parts = memStore.scanPartitions(dataset1.ref, Seq(0, 1), FilteredPartitionScan(split)) + val parts = memStore.scanPartitions(dataset1.ref, Seq(0, 1), FilteredPartitionScan(split), + querySession = QuerySession.forTestingOnly) .toListL.runAsync .futureValue .asInstanceOf[Seq[TimeSeriesPartition]] diff --git a/core/src/test/scala/filodb.core/store/ColumnStoreSpec.scala b/core/src/test/scala/filodb.core/store/ColumnStoreSpec.scala index e896aa6f5a..81133c5728 100644 --- a/core/src/test/scala/filodb.core/store/ColumnStoreSpec.scala +++ b/core/src/test/scala/filodb.core/store/ColumnStoreSpec.scala @@ -12,7 +12,7 @@ import org.scalatest.time.{Millis, Seconds, Span} import filodb.core._ import filodb.core.memstore.{FixedMaxPartitionsEvictionPolicy, TimeSeriesMemStore} import filodb.core.metadata.Schemas -import filodb.core.query.{ColumnFilter, Filter} +import filodb.core.query.{ColumnFilter, Filter, QuerySession} // TODO: figure out what to do with this.. most of the tests are really irrelevant trait ColumnStoreSpec extends FlatSpec with Matchers @@ -151,9 +151,11 @@ with BeforeAndAfter with BeforeAndAfterAll with ScalaFutures { val filter = ColumnFilter("MonthYear", Filter.Equals(197902)) val method = FilteredPartitionScan(paramSet.head, Seq(filter)) - val lookupRes = memStore.lookupPartitions(dataset2.ref, method, AllChunkScan) + val lookupRes = memStore.lookupPartitions(dataset2.ref, method, AllChunkScan, + querySession = QuerySession.forTestingOnly) val rangeVectorObs = memStore.rangeVectors(dataset2.ref, lookupRes, schema2.colIDs("NumArticles").get, - schema2, false) + schema2, false, + QuerySession.forTestingOnly) val rangeVectors = rangeVectorObs.toListL.runAsync.futureValue rangeVectors should have length (1) diff --git a/jmh/src/main/scala/filodb.jmh/HistogramQueryBenchmark.scala b/jmh/src/main/scala/filodb.jmh/HistogramQueryBenchmark.scala index 3ac8a4eace..85dd75971a 100644 --- a/jmh/src/main/scala/filodb.jmh/HistogramQueryBenchmark.scala +++ b/jmh/src/main/scala/filodb.jmh/HistogramQueryBenchmark.scala @@ -19,12 +19,11 @@ 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.QueryContext +import filodb.core.query.{QueryConfig, QueryContext, QuerySession} import filodb.core.store._ import filodb.memory.MemFactory import filodb.memory.format.SeqRowReader import filodb.prometheus.parse.Parser -import filodb.query.QueryConfig //scalastyle:off regex /** @@ -113,7 +112,8 @@ class HistogramQueryBenchmark { @OperationsPerInvocation(500) def histSchemaQuantileQuery(): Long = { val f = Observable.fromIterable(0 until numQueries).mapAsync(1) { n => - hExecPlan.execute(memStore, queryConfig)(querySched) + val querySession = QuerySession(QueryContext(), queryConfig) + hExecPlan.execute(memStore, querySession)(querySched) }.executeOn(querySched) .countL.runAsync Await.result(f, 60.seconds) @@ -125,7 +125,8 @@ class HistogramQueryBenchmark { @OperationsPerInvocation(500) def promSchemaQuantileQuery(): Long = { val f = Observable.fromIterable(0 until numQueries).mapAsync(1) { n => - pExecPlan.execute(memStore, queryConfig)(querySched) + val querySession = QuerySession(QueryContext(), queryConfig) + pExecPlan.execute(memStore, querySession)(querySched) }.executeOn(querySched) .countL.runAsync Await.result(f, 60.seconds) diff --git a/jmh/src/main/scala/filodb.jmh/PartKeyIndexBenchmark.scala b/jmh/src/main/scala/filodb.jmh/PartKeyIndexBenchmark.scala index 2f7ce8a750..8067b8f7f7 100644 --- a/jmh/src/main/scala/filodb.jmh/PartKeyIndexBenchmark.scala +++ b/jmh/src/main/scala/filodb.jmh/PartKeyIndexBenchmark.scala @@ -12,7 +12,7 @@ import scalaxy.loops._ import filodb.core.DatasetRef import filodb.core.binaryrecord2.RecordBuilder import filodb.core.memstore.PartKeyLuceneIndex -import filodb.core.metadata.Schemas.promCounter +import filodb.core.metadata.Schemas.untyped import filodb.core.query.{ColumnFilter, Filter} import filodb.memory.{BinaryRegionConsumer, MemFactory} import filodb.timeseries.TestTimeseriesProducer @@ -23,17 +23,25 @@ class PartKeyIndexBenchmark { org.slf4j.LoggerFactory.getLogger("filodb").asInstanceOf[Logger].setLevel(Level.ERROR) val ref = DatasetRef("prometheus") - val partKeyIndex = new PartKeyLuceneIndex(ref, promCounter.partition, 0, 1.hour) + val partKeyIndex = new PartKeyLuceneIndex(ref, untyped.partition, 0, 1.hour) val numSeries = 1000000 - val partKeyData = TestTimeseriesProducer.timeSeriesData(0, numSeries) take numSeries + val ingestBuilder = new RecordBuilder(MemFactory.onHeapFactory) + val untypedData = TestTimeseriesProducer.timeSeriesData(0, numSeries) take numSeries + untypedData.foreach(_.addToBuilder(ingestBuilder)) + val partKeyBuilder = new RecordBuilder(MemFactory.onHeapFactory) - partKeyData.foreach(_.addToBuilder(partKeyBuilder)) + + val converter = new BinaryRegionConsumer { + def onNext(base: Any, offset: Long): Unit = untyped.comparator.buildPartKeyFromIngest(base, offset, partKeyBuilder) + } + // Build part keys from the ingestion records + ingestBuilder.allContainers.head.consumeRecords(converter) var partId = 1 val now = System.currentTimeMillis() val consumer = new BinaryRegionConsumer { def onNext(base: Any, offset: Long): Unit = { - val partKey = promCounter.partition.binSchema.asByteArray(base, offset) + val partKey = untyped.partition.binSchema.asByteArray(base, offset) partKeyIndex.addPartKey(partKey, partId, now)() partId += 1 } @@ -45,9 +53,10 @@ class PartKeyIndexBenchmark { @BenchmarkMode(Array(Mode.Throughput)) @OutputTimeUnit(TimeUnit.SECONDS) def partIdsLookupWithEqualsFilters(): Unit = { - for ( i <- 0 to 20 optimized) { + for ( i <- 0 until 8 optimized) { partKeyIndex.partIdsFromFilters( - Seq(ColumnFilter("job", Filter.Equals(s"App-$i")), + Seq(ColumnFilter("_ns_", Filter.Equals(s"App-$i")), + ColumnFilter("_ws_", Filter.Equals("demo")), ColumnFilter("host", Filter.EqualsRegex("H0")), ColumnFilter("__name__", Filter.Equals("heap_usage"))), now, @@ -55,13 +64,29 @@ class PartKeyIndexBenchmark { } } + @Benchmark + @BenchmarkMode(Array(Mode.Throughput)) + @OutputTimeUnit(TimeUnit.SECONDS) + def emptyPartIdsLookupWithEqualsFilters(): Unit = { + for ( i <- 0 until 8 optimized) { + partKeyIndex.partIdsFromFilters( + Seq(ColumnFilter("_ns_", Filter.Equals(s"App-${i + 200}")), + ColumnFilter("_ws_", Filter.Equals("demo")), + ColumnFilter("host", Filter.EqualsRegex("H0")), + ColumnFilter("__name__", Filter.Equals("heap_usage"))), + now, + now + 1000) + } + } + @Benchmark @BenchmarkMode(Array(Mode.Throughput)) @OutputTimeUnit(TimeUnit.SECONDS) def partIdsLookupWithSuffixRegexFilters(): Unit = { - for ( i <- 0 to 20 optimized) { + for ( i <- 0 until 8 optimized) { partKeyIndex.partIdsFromFilters( - Seq(ColumnFilter("job", Filter.Equals(s"App-$i")), + Seq(ColumnFilter("_ns_", Filter.Equals(s"App-$i")), + ColumnFilter("_ws_", Filter.Equals("demo")), ColumnFilter("__name__", Filter.Equals("heap_usage")), ColumnFilter("instance", Filter.EqualsRegex("Instance-2.*"))), now, @@ -73,9 +98,10 @@ class PartKeyIndexBenchmark { @BenchmarkMode(Array(Mode.Throughput)) @OutputTimeUnit(TimeUnit.SECONDS) def partIdsLookupWithPrefixRegexFilters(): Unit = { - for ( i <- 0 to 20 optimized) { + for ( i <- 0 until 8 optimized) { partKeyIndex.partIdsFromFilters( - Seq(ColumnFilter("job", Filter.Equals(s"App-$i")), + Seq(ColumnFilter("_ns_", Filter.Equals(s"App-$i")), + ColumnFilter("_ws_", Filter.Equals("demo")), ColumnFilter("__name__", Filter.Equals("heap_usage")), ColumnFilter("instance", Filter.EqualsRegex(".*2"))), now, @@ -87,8 +113,10 @@ class PartKeyIndexBenchmark { @BenchmarkMode(Array(Mode.Throughput)) @OutputTimeUnit(TimeUnit.SECONDS) def startTimeLookupWithPartId(): Unit = { - for ( i <- 0 to 20 optimized) { - partKeyIndex.startTimeFromPartId(i) + for ( i <- 0 until 8 optimized) { + val pIds = debox.Buffer.empty[Int] + for ( j <- i * 1000 to i * 1000 + 1000 optimized) { pIds += j } + partKeyIndex.startTimeFromPartIds(pIds.iterator()) } } diff --git a/jmh/src/main/scala/filodb.jmh/QueryHiCardInMemoryBenchmark.scala b/jmh/src/main/scala/filodb.jmh/QueryHiCardInMemoryBenchmark.scala index c6c3974a20..11a1f3d839 100644 --- a/jmh/src/main/scala/filodb.jmh/QueryHiCardInMemoryBenchmark.scala +++ b/jmh/src/main/scala/filodb.jmh/QueryHiCardInMemoryBenchmark.scala @@ -18,11 +18,10 @@ import filodb.core.SpreadChange import filodb.core.binaryrecord2.RecordContainer import filodb.core.memstore.{SomeData, TimeSeriesMemStore} import filodb.core.metadata.Schemas -import filodb.core.query.QueryContext +import filodb.core.query.{QueryConfig, QueryContext, QuerySession} import filodb.core.store.StoreConfig import filodb.prometheus.ast.TimeStepParams import filodb.prometheus.parse.Parser -import filodb.query.QueryConfig import filodb.query.exec.ExecPlan import filodb.timeseries.TestTimeseriesProducer @@ -133,7 +132,8 @@ class QueryHiCardInMemoryBenchmark extends StrictLogging { @OperationsPerInvocation(100) def scanSumOfRateBenchmark(): Unit = { (0 until numQueries).foreach { _ => - Await.result(scanSumOfRate.execute(store, queryConfig).runAsync, 60.seconds) + val querySession = QuerySession(QueryContext(), queryConfig) + Await.result(scanSumOfRate.execute(store, querySession).runAsync, 60.seconds) } } @@ -144,7 +144,8 @@ class QueryHiCardInMemoryBenchmark extends StrictLogging { @OperationsPerInvocation(100) def scanSumOfSumOverTimeBenchmark(): Unit = { (0 until numQueries).foreach { _ => - Await.result(scanSumSumOverTime.execute(store, queryConfig).runAsync, 60.seconds) + val querySession = QuerySession(QueryContext(), queryConfig) + Await.result(scanSumSumOverTime.execute(store, querySession).runAsync, 60.seconds) } } diff --git a/jmh/src/main/scala/filodb.jmh/QueryInMemoryBenchmark.scala b/jmh/src/main/scala/filodb.jmh/QueryInMemoryBenchmark.scala index 84041eec5c..2018037351 100644 --- a/jmh/src/main/scala/filodb.jmh/QueryInMemoryBenchmark.scala +++ b/jmh/src/main/scala/filodb.jmh/QueryInMemoryBenchmark.scala @@ -19,11 +19,11 @@ import filodb.core.SpreadChange import filodb.core.binaryrecord2.RecordContainer import filodb.core.memstore.{SomeData, TimeSeriesMemStore} import filodb.core.metadata.Schemas -import filodb.core.query.QueryContext +import filodb.core.query.{QueryConfig, QueryContext, QuerySession} import filodb.core.store.StoreConfig import filodb.prometheus.ast.TimeStepParams import filodb.prometheus.parse.Parser -import filodb.query.{QueryConfig, QueryError => QError, QueryResult => QueryResult2} +import filodb.query.{QueryError => QError, QueryResult => QueryResult2} import filodb.timeseries.TestTimeseriesProducer //scalastyle:off regex @@ -42,14 +42,17 @@ class QueryInMemoryBenchmark extends StrictLogging { import client.QueryCommands._ import NodeClusterActor._ - val numShards = 2 + import filodb.standalone.SimpleProfiler + val prof = new SimpleProfiler(5, 60, 50) + + val numShards = 32 val numSamples = 720 // 2 hours * 3600 / 10 sec interval val numSeries = 100 val startTime = System.currentTimeMillis - (3600*1000) val numQueries = 500 // Please make sure this number matches the OperationsPerInvocation below val queryIntervalMin = 55 // # minutes between start and stop val queryStep = 150 // # of seconds between each query sample "step" - val spread = 1 + val spread = 5 // TODO: move setup and ingestion to another trait val system = ActorSystem("test", ConfigFactory.load("filodb-defaults.conf")) @@ -70,8 +73,8 @@ class QueryInMemoryBenchmark extends StrictLogging { val storeConf = StoreConfig(ConfigFactory.parseString(""" | flush-interval = 1h - | shard-mem-size = 512MB - | ingestion-buffer-mem-size = 50MB + | shard-mem-size = 96MB + | ingestion-buffer-mem-size = 30MB | groups-per-shard = 4 | demand-paging-enabled = false """.stripMargin)) @@ -104,6 +107,7 @@ class QueryInMemoryBenchmark extends StrictLogging { Thread sleep 2000 cluster.memStore.asInstanceOf[TimeSeriesMemStore].refreshIndexForTesting(dataset.ref) // commit lucene index println(s"Ingestion ended") + prof.start() // Stuff for directly executing queries ourselves val engine = new SingleClusterPlanner(dataset.ref, Schemas(dataset.schema), shardMapper, 0, @@ -124,12 +128,17 @@ class QueryInMemoryBenchmark extends StrictLogging { val qParams = TimeStepParams(queryTime/1000, queryStep, (queryTime/1000) + queryIntervalMin*60) val logicalPlans = queries.map { q => Parser.queryRangeToLogicalPlan(q, qParams) } val queryCommands = logicalPlans.map { plan => - LogicalPlan2Query(dataset.ref, plan, QueryContext(Some(new StaticSpreadProvider(SpreadChange(0, 1))), 20000)) + LogicalPlan2Query(dataset.ref, plan, QueryContext(Some(new StaticSpreadProvider(SpreadChange(0, spread))), 20000)) } + var queriesSucceeded = 0 + var queriesFailed = 0 + @TearDown def shutdownFiloActors(): Unit = { cluster.shutdown() + println(s"Succeeded: $queriesSucceeded Failed: $queriesFailed") + prof.stop() } // Window = 5 min and step=2.5 min, so 50% overlap @@ -139,10 +148,12 @@ class QueryInMemoryBenchmark extends StrictLogging { @OperationsPerInvocation(500) def someOverlapQueries(): Unit = { val futures = (0 until numQueries).map { n => - val f = asyncAsk(coordinator, queryCommands(n % queryCommands.length)) + val qCmd = queryCommands(n % queryCommands.length) + val time = System.currentTimeMillis + val f = asyncAsk(coordinator, qCmd.copy(qContext = qCmd.qContext.copy(queryId = n.toString, submitTime = time))) f.onSuccess { - case q: QueryResult2 => - case e: QError => throw new RuntimeException(s"Query error $e") + case q: QueryResult2 => queriesSucceeded += 1 + case e: QError => queriesFailed += 1 } f } @@ -164,8 +175,8 @@ class QueryInMemoryBenchmark extends StrictLogging { val futures = (0 until numQueries).map { n => val f = asyncAsk(coordinator, queryCommands2(n % queryCommands2.length)) f.onSuccess { - case q: QueryResult2 => - case e: QError => throw new RuntimeException(s"Query error $e") + case q: QueryResult2 => queriesSucceeded += 1 + case e: QError => queriesFailed += 1 } f } @@ -186,8 +197,10 @@ class QueryInMemoryBenchmark extends StrictLogging { @OutputTimeUnit(TimeUnit.SECONDS) @OperationsPerInvocation(500) def singleThreadedRawQuery(): Long = { + val querySession = QuerySession(QueryContext(), queryConfig) + val f = Observable.fromIterable(0 until numQueries).mapAsync(1) { n => - execPlan.execute(cluster.memStore, queryConfig)(querySched) + execPlan.execute(cluster.memStore, querySession)(querySched) }.executeOn(querySched) .countL.runAsync Await.result(f, 60.seconds) @@ -203,7 +216,8 @@ class QueryInMemoryBenchmark extends StrictLogging { @OperationsPerInvocation(500) def singleThreadedMinOverTimeQuery(): Long = { val f = Observable.fromIterable(0 until numQueries).mapAsync(1) { n => - minEP.execute(cluster.memStore, queryConfig)(querySched) + val querySession = QuerySession(QueryContext(), queryConfig) + minEP.execute(cluster.memStore, querySession)(querySched) }.executeOn(querySched) .countL.runAsync Await.result(f, 60.seconds) @@ -219,7 +233,8 @@ class QueryInMemoryBenchmark extends StrictLogging { @OperationsPerInvocation(500) def singleThreadedSumRateCCQuery(): Long = { val f = Observable.fromIterable(0 until numQueries).mapAsync(1) { n => - sumRateEP.execute(cluster.memStore, queryConfig)(querySched) + val querySession = QuerySession(QueryContext(), queryConfig) + sumRateEP.execute(cluster.memStore, querySession)(querySched) }.executeOn(querySched) .countL.runAsync Await.result(f, 60.seconds) diff --git a/memory/src/main/java/filodb/memory/Latch.java b/memory/src/main/java/filodb/memory/Latch.java new file mode 100644 index 0000000000..296854c422 --- /dev/null +++ b/memory/src/main/java/filodb/memory/Latch.java @@ -0,0 +1,1138 @@ +/* + * Copyright (C) 2011-2017 Cojen.org + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * This class is copied from the Tupl project and modified slightly. In particular, it doesn't + * support LatchCondition, and it doesn't use Parker. Also, modified to implement the Lock + * interface for acquiring shared locks. + */ + +package filodb.memory; + +import java.lang.invoke.MethodHandles; +import java.lang.invoke.VarHandle; + +import java.util.concurrent.TimeUnit; + +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.LockSupport; + +/** + * Non-reentrant read-write latch, designed for throughout over fairness. Implementation + * doesn't track thread ownership or check for illegal usage. As a result, it typically + * outperforms ReentrantLock and built-in Java synchronization. Although latch acquisition is + * typically unfair, waiting threads aren't starved indefinitely. + * + * @author Brian S O'Neill + */ +public class Latch implements Lock { + public static final int UNLATCHED = 0, EXCLUSIVE = 0x80000000, SHARED = 1; + + static final int SPIN_LIMIT = Runtime.getRuntime().availableProcessors() > 1 ? 1 << 10 : 1; + + static final VarHandle cStateHandle, cFirstHandle, cLastHandle, + cWaiterHandle, cWaitStateHandle, cPrevHandle, cNextHandle; + + static { + try { + cStateHandle = + MethodHandles.lookup().findVarHandle + (Latch.class, "mLatchState", int.class); + + cFirstHandle = + MethodHandles.lookup().findVarHandle + (Latch.class, "mLatchFirst", WaitNode.class); + + cLastHandle = + MethodHandles.lookup().findVarHandle + (Latch.class, "mLatchLast", WaitNode.class); + + cWaiterHandle = + MethodHandles.lookup().findVarHandle + (WaitNode.class, "mWaiter", Object.class); + + cWaitStateHandle = + MethodHandles.lookup().findVarHandle + (WaitNode.class, "mWaitState", int.class); + + cPrevHandle = + MethodHandles.lookup().findVarHandle + (WaitNode.class, "mPrev", WaitNode.class); + + cNextHandle = + MethodHandles.lookup().findVarHandle + (WaitNode.class, "mNext", WaitNode.class); + } catch (Throwable e) { + throw new Error(e); + } + } + + private static void uncaught(Throwable e) { + Thread t = Thread.currentThread(); + t.getUncaughtExceptionHandler().uncaughtException(t, e); + } + + /* + unlatched: 0 latch is available + shared: 1..0x7fffffff latch is held shared + exclusive: 0x80000000 latch is held exclusively + illegal: 0x80000001..0xffffffff illegal exclusive state + */ + volatile int mLatchState; + + // Queue of waiting threads. + private volatile WaitNode mLatchFirst; + private volatile WaitNode mLatchLast; + + public Latch() { + } + + /** + * @param initialState UNLATCHED, EXCLUSIVE, or SHARED + */ + public Latch(int initialState) { + // Assume that this latch instance is published to other threads safely, and so a + // volatile store isn't required. + cStateHandle.set(this, initialState); + } + + @Override + public void lock() { + acquireShared(); + } + + @Override + public void lockInterruptibly() throws InterruptedException { + acquireSharedInterruptibly(); + } + + @Override + public boolean tryLock() { + return tryAcquireShared(); + } + + @Override + public boolean tryLock(long time, TimeUnit unit) throws InterruptedException { + return tryAcquireSharedNanos(unit.toNanos(time)); + } + + @Override + public void unlock() { + releaseShared(); + } + + @Override + public Condition newCondition() { + throw new UnsupportedOperationException(); + } + + boolean isHeldExclusive() { + return mLatchState == EXCLUSIVE; + } + + /** + * Try to acquire the exclusive latch, barging ahead of any waiting threads if possible. + */ + public boolean tryAcquireExclusive() { + return doTryAcquireExclusive(); + } + + private boolean doTryAcquireExclusive() { + return mLatchState == 0 && cStateHandle.compareAndSet(this, 0, EXCLUSIVE); + } + + private void doAcquireExclusiveSpin() { + while (!doTryAcquireExclusive()) { + Thread.onSpinWait(); + } + } + + /** + * Attempt to acquire the exclusive latch, aborting if interrupted. + * + * @param nanosTimeout pass negative for infinite timeout + */ + public boolean tryAcquireExclusiveNanos(long nanosTimeout) throws InterruptedException { + return doTryAcquireExclusiveNanos(nanosTimeout); + } + + private boolean doTryAcquireExclusiveNanos(long nanosTimeout) throws InterruptedException { + if (doTryAcquireExclusive()) { + return true; + } + + if (nanosTimeout == 0) { + return false; + } + + boolean result; + try { + result = acquire(new Timed(nanosTimeout)); + } catch (Throwable e) { + // Possibly an OutOfMemoryError. + if (nanosTimeout < 0) { + // Caller isn't expecting an exception, so spin. + doAcquireExclusiveSpin(); + return true; + } + return false; + } + + return checkTimedResult(result, nanosTimeout); + } + + /** + * Acquire the exclusive latch, barging ahead of any waiting threads if possible. + */ + public void acquireExclusive() { + if (!doTryAcquireExclusive()) { + doAcquireExclusive(); + } + } + + /** + * Caller should have already called tryAcquireExclusive. + */ + private void doAcquireExclusive() { + try { + acquire(new WaitNode()); + } catch (Throwable e) { + // Possibly an OutOfMemoryError. Caller isn't expecting an exception, so spin. + doAcquireExclusiveSpin(); + } + } + + /** + * Acquire the exclusive latch, aborting if interrupted. + */ + public void acquireExclusiveInterruptibly() throws InterruptedException { + doTryAcquireExclusiveNanos(-1); + } + + /** + * Invokes the given continuation upon the latch being acquired exclusively. When acquired, + * the continuation is run by the current thread, or it's enqueued to be run by a thread + * which releases the latch. The releasing thread actually retains the latch and runs the + * continuation, effectively transferring latch ownership. The continuation must not + * explicitly release the latch, although it can downgrade the latch. Any exception thrown + * by the continuation is passed to the uncaught exception handler of the running thread, + * and then the latch is released. + * + * @param cont called with latch held + */ + public void uponExclusive(Runnable cont) { + if (!doTryAcquireExclusive()) enqueue: { + WaitNode node; + try { + node = new WaitNode(cont, WaitNode.SIGNALED); + } catch (Throwable e) { + // Possibly an OutOfMemoryError. Caller isn't expecting an exception, so spin. + doAcquireExclusiveSpin(); + break enqueue; + } + + WaitNode prev = enqueue(node); + + boolean acquired = doTryAcquireExclusive(); + + if (node.mWaiter == null) { + // Continuation already ran or is running right now. + if (acquired) { + releaseExclusive(); + } + return; + } + + if (!acquired) { + return; + } + + cWaiterHandle.setOpaque(node, null); + + // Acquired while still in the queue. Remove the node now, releasing memory. + if (mLatchFirst != node) { + remove(node, prev); + } else { + removeFirst(node); + } + } + + try { + cont.run(); + } catch (Throwable e) { + uncaught(e); + } + + releaseExclusive(); + } + + /** + * Downgrade the held exclusive latch into a shared latch. Caller must later call + * releaseShared instead of releaseExclusive. + */ + public final void downgrade() { + mLatchState = 1; + + while (true) { + // Sweep through the queue, waking up a contiguous run of shared waiters. + final WaitNode first = first(); + if (first == null) { + return; + } + + WaitNode node = first; + while (true) { + Object waiter = node.mWaiter; + if (waiter != null) { + if (node instanceof Shared) { + cStateHandle.getAndAdd(this, 1); + if (cWaiterHandle.compareAndSet(node, waiter, null)) { + LockSupport.unpark((Thread) waiter); + } else { + // Already unparked, so fix the share count. + cStateHandle.getAndAdd(this, -1); + } + } else { + if (node != first) { + // Advance the queue past any shared waiters that were encountered. + mLatchFirst = node; + } + return; + } + } + + WaitNode next = node.mNext; + + if (next == null) { + // Queue is now empty, unless an enqueue is in progress. + if (cLastHandle.compareAndSet(this, node, null)) { + cFirstHandle.compareAndSet(this, first, null); + return; + } + // Sweep from the start again. + break; + } + + node = next; + } + } + } + + /** + * Release the held exclusive latch. + */ + public final void releaseExclusive() { + int trials = 0; + while (true) { + WaitNode last = mLatchLast; + + if (last == null) { + // No waiters, so release the latch. + mLatchState = 0; + + // Need to check if any waiters again, due to race with enqueue. If cannot + // immediately re-acquire the latch, then let the new owner (which barged in) + // unpark the waiters when it releases the latch. + last = mLatchLast; + if (last == null || !cStateHandle.compareAndSet(this, 0, EXCLUSIVE)) { + return; + } + } + + // Although the last waiter has been observed to exist, the first waiter field + // might not be set yet. + WaitNode first = mLatchFirst; + + unpark: if (first != null) { + Object waiter = first.mWaiter; + + if (waiter != null) { + if (first instanceof Shared) { + // TODO: can this be combined into one downgrade step? + downgrade(); + if (doReleaseShared()) { + return; + } + trials = 0; + continue; + } + + if (first.mWaitState != WaitNode.SIGNALED) { + // Unpark the waiter, but allow another thread to barge in. + mLatchState = 0; + LockSupport.unpark((Thread) waiter); + return; + } + } + + // Remove first from the queue. + { + WaitNode next = first.mNext; + if (next != null) { + mLatchFirst = next; + } else { + // Queue is now empty, unless an enqueue is in progress. + if (last != first || !cLastHandle.compareAndSet(this, last, null)) { + break unpark; + } + cFirstHandle.compareAndSet(this, last, null); + } + } + + if (waiter != null && cWaiterHandle.compareAndSet(first, waiter, null)) { + // Fair handoff to waiting thread or continuation. + if (waiter instanceof Thread) { + LockSupport.unpark((Thread) waiter); + return; + } + try { + ((Runnable) waiter).run(); + } catch (Throwable e) { + uncaught(e); + } + if (!isHeldExclusive()) { + if (mLatchState <= 0) { + throw new IllegalStateException + ("Illegal latch state: " + mLatchState + ", caused by " + waiter); + } + if (doReleaseShared()) { + return; + } + } + trials = 0; + continue; + } + } + + trials = spin(trials); + } + } + + /** + * Convenience method, which releases the held exclusive or shared latch. + * + * @param exclusive call releaseExclusive if true, else call releaseShared + */ + public final void release(boolean exclusive) { + if (exclusive) { + releaseExclusive(); + } else { + releaseShared(); + } + } + + /** + * Releases an exclusive or shared latch. + */ + public final void releaseEither() { + if (((int) cStateHandle.get(this)) == EXCLUSIVE) { + releaseExclusive(); + } else { + releaseShared(); + } + } + + /** + * Try to acquire a shared latch, barging ahead of any waiting threads if possible. + */ + public boolean tryAcquireShared() { + return doTryAcquireShared(); + } + + private boolean doTryAcquireShared() { + WaitNode first = mLatchFirst; + if (first != null && !(first instanceof Shared)) { + return false; + } + int state = mLatchState; + return state >= 0 && cStateHandle.compareAndSet(this, state, state + 1); + } + + private void doAcquireSharedSpin() { + while (!doTryAcquireShared()) { + Thread.onSpinWait(); + } + } + + /** + * Attempt to acquire a shared latch, aborting if interrupted. + * + * @param nanosTimeout pass negative for infinite timeout + */ + public boolean tryAcquireSharedNanos(long nanosTimeout) throws InterruptedException { + return doTryAcquireSharedNanos(nanosTimeout); + } + + private final boolean doTryAcquireSharedNanos(long nanosTimeout) throws InterruptedException { + WaitNode first = mLatchFirst; + if (first == null || first instanceof Shared) { + int trials = 0; + int state; + while ((state = mLatchState) >= 0) { + if (cStateHandle.compareAndSet(this, state, state + 1)) { + return true; + } + // Spin even if timeout is zero. The timeout applies to a blocking acquire. + trials = spin(trials); + } + } + + if (nanosTimeout == 0) { + return false; + } + + boolean result; + try { + result = acquire(new TimedShared(nanosTimeout)); + } catch (Throwable e) { + // Possibly an OutOfMemoryError. + if (nanosTimeout < 0) { + // Caller isn't expecting an exception, so spin. + doAcquireSharedSpin(); + return true; + } + return false; + } + + return checkTimedResult(result, nanosTimeout); + } + + private static boolean checkTimedResult(boolean result, long nanosTimeout) + throws InterruptedException + { + if (!result && (Thread.interrupted() || nanosTimeout < 0)) { + InterruptedException e; + try { + e = new InterruptedException(); + } catch (Throwable e2) { + // Possibly an OutOfMemoryError. + if (nanosTimeout < 0) { + throw e2; + } + return false; + } + throw e; + } + + return result; + } + + /** + * Like tryAcquireShared, except blocks if an exclusive latch is held. + * + * @return false if not acquired due to contention with other shared requests + */ + public boolean acquireSharedUncontended() { + WaitNode first = mLatchFirst; + if (first == null || first instanceof Shared) { + int state = mLatchState; + if (state >= 0) { + return cStateHandle.compareAndSet(this, state, state + 1); + } + } + + try { + acquire(new Shared()); + } catch (Throwable e) { + // Possibly an OutOfMemoryError. Caller isn't expecting an exception, so spin. + doAcquireSharedSpin(); + } + + return true; + } + + /** + * Like tryAcquireSharedNanos, except blocks if an exclusive latch is held. + * + * @param nanosTimeout pass negative for infinite timeout + * @return -1 if not acquired due to contention with other shared requests, 0 if timed out, + * or 1 if acquired + */ + public int acquireSharedUncontendedNanos(long nanosTimeout) throws InterruptedException { + WaitNode first = mLatchFirst; + if (first == null || first instanceof Shared) { + int state = mLatchState; + if (state >= 0) { + return cStateHandle.compareAndSet(this, state, state + 1) ? 1 : -1; + } + } + + boolean result; + try { + result = acquire(new TimedShared(nanosTimeout)); + } catch (Throwable e) { + // Possibly an OutOfMemoryError. + if (nanosTimeout < 0) { + // Caller isn't expecting an exception, so spin. + doAcquireSharedSpin(); + return 1; + } + return 0; + } + + return checkTimedResult(result, nanosTimeout) ? 1 : 0; + } + + /** + * Acquire a shared latch, barging ahead of any waiting threads if possible. + */ + public void acquireShared() { + if (!tryAcquireSharedSpin()) { + try { + acquire(new Shared()); + } catch (Throwable e) { + // Possibly an OutOfMemoryError. Caller isn't expecting an exception, so spin. + doAcquireSharedSpin(); + } + } + } + + private boolean tryAcquireSharedSpin() { + WaitNode first = mLatchFirst; + if (first == null || first instanceof Shared) { + int trials = 0; + int state; + while ((state = mLatchState) >= 0) { + if (cStateHandle.compareAndSet(this, state, state + 1)) { + return true; + } + trials = spin(trials); + } + } + return false; + } + + /** + * Acquire a shared latch, aborting if interrupted. + */ + public void acquireSharedInterruptibly() throws InterruptedException { + doTryAcquireSharedNanos(-1); + } + + /** + * Attempt to upgrade a held shared latch into an exclusive latch. Upgrade fails if shared + * latch is held by more than one thread. If successful, caller must later call + * releaseExclusive instead of releaseShared. + */ + public boolean tryUpgrade() { + return doTryUpgrade(); + } + + private boolean doTryUpgrade() { + while (true) { + if (mLatchState != 1) { + return false; + } + if (cStateHandle.compareAndSet(this, 1, EXCLUSIVE)) { + return true; + } + Thread.onSpinWait(); + } + } + + /** + * Release a held shared latch. + */ + public void releaseShared() { + int trials = 0; + while (true) { + int state = mLatchState; + + WaitNode last = mLatchLast; + if (last == null) { + // No waiters, so release the latch. + if (cStateHandle.compareAndSet(this, state, --state)) { + if (state == 0) { + // Need to check if any waiters again, due to race with enqueue. If + // cannot immediately re-acquire the latch, then let the new owner + // (which barged in) unpark the waiters when it releases the latch. + last = mLatchLast; + if (last != null && cStateHandle.compareAndSet(this, 0, EXCLUSIVE)) { + releaseExclusive(); + } + } + return; + } + } else if (state == 1) { + // Try to switch to exclusive, and then let releaseExclusive deal with + // unparking the waiters. + if (cStateHandle.compareAndSet(this, 1, EXCLUSIVE) || doTryUpgrade()) { + releaseExclusive(); + return; + } + } else if (cStateHandle.compareAndSet(this, state, --state)) { + return; + } + + trials = spin(trials); + } + } + + /** + * @return false if latch is held exclusive now + */ + private boolean doReleaseShared() { + // Note: Same as regular releaseShared, except doesn't recurse into the + // releaseExclusive method. + + int trials = 0; + while (true) { + int state = mLatchState; + + WaitNode last = mLatchLast; + if (last == null) { + if (cStateHandle.compareAndSet(this, state, --state)) { + if (state == 0) { + last = mLatchLast; + if (last != null && cStateHandle.compareAndSet(this, 0, EXCLUSIVE)) { + return false; + } + } + return true; + } + } else if (state == 1) { + if (cStateHandle.compareAndSet(this, 1, EXCLUSIVE) || doTryUpgrade()) { + return false; + } + } else if (cStateHandle.compareAndSet(this, state, --state)) { + return true; + } + + trials = spin(trials); + } + } + + private boolean acquire(final WaitNode node) { + node.mWaiter = Thread.currentThread(); + WaitNode prev = enqueue(node); + int acquireResult = node.tryAcquire(this); + + if (acquireResult < 0) { + int denied = 0; + while (true) { + boolean parkAbort = node.parkNow(this); + + acquireResult = node.tryAcquire(this); + + if (acquireResult >= 0) { + // Latch acquired after parking. + break; + } + + if (parkAbort) { + if (!cWaiterHandle.compareAndSet(node, Thread.currentThread(), null)) { + // Fair handoff just occurred. + return true; + } + + if (!(node instanceof Shared) && mLatchState >= 0) { + // Unpark any shared waiters that queued behind this exclusive request. + WaitNode wnode = node; + while ((wnode = wnode.mNext) instanceof Shared) { + Object waiter = wnode.mWaiter; + if (waiter instanceof Thread) { + LockSupport.unpark((Thread) waiter); + } + } + } + + // Remove the node from the queue. If it's the first, it cannot be safely + // removed without the latch having been properly acquired. So let it + // linger around until the latch is released. + if (prev != null) { + remove(node, prev); + } + + return false; + } + + // Lost the race. Request fair handoff. + + if (denied++ == 0) { + node.mWaitState = WaitNode.SIGNALED; + } + } + } + + if (acquireResult == 0) { + // Remove the node now, releasing memory. + if (mLatchFirst != node) { + remove(node, prev); + } else { + removeFirst(node); + } + } + + return true; + } + + private void removeFirst(WaitNode node) { + // Removing the first node requires special attention. Because the latch is now held by + // the current thread, no other dequeues are in progress, but enqueues still are. + + while (true) { + WaitNode next = node.mNext; + if (next != null) { + mLatchFirst = next; + return; + } else { + // Queue is now empty, unless an enqueue is in progress. + WaitNode last = mLatchLast; + if (last == node && cLastHandle.compareAndSet(this, last, null)) { + cFirstHandle.compareAndSet(this, last, null); + return; + } + } + } + } + + private WaitNode enqueue(final WaitNode node) { + var prev = (WaitNode) cLastHandle.getAndSet(this, node); + + if (prev == null) { + mLatchFirst = node; + } else { + prev.mNext = node; + WaitNode pp = prev.mPrev; + if (pp != null) { + // The old last node was intended to be removed, but the last node cannot + // be removed unless it's also the first. Bypass it now that a new last + // node has been enqueued. + cNextHandle.setRelease(pp, node); + // Return a more correct previous node, although it might be stale. Node + // removal is somewhat lazy, and accurate removal is performed when the + // exclusive latch is released. + prev = pp; + } + } + + return prev; + } + + /** + * Should only be called after clearing the mWaiter field. Ideally, multiple threads + * shouldn't be calling this method, because it can cause nodes to be resurrected and + * remain in the queue longer than necessary. They'll get cleaned out eventually. The + * problem is caused by the prev node reference, which might have changed or have been + * removed by the time this method is called. + * + * @param node node to remove, not null + * @param prev previous node, not null + */ + private void remove(final WaitNode node, final WaitNode prev) { + WaitNode next = node.mNext; + + if (next == null) { + // Removing the last node creates race conditions with enqueues. Instead, stash a + // reference to the previous node and let the enqueue deal with it after a new node + // has been enqueued. + node.mPrev = prev; + next = node.mNext; + // Double check in case an enqueue just occurred that may have failed to notice the + // previous node assignment. + if (next == null) { + return; + } + } + + while (next.mWaiter == null) { + // Skip more nodes if possible. + WaitNode nextNext = next.mNext; + if (nextNext == null) { + break; + } + next = nextNext; + } + + // Bypass the removed node, allowing it to be released. + cNextHandle.setRelease(prev, next); + } + + private WaitNode first() { + int trials = 0; + while (true) { + WaitNode last = mLatchLast; + if (last == null) { + return null; + } + // Although the last waiter has been observed to exist, the first waiter field + // might not be set yet. + WaitNode first = mLatchFirst; + if (first != null) { + return first; + } + trials = spin(trials); + } + } + + /** + * Returns the first waiter in the queue that's actually still waiting. + */ + private WaitNode firstWaiter() { + WaitNode first = mLatchFirst; + WaitNode next; + if (first == null || first.mWaiter != null || (next = first.mNext) == null) { + return first; + } + if (next.mWaiter != null) { + return next; + } + // Clean out some stale nodes. Note that removing the first node isn't safe. + remove(next, first); + return null; + } + + public final boolean hasQueuedThreads() { + return mLatchLast != null; + } + + @Override + public String toString() { + var b = new StringBuilder(); + appendMiniString(b, this); + b.append(" {state="); + + int state = mLatchState; + if (state == 0) { + b.append("unlatched"); + } else if (state == EXCLUSIVE) { + b.append("exclusive"); + } else if (state >= 0) { + b.append("shared:").append(state); + } else { + b.append("illegal:").append(state); + } + + WaitNode last = mLatchLast; + + if (last != null) { + b.append(", "); + WaitNode first = mLatchFirst; + if (first == last) { + b.append("firstQueued=").append(last); + } else if (first == null) { + b.append("lastQueued=").append(last); + } else { + b.append("firstQueued=").append(first) + .append(", lastQueued=").append(last); + } + } + + return b.append('}').toString(); + } + + static void appendMiniString(StringBuilder b, Object obj) { + if (obj == null) { + b.append("null"); + return; + } + b.append(obj.getClass().getName()).append('@').append(Integer.toHexString(obj.hashCode())); + } + + /** + * @return new trials value + */ + static int spin(int trials) { + trials++; + if (trials >= SPIN_LIMIT) { + Thread.yield(); + trials = 0; + } else { + Thread.onSpinWait(); + } + return trials; + } + + static class WaitNode { + volatile Object mWaiter; + + static final int SIGNALED = 1, COND_WAIT = 2, COND_WAIT_SHARED = 3; + volatile int mWaitState; + + // Only set if node was deleted and must be bypassed when a new node is enqueued. + volatile WaitNode mPrev; + volatile WaitNode mNext; + + /** + * Constructor for latch wait. + */ + WaitNode() { + } + + /** + * Constructor for condition wait. Caller must hold exclusive latch. + */ + WaitNode(Object waiter, int waitState) { + cWaiterHandle.set(this, waiter); + cWaitStateHandle.set(this, waitState); + } + + /** + * @return true if timed out or interrupted + */ + boolean parkNow(Latch latch) { + LockSupport.park(latch); + return false; + } + + /** + * @return {@literal <0 if thread should park; 0 if acquired and node should also be + * removed; >0 if acquired and node should not be removed} + */ + int tryAcquire(Latch latch) { + int trials = 0; + while (true) { + for (int i=0; i= SPIN_LIMIT >> 1) { + return -1; + } + // Yield to avoid parking. + Thread.yield(); + } + } + + @Override + public String toString() { + var b = new StringBuilder(); + appendMiniString(b, this); + b.append(" {waiter=").append(mWaiter); + b.append(", state=").append(mWaitState); + b.append(", next="); appendMiniString(b, mNext); + b.append(", prev="); appendMiniString(b, mPrev); + return b.append('}').toString(); + } + } + + static class Timed extends WaitNode { + private long mNanosTimeout; + private long mEndNanos; + + Timed(long nanosTimeout) { + mNanosTimeout = nanosTimeout; + if (nanosTimeout >= 0) { + mEndNanos = System.nanoTime() + nanosTimeout; + } + } + + @Override + final boolean parkNow(Latch latch) { + if (mNanosTimeout < 0) { + LockSupport.park(latch); + return Thread.currentThread().isInterrupted(); + } else { + LockSupport.parkNanos(latch, mNanosTimeout); + if (Thread.currentThread().isInterrupted()) { + return true; + } + return (mNanosTimeout = mEndNanos - System.nanoTime()) <= 0; + } + } + } + + static class Shared extends WaitNode { + /** + * @return {@literal <0 if thread should park; 0 if acquired and node should also be + * removed; >0 if acquired and node should not be removed} + */ + @Override + final int tryAcquire(Latch latch) { + // Note: If mWaiter is null, then handoff was fair. The shared count should already + // be correct, and this node won't be in the queue anymore. + + WaitNode first = latch.firstWaiter(); + if (first != null && !(first instanceof Shared)) { + return mWaiter == null ? 1 : -1; + } + + int trials = 0; + while (true) { + if (mWaiter == null) { + return 1; + } + + int state = latch.mLatchState; + if (state < 0) { + return state; + } + + if (cStateHandle.compareAndSet(latch, state, state + 1)) { + // Acquired, so no need to reference the thread anymore. + Object waiter = mWaiter; + if (waiter == null || !cWaiterHandle.compareAndSet(this, waiter, null)) { + if (!cStateHandle.compareAndSet(latch, state + 1, state)) { + cStateHandle.getAndAdd(latch, -1); + } + return 1; + } + + // Only instruct the caller to remove this node if this is the first shared + // latch owner (the returned state value will be 0). This guarantees that + // no other thread will be concurrently calling removeFirst. The node will + // be removed after an exclusive latch is released, or when firstWaiter is + // called again. Note that it's possible to return 0 every time, but only + // if the caller is also instructed to never call removeFirst. + return state; + } + + trials = spin(trials); + } + } + } + + static class TimedShared extends Shared { + private long mNanosTimeout; + private long mEndNanos; + + TimedShared(long nanosTimeout) { + mNanosTimeout = nanosTimeout; + if (nanosTimeout >= 0) { + mEndNanos = System.nanoTime() + nanosTimeout; + } + } + + @Override + final boolean parkNow(Latch latch) { + if (mNanosTimeout < 0) { + LockSupport.park(latch); + return Thread.currentThread().isInterrupted(); + } else { + LockSupport.parkNanos(latch, mNanosTimeout); + if (Thread.currentThread().isInterrupted()) { + return true; + } + return (mNanosTimeout = mEndNanos - System.nanoTime()) <= 0; + } + } + } +} diff --git a/memory/src/main/scala/filodb.memory/BlockDetective.scala b/memory/src/main/scala/filodb.memory/BlockDetective.scala index be5ce818cf..0126a2c3df 100644 --- a/memory/src/main/scala/filodb.memory/BlockDetective.scala +++ b/memory/src/main/scala/filodb.memory/BlockDetective.scala @@ -12,8 +12,8 @@ object BlockDetective { def containsPtr(ptr: BinaryRegion.NativePointer, blocks: Seq[Block]): Seq[Block] = blocks.filter { blk => ptr >= blk.address && ptr < (blk.address + blk.capacity) } - def containsPtr(ptr: BinaryRegion.NativePointer, blocks: java.util.List[Block]): Seq[Block] = - containsPtr(ptr, blocks.asScala) + def containsPtr(ptr: BinaryRegion.NativePointer, blocks: java.lang.Iterable[Block]): Seq[Block] = + containsPtr(ptr, Seq() ++ blocks.asScala) /** * Produces a string report containing reclaim history and ownership changes for diff --git a/memory/src/main/scala/filodb.memory/BlockManager.scala b/memory/src/main/scala/filodb.memory/BlockManager.scala index 548ce2fc59..dc1e57e503 100644 --- a/memory/src/main/scala/filodb.memory/BlockManager.scala +++ b/memory/src/main/scala/filodb.memory/BlockManager.scala @@ -2,6 +2,7 @@ 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} @@ -10,6 +11,8 @@ import kamon.Kamon import kamon.metric.Counter import kamon.tag.TagSet +final case class MemoryRequestException(msg: String) extends Exception(msg) + /** * Allows requesting blocks. */ @@ -45,6 +48,39 @@ trait BlockManager { */ def requestBlock(bucketTime: Option[Long], owner: Option[BlockMemFactory] = None): Option[Block] + /** + * Attempts to reclaim as many blocks as necessary to ensure that enough free blocks are + * available. + * + * @return numFreeBlocks + */ + def ensureFreeBlocks(num: Int): Int + + /** + * Attempts to reclaim as many blocks as necessary to ensure that enough free bytes are + * available. The actual amount reclaimed might be higher than requested. + * + * @return numFreeBlocks + */ + def ensureFreeBytes(amt: Long): Int = { + val blocks = (amt + blockSizeInBytes - 1) / blockSizeInBytes + ensureFreeBlocks(Math.min(Integer.MAX_VALUE, blocks).toInt) + } + + /** + * Attempts to reclaim as many blocks as necessary to ensure that enough free bytes are + * available as a percentage of total size. The actual amount reclaimed might be higher than + * requested. + * + * @param pct percentage: 0.0 to 100.0 + * @return numFreeBlocks + */ + def ensureFreePercent(pct: Double): Int = { + ensureFreeBytes((totalMemorySizeInBytes * pct * 0.01).toLong) + } + + def totalMemorySizeInBytes: Long + /** * Releases all blocks allocated by this store. */ @@ -69,6 +105,18 @@ class MemoryStats(tags: Map[String, String]) { val timeOrderedBlocksReclaimedMetric = Kamon.counter("blockstore-time-ordered-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 + * free space. Unit is nanoseconds. + */ + val blockHeadroomStall = Kamon.counter("blockstore-headroom-stall-nanos").withTags(TagSet.from(tags)) + + /** + * How much time a thread was stalled while attempting to acquire the reclaim lock. + * Unit is nanoseconds. + */ + val blockReclaimStall = Kamon.counter("blockstore-reclaim-stall-nanos").withTags(TagSet.from(tags)) } final case class ReclaimEvent(block: Block, reclaimTime: Long, oldOwner: Option[BlockMemFactory], remaining: Long) @@ -100,13 +148,16 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, protected var firstPageAddress: Long = 0L - protected val freeBlocks: util.LinkedList[Block] = allocate() - protected[memory] val usedBlocks: util.LinkedList[Block] = new util.LinkedList[Block]() - protected[memory] val usedBlocksTimeOrdered = new util.TreeMap[Long, util.LinkedList[Block]] + 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]] val reclaimLog = new collection.mutable.Queue[ReclaimEvent] protected val lock = new ReentrantLock() + // Acquired when reclaiming on demand. Acquire shared lock to prevent block reclamation. + final val reclaimLock = new Latch + override def blockSizeInBytes: Long = PageManager.getInstance().pageSize() * numPagesPerBlock def availablePreAllocated: Long = numFreeBlocks * blockSizeInBytes @@ -136,16 +187,33 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, * 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 + * 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 + * 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] = { + val num: Int = Math.ceil(memorySize / blockSizeInBytes).toInt + stats.requestedBlocksMetric.increment(num) + lock.lock() try { - val num: Int = Math.ceil(memorySize / blockSizeInBytes).toInt - stats.requestedBlocksMetric.increment(num) - - if (freeBlocks.size < num) tryReclaim(num) + if (freeBlocks.size < num) { + if (bucketTime.isEmpty) { + tryReclaimOnDemand(num) + } else { + val msg = s"Unable to allocate time ordered block(s) without forcing a reclamation: " + + s"num_blocks=$num num_bytes=$memorySize freeBlocks=${freeBlocks.size}" + throw new MemoryRequestException(msg) + } + } if (freeBlocks.size >= num) { val allocated = new Array[Block](num) @@ -165,9 +233,123 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, } } - protected def allocate(): util.LinkedList[Block] = { + /** + * Internal variant of the tryReclaim method which is called when blocks are requested, but + * none are available. Instead of blindly reclaiming blocks, it attempts to exclusively + * acquire the reclaim lock. By doing so, it avoids reclaiming blocks which are currently + * being accessed. To work properly, all threads which require this protection must hold the + * shared reclaimLock. To prevent indefinite stalls, this method times out lock acquisition, + * logs an error, and then reclaims anyhow. + * + * This method must be called with the primary lock object held. To avoid deadlock, this + * method releases and re-acquires the lock. + */ + private def tryReclaimOnDemand(num: Int): Unit = { + lock.unlock() + var acquired: Boolean = false + try { + val start = System.nanoTime() + // Give up after waiting (in total) a little over 16 seconds. + acquired = tryExclusiveReclaimLock(8192) + + if (!acquired) { + // Don't stall ingestion forever. Some queries might return invalid results because + // the lock isn't held. If the lock state is broken, then ingestion is really stuck + // and the node must be restarted. Queries should always release the lock. + logger.error(s"Lock for BlockManager.tryReclaimOnDemand timed out: ${reclaimLock}") + } else { + logger.debug("Lock for BlockManager.tryReclaimOnDemand aquired") + } + + val stall = System.nanoTime() - start + stats.blockReclaimStall.increment(stall) + } finally { + lock.lock() + } + + try { + if (numFreeBlocks < num) { // double check since lock was released + tryReclaim(num) + } + } finally { + if (acquired) { + reclaimLock.releaseExclusive() + } + } + } + + private def tryExclusiveReclaimLock(finalTimeoutMillis: Int): Boolean = { + // Attempting to acquire the exclusive lock must wait for concurrent queries to finish, but + // waiting will also stall new queries from starting. To protect against this, attempt with + // a timeout to let any stalled queries through. To prevent starvation of the exclusive + // lock attempt, increase the timeout each time, but eventually give up. The reason why + // waiting for an exclusive lock causes this problem is that the thread must enqueue itself + // into the lock as a waiter, and all new shared requests must wait their turn. The risk + // with timing out is that if there's a continuous stream of long running queries (more than + // one second), then the exclusive lock will never be acqiured, and then ensureFreeBlocks + // won't be able to do its job. The timeout settings might need to be adjusted in that case. + // Perhaps the timeout should increase automatically if ensureFreeBlocks failed the last time? + // This isn't safe to do until we gain higher confidence that the shared lock is always + // released by queries. + + var timeout = 1; + while (true) { + val acquired = reclaimLock.tryAcquireExclusiveNanos(TimeUnit.MILLISECONDS.toNanos(timeout)) + if (acquired) { + return true + } + timeout <<= 1 + if (timeout > finalTimeoutMillis) { + return false + } + Thread.`yield`() + } + false // never reached, but scala compiler complains otherwise + } + + /** + * Expected to be called via a background task, to periodically ensure that enough blocks + * are free for new allocations. This helps prevent ODP activity from reclaiming immediately + * from itself. + * + * @param pct percentage: 0.0 to 100.0 + */ + def ensureHeadroom(pct: Double): Int = { + var numFree: Int = 0 + val start = System.nanoTime() + // Give up after waiting (in total) a little over 2 seconds. + val acquired = tryExclusiveReclaimLock(1024) + if (!acquired) { + logger.warn(s"Lock for BlockManager.ensureFreePercent timed out: ${reclaimLock}") + numFree = numFreeBlocks + } else { + try { + numFree = ensureFreePercent(pct) + } finally { + reclaimLock.releaseExclusive() + } + val numBytes = numFree * blockSizeInBytes + logger.debug(s"BlockManager.ensureFreePercent numFree: $numFree ($numBytes bytes)") + } + val stall = System.nanoTime() - start + stats.blockHeadroomStall.increment(stall) + numFree + } + + override def ensureFreeBlocks(num: Int): Int = { + lock.lock() + try { + val require = num - numFreeBlocks + if (require > 0) tryReclaim(require) + numFreeBlocks + } finally { + lock.unlock() + } + } + + protected def allocate(): util.ArrayDeque[Block] = { val numBlocks: Int = Math.floor(totalMemorySizeInBytes / blockSizeInBytes).toInt - val blocks = new util.LinkedList[Block]() + val blocks = new util.ArrayDeque[Block]() logger.info(s"Allocating $numBlocks blocks of $blockSizeInBytes bytes each, total $totalMemorySizeInBytes") firstPageAddress = MemoryIO.getCheckedInstance().allocateMemory(totalMemorySizeInBytes, false) for (i <- 0 until numBlocks) { @@ -182,7 +364,7 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, block.markInUse bucketTime match { case Some(bucket) => val blockList = Option(usedBlocksTimeOrdered.get(bucket)).getOrElse { - val list = new util.LinkedList[Block]() + val list = new util.ArrayDeque[Block]() usedBlocksTimeOrdered.put(bucket, list) list } @@ -224,7 +406,7 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, s"usedBlocksTimeOrdered=${usedBlocksTimeOrdered.asScala.toList.map{case(n, l) => (n, l.size)}}") } - def reclaimFrom(list: util.LinkedList[Block], reclaimedCounter: Counter): Seq[Block] = { + def reclaimFrom(list: util.ArrayDeque[Block], reclaimedCounter: Counter): Seq[Block] = { val entries = list.iterator val removed = new collection.mutable.ArrayBuffer[Block] while (entries.hasNext && reclaimed < num) { diff --git a/memory/src/test/scala/filodb.memory/PageAlignedBlockManagerSpec.scala b/memory/src/test/scala/filodb.memory/PageAlignedBlockManagerSpec.scala index 4e62faa440..7db1be1180 100644 --- a/memory/src/test/scala/filodb.memory/PageAlignedBlockManagerSpec.scala +++ b/memory/src/test/scala/filodb.memory/PageAlignedBlockManagerSpec.scala @@ -18,7 +18,6 @@ object PageAlignedBlockManagerSpec { class PageAlignedBlockManagerSpec extends FlatSpec with Matchers with BeforeAndAfter { import PageAlignedBlockManagerSpec._ - import collection.JavaConverters._ val pageSize = PageManager.getInstance().pageSize() @@ -114,6 +113,7 @@ class PageAlignedBlockManagerSpec extends FlatSpec with Matchers with BeforeAndA blockManager.releaseBlocks() } + /* No longer valid now that time ordered allocation doesn't force reclamation. it should "allocate and reclaim blocks with time order" in { val stats = new MemoryStats(Map("test5" -> "test5")) // This block manager has 5 blocks capacity @@ -168,6 +168,42 @@ class PageAlignedBlockManagerSpec extends FlatSpec with Matchers with BeforeAndA blockManager.releaseBlocks() } + */ + + it should "fail to allocate time order block" in { + val stats = new MemoryStats(Map("test5" -> "test5")) + // This block manager has 5 blocks capacity + val blockManager = new PageAlignedBlockManager(5 * pageSize, stats, testReclaimer, 1) + + blockManager.usedBlocks.size() shouldEqual 0 + blockManager.numTimeOrderedBlocks shouldEqual 0 + blockManager.usedBlocksTimeOrdered.size shouldEqual 0 + + // 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.requestBlock(Some(9000L)).map(_.markReclaimable).isDefined shouldEqual true + blockManager.usedBlocksTimeOrdered.get(9000L).size() shouldEqual 1 + + blockManager.numTimeOrderedBlocks shouldEqual 4 + blockManager.usedBlocksTimeOrdered.size shouldEqual 2 + + // reclaim from time ordered blocks should fail now + try { + blockManager.requestBlock(Some(10000L)) + fail + } catch { + case e: MemoryRequestException => // expected + } + + blockManager.releaseBlocks() + } it should ("allocate blocks using BlockMemFactory with ownership and reclaims") in { val stats = new MemoryStats(Map("test5" -> "test5")) @@ -198,10 +234,45 @@ class PageAlignedBlockManagerSpec extends FlatSpec with Matchers with BeforeAndA // Mark as reclaimable the blockMemFactory's block. Then request more blocks, that one will be reclaimed. // Check ownership is now cleared. factory.currentBlock.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 } + + it should "ensure free space" in { + val stats = new MemoryStats(Map("test5" -> "test5")) + // This block manager has 5 blocks capacity + val blockManager = new PageAlignedBlockManager(5 * pageSize, stats, testReclaimer, 1) + + blockManager.numFreeBlocks shouldEqual 5 + blockManager.ensureFreePercent(50) + blockManager.numFreeBlocks shouldEqual 5 + + blockManager.requestBlock(None).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.numFreeBlocks shouldEqual 3 + blockManager.ensureFreePercent(50) + blockManager.numFreeBlocks shouldEqual 3 + + blockManager.requestBlock(None).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.numFreeBlocks shouldEqual 2 + blockManager.ensureFreePercent(90) + // Should reclaim multiple blocks. + blockManager.numFreeBlocks shouldEqual 5 + + blockManager.releaseBlocks() + } } diff --git a/project/FiloBuild.scala b/project/FiloBuild.scala index bee2872a20..27989aab7b 100644 --- a/project/FiloBuild.scala +++ b/project/FiloBuild.scala @@ -155,7 +155,7 @@ object Submodules { lazy val jmh = (project in file("jmh")) .enablePlugins(JmhPlugin) - .dependsOn(core % "compile->compile; compile->test", gateway) + .dependsOn(core % "compile->compile; compile->test", gateway, standalone) .settings( commonSettings, name := "filodb-jmh", diff --git a/project/plugins.sbt b/project/plugins.sbt index 54f4063a75..63de00f3d3 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -2,7 +2,7 @@ addSbtPlugin("org.scalastyle" %% "scalastyle-sbt-plugin" % "1.0.0") addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.14.5") -addSbtPlugin("pl.project13.scala" % "sbt-jmh" % "0.3.3") +addSbtPlugin("pl.project13.scala" % "sbt-jmh" % "0.3.7") addSbtPlugin("com.typesafe.sbt" % "sbt-multi-jvm" % "0.4.0") diff --git a/query/src/main/scala/filodb/query/LogicalPlan.scala b/query/src/main/scala/filodb/query/LogicalPlan.scala index d67354f91a..f449552d3e 100644 --- a/query/src/main/scala/filodb/query/LogicalPlan.scala +++ b/query/src/main/scala/filodb/query/LogicalPlan.scala @@ -336,6 +336,10 @@ case class ApplyAbsentFunction(vectors: PeriodicSeriesPlan, override def endMs: Long = vectors.endMs } +case class LabelValueOperator(columnName: String, value: Seq[String], operator: String) + +case class LabelValueOperatorGroup(labelValueOperators: Seq[LabelValueOperator]) + object LogicalPlan { /** * Get leaf Logical Plans @@ -347,5 +351,51 @@ object LogicalPlan { case _ => Seq(logicalPlan) } } + + def getLabelValueFromLogicalPlan(logicalPlan: LogicalPlan, labelName: String): Option[Seq[String]] = { + getLabelValueFromLogicalPlan(getLabelValueOperatorsFromLogicalPlan(logicalPlan), labelName) + } + + def getLabelValueFromLogicalPlan(labelValues: Option[Seq[LabelValueOperatorGroup]], + labelName: String): Option[Seq[String]] = { + labelValues match { + case None => None + case _ => labelValues.get.flatMap(group => + group.labelValueOperators.flatMap(lops => { + lops.columnName.equals(labelName) match { + case true => lops.value + case false => Seq() + } + })).distinct match { + case Nil => None + case lVFilters: Seq[String] => Some(lVFilters) + } + } + } + + private def getLabelValueOpsFromFilters(filters: Seq[ColumnFilter]): Option[LabelValueOperatorGroup] = { + Some(LabelValueOperatorGroup(filters.map(cf => LabelValueOperator(cf.column, + cf.filter.valuesStrings.map(_.toString).toSeq.sorted, cf.filter.operatorString)))) + } + + def getLabelValueOperatorsFromLogicalPlan(logicalPlan: LogicalPlan): Option[Seq[LabelValueOperatorGroup]] = { + LogicalPlan.findLeafLogicalPlans(logicalPlan).flatMap { lp => + lp match { + case lp: LabelValues => Some( + LabelValueOperatorGroup( + lp.labelConstraints.map(lbc => LabelValueOperator(lbc._1, Seq(lbc._2), "=")).toSeq)) + case lp: RawSeries => getLabelValueOpsFromFilters(lp.filters) + case lp: RawChunkMeta => getLabelValueOpsFromFilters(lp.filters) + case lp: SeriesKeysByFilters => getLabelValueOpsFromFilters(lp.filters) + case _: ScalarTimeBasedPlan => Nil // Plan does not have labels + case _: ScalarFixedDoublePlan => Nil + case _: ScalarBinaryOperation => Nil + case _ => throw new BadQueryException(s"Invalid logical plan $logicalPlan") + } + } match { + case Nil => None + case groupSeq: Seq[LabelValueOperatorGroup] => Some(groupSeq) + } + } } //scalastyle:on number.of.types \ 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 496a0e2d4e..1ec4cd38fe 100644 --- a/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala +++ b/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala @@ -25,7 +25,7 @@ final case class ReduceAggregateExec(queryContext: QueryContext, protected def compose(childResponses: Observable[(QueryResponse, Int)], firstSchema: Task[ResultSchema], - queryConfig: QueryConfig): Observable[RangeVector] = { + querySession: QuerySession): Observable[RangeVector] = { val results = childResponses.flatMap { case (QueryResult(_, _, result), _) => Observable.fromIterable(result) case (QueryError(_, ex), _) => throw ex @@ -55,7 +55,7 @@ final case class AggregateMapReduce(aggrOp: AggregationOperator, s"aggrOp=$aggrOp, aggrParams=$aggrParams, without=$without, by=$by" def apply(source: Observable[RangeVector], - queryConfig: QueryConfig, + querySession: QuerySession, limit: Int, sourceSchema: ResultSchema, paramResponse: Seq[Observable[ScalarRangeVector]] = Nil): Observable[RangeVector] = { @@ -71,7 +71,7 @@ final case class AggregateMapReduce(aggrOp: AggregationOperator, // IF no grouping is done AND prev transformer is Periodic (has fixed length), use optimal path if (without.isEmpty && by.isEmpty && sourceSchema.fixedVectorLen.isDefined) { - sourceSchema.fixedVectorLen.filter(_ <= queryConfig.fastReduceMaxWindows).map { numWindows => + sourceSchema.fixedVectorLen.filter(_ <= querySession.queryConfig.fastReduceMaxWindows).map { numWindows => RangeVectorAggregator.fastReduce(aggregator, false, source, numWindows) }.getOrElse { RangeVectorAggregator.mapReduce(aggregator, skipMapPhase = false, source, grouping) @@ -95,7 +95,7 @@ final case class AggregatePresenter(aggrOp: AggregationOperator, protected[exec] def args: String = s"aggrOp=$aggrOp, aggrParams=$aggrParams" def apply(source: Observable[RangeVector], - queryConfig: QueryConfig, + querySession: QuerySession, limit: Int, sourceSchema: ResultSchema, paramResponse: Seq[Observable[ScalarRangeVector]]): Observable[RangeVector] = { diff --git a/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala b/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala index b63e76240b..59277a6676 100644 --- a/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala +++ b/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala @@ -63,7 +63,7 @@ final case class BinaryJoinExec(queryContext: QueryContext, protected[exec] def compose(childResponses: Observable[(QueryResponse, Int)], firstSchema: Task[ResultSchema], - queryConfig: QueryConfig): Observable[RangeVector] = { + querySession: QuerySession): Observable[RangeVector] = { val taskOfResults = childResponses.map { case (QueryResult(_, _, result), i) => (result, i) case (QueryError(_, ex), _) => throw ex diff --git a/query/src/main/scala/filodb/query/exec/DistConcatExec.scala b/query/src/main/scala/filodb/query/exec/DistConcatExec.scala index cab9ebd719..d32169d621 100644 --- a/query/src/main/scala/filodb/query/exec/DistConcatExec.scala +++ b/query/src/main/scala/filodb/query/exec/DistConcatExec.scala @@ -18,7 +18,7 @@ final case class DistConcatExec(queryContext: QueryContext, protected def compose(childResponses: Observable[(QueryResponse, Int)], firstSchema: Task[ResultSchema], - queryConfig: QueryConfig): Observable[RangeVector] = { + querySession: QuerySession): Observable[RangeVector] = { childResponses.flatMap { case (QueryResult(_, _, result), _) => Observable.fromIterable(result) case (QueryError(_, ex), _) => throw ex diff --git a/query/src/main/scala/filodb/query/exec/EmptyResultExec.scala b/query/src/main/scala/filodb/query/exec/EmptyResultExec.scala index 44f91e8863..fe8103ed05 100644 --- a/query/src/main/scala/filodb/query/exec/EmptyResultExec.scala +++ b/query/src/main/scala/filodb/query/exec/EmptyResultExec.scala @@ -5,16 +5,16 @@ import monix.execution.Scheduler import filodb.core.DatasetRef import filodb.core.metadata.Column.ColumnType -import filodb.core.query.{ColumnInfo, QueryContext, ResultSchema} +import filodb.core.query.{ColumnInfo, QueryContext, QuerySession, ResultSchema} import filodb.core.store.ChunkSource -import filodb.query.{QueryConfig, QueryResponse, QueryResult} +import filodb.query.{QueryResponse, QueryResult} case class EmptyResultExec(queryContext: QueryContext, dataset: DatasetRef) extends LeafExecPlan { override def dispatcher: PlanDispatcher = InProcessPlanDispatcher override def execute(source: ChunkSource, - queryConfig: QueryConfig) + querySession: QuerySession) (implicit sched: Scheduler): Task[QueryResponse] = { Task(QueryResult(queryContext.queryId, new ResultSchema(Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), @@ -22,7 +22,8 @@ case class EmptyResultExec(queryContext: QueryContext, Seq.empty)) } - override def doExecute(source: ChunkSource, queryConfig: QueryConfig) + override def doExecute(source: ChunkSource, + querySession: QuerySession) (implicit sched: Scheduler): ExecResult = ??? override protected def args: String = "" diff --git a/query/src/main/scala/filodb/query/exec/ExecPlan.scala b/query/src/main/scala/filodb/query/exec/ExecPlan.scala index 4004f3a8e2..9ed7f309d8 100644 --- a/query/src/main/scala/filodb/query/exec/ExecPlan.scala +++ b/query/src/main/scala/filodb/query/exec/ExecPlan.scala @@ -92,7 +92,8 @@ trait ExecPlan extends QueryCommand { * */ // scalastyle:off method.length - def execute(source: ChunkSource, queryConfig: QueryConfig) + def execute(source: ChunkSource, + querySession: QuerySession) (implicit sched: Scheduler): Task[QueryResponse] = { val parentSpan = Kamon.currentSpan() @@ -110,7 +111,7 @@ trait ExecPlan extends QueryCommand { // 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) { - doExecute(source, queryConfig) + doExecute(source, querySession) } } @@ -132,7 +133,8 @@ trait ExecPlan extends QueryCommand { val finalRes = allTransformers.foldLeft((res.rvs, resSchema)) { (acc, transf) => span.mark(transf.getClass.getSimpleName) val paramRangeVector: Seq[Observable[ScalarRangeVector]] = transf.funcParams.map(_.getResult) - (transf.apply(acc._1, queryConfig, queryContext.sampleLimit, acc._2, paramRangeVector), transf.schema(acc._2)) + (transf.apply(acc._1, querySession, queryContext.sampleLimit, acc._2, + paramRangeVector), transf.schema(acc._2)) } val recSchema = SerializedRangeVector.toSchema(finalRes._2.columns, finalRes._2.brSchemas) val builder = SerializedRangeVector.newBuilder() @@ -204,7 +206,7 @@ trait ExecPlan extends QueryCommand { * Note that this should not include any operations done in the transformers. */ def doExecute(source: ChunkSource, - queryConfig: QueryConfig) + querySession: QuerySession) (implicit sched: Scheduler): ExecResult /** @@ -363,7 +365,7 @@ abstract class NonLeafExecPlan extends ExecPlan { * from the non-empty results. */ final def doExecute(source: ChunkSource, - queryConfig: QueryConfig) + querySession: QuerySession) (implicit sched: Scheduler): ExecResult = { val parentSpan = Kamon.currentSpan() parentSpan.mark("create-child-tasks") @@ -390,7 +392,7 @@ abstract class NonLeafExecPlan extends ExecPlan { case (QueryResult(_, schema, _), _) => schema }.firstOptionL.map(_.getOrElse(ResultSchema.empty)) parentSpan.mark("output-compose") - val outputRvs = compose(processedTasks, outputSchema, queryConfig) + val outputRvs = compose(processedTasks, outputSchema, querySession) parentSpan.mark("return-results") ExecResult(outputRvs, outputSchema) } @@ -422,6 +424,6 @@ abstract class NonLeafExecPlan extends ExecPlan { */ protected def compose(childResponses: Observable[(QueryResponse, Int)], firstSchema: Task[ResultSchema], - queryConfig: QueryConfig): Observable[RangeVector] + querySession: QuerySession): Observable[RangeVector] } diff --git a/query/src/main/scala/filodb/query/exec/HistogramQuantileMapper.scala b/query/src/main/scala/filodb/query/exec/HistogramQuantileMapper.scala index 424334a6d8..3386f36098 100644 --- a/query/src/main/scala/filodb/query/exec/HistogramQuantileMapper.scala +++ b/query/src/main/scala/filodb/query/exec/HistogramQuantileMapper.scala @@ -7,7 +7,6 @@ import scalaxy.loops._ import filodb.core.query._ import filodb.memory.format.{RowReader, ZeroCopyUTF8String} import filodb.memory.format.vectors.Histogram -import filodb.query.QueryConfig object HistogramQuantileMapper { import ZeroCopyUTF8String._ @@ -47,7 +46,7 @@ final case class HistogramQuantileMapper(funcParams: Seq[FuncArgs]) extends Rang * be preceded by a rate function or a sum-of-rate function. */ override def apply(source: Observable[RangeVector], - queryConfig: QueryConfig, + querySession: QuerySession, limit: Int, sourceSchema: ResultSchema, paramResponse: Seq[Observable[ScalarRangeVector]]): Observable[RangeVector] = { diff --git a/query/src/main/scala/filodb/query/exec/InProcessPlanDispatcher.scala b/query/src/main/scala/filodb/query/exec/InProcessPlanDispatcher.scala index 5443360877..14e794a786 100644 --- a/query/src/main/scala/filodb/query/exec/InProcessPlanDispatcher.scala +++ b/query/src/main/scala/filodb/query/exec/InProcessPlanDispatcher.scala @@ -5,11 +5,12 @@ import monix.eval.Task import monix.execution.Scheduler import monix.reactive.Observable -import filodb.core.DatasetRef +import filodb.core.{DatasetRef, Types} import filodb.core.memstore.PartLookupResult import filodb.core.metadata.Schemas +import filodb.core.query.{EmptyQueryConfig, QueryConfig, QuerySession} import filodb.core.store._ -import filodb.query.{EmptyQueryConfig, QueryConfig, QueryResponse} +import filodb.query.QueryResponse /** * Dispatcher which will make a No-Op style call to ExecPlan#excecute(). @@ -30,7 +31,8 @@ case object InProcessPlanDispatcher extends PlanDispatcher { // kamon uses thread-locals. Kamon.runWithSpan(Kamon.currentSpan(), false) { // translate implicit ExecutionContext to monix.Scheduler - plan.execute(source, queryConfig) + val querySession = QuerySession(plan.queryContext, queryConfig) + plan.execute(source, querySession) } } @@ -41,12 +43,15 @@ case object InProcessPlanDispatcher extends PlanDispatcher { */ case class UnsupportedChunkSource() extends ChunkSource { def scanPartitions(ref: DatasetRef, - iter: PartLookupResult): Observable[ReadablePartition] = + iter: PartLookupResult, + colIds: Seq[Types.ColumnId], + querySession: QuerySession): Observable[ReadablePartition] = throw new UnsupportedOperationException("This operation is not supported") def lookupPartitions(ref: DatasetRef, partMethod: PartitionScanMethod, - chunkMethod: ChunkScanMethod): PartLookupResult = + chunkMethod: ChunkScanMethod, + querySession: QuerySession): PartLookupResult = throw new UnsupportedOperationException("This operation is not supported") override def groupsInDataset(dataset: DatasetRef): Int = diff --git a/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala b/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala index 03a369449d..b5d75b4fd6 100644 --- a/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala +++ b/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala @@ -36,7 +36,7 @@ final case class PartKeysDistConcatExec(queryContext: QueryContext, */ protected def compose(childResponses: Observable[(QueryResponse, Int)], firstSchema: Task[ResultSchema], - queryConfig: QueryConfig): Observable[RangeVector] = { + querySession: QuerySession): Observable[RangeVector] = { qLogger.debug(s"NonLeafMetadataExecPlan: Concatenating results") val taskOfResults = childResponses.map { case (QueryResult(_, _, result), _) => result @@ -68,7 +68,7 @@ final case class LabelValuesDistConcatExec(queryContext: QueryContext, */ protected def compose(childResponses: Observable[(QueryResponse, Int)], firstSchema: Task[ResultSchema], - queryConfig: QueryConfig): Observable[RangeVector] = { + querySession: QuerySession): Observable[RangeVector] = { qLogger.debug(s"NonLeafMetadataExecPlan: Concatenating results") val taskOfResults = childResponses.map { case (QueryResult(_, _, result), _) => result @@ -108,7 +108,7 @@ final case class PartKeysExec(queryContext: QueryContext, override def enforceLimit: Boolean = false def doExecute(source: ChunkSource, - queryConfig: QueryConfig) + querySession: QuerySession) (implicit sched: Scheduler): ExecResult = { val rvs = source match { case memStore: MemStore => @@ -140,7 +140,7 @@ final case class LabelValuesExec(queryContext: QueryContext, override def enforceLimit: Boolean = false def doExecute(source: ChunkSource, - queryConfig: QueryConfig) + querySession: QuerySession) (implicit sched: Scheduler): ExecResult = { val parentSpan = Kamon.currentSpan() val rvs = if (source.isInstanceOf[MemStore]) { diff --git a/query/src/main/scala/filodb/query/exec/MultiSchemaPartitionsExec.scala b/query/src/main/scala/filodb/query/exec/MultiSchemaPartitionsExec.scala index 3d2228501f..5b5fb12be0 100644 --- a/query/src/main/scala/filodb/query/exec/MultiSchemaPartitionsExec.scala +++ b/query/src/main/scala/filodb/query/exec/MultiSchemaPartitionsExec.scala @@ -5,10 +5,9 @@ import monix.execution.Scheduler import filodb.core.{DatasetRef, QueryTimeoutException} import filodb.core.metadata.Schemas -import filodb.core.query.{ColumnFilter, QueryContext} +import filodb.core.query.{ColumnFilter, QueryContext, QuerySession} import filodb.core.store._ import filodb.query.Query.qLogger -import filodb.query.QueryConfig final case class UnknownSchemaQueryErr(id: Int) extends Exception(s"Unknown schema ID $id during query. This likely means a schema config change happened and " + @@ -39,10 +38,11 @@ final case class MultiSchemaPartitionsExec(queryContext: QueryContext, var finalPlan: ExecPlan = _ - private def finalizePlan(source: ChunkSource): ExecPlan = { + private def finalizePlan(source: ChunkSource, + querySession: QuerySession): ExecPlan = { val partMethod = FilteredPartitionScan(ShardSplit(shard), filters) Kamon.currentSpan().mark("filtered-partition-scan") - val lookupRes = source.lookupPartitions(dataset, partMethod, chunkMethod) + val lookupRes = source.lookupPartitions(dataset, partMethod, chunkMethod, querySession) Kamon.currentSpan().mark("lookup-partitions-done") val queryTimeElapsed = System.currentTimeMillis() - queryContext.submitTime @@ -83,14 +83,14 @@ final case class MultiSchemaPartitionsExec(queryContext: QueryContext, } def doExecute(source: ChunkSource, - queryConfig: QueryConfig) + querySession: QuerySession) (implicit sched: Scheduler): ExecResult = { - finalPlan = finalizePlan(source) - finalPlan.doExecute(source, queryConfig)(sched) + finalPlan = finalizePlan(source, querySession) + finalPlan.doExecute(source, querySession)(sched) } protected def args: String = s"dataset=$dataset, shard=$shard, " + - s"chunkMethod=$chunkMethod, filters=$filters, colName=$colName" + s"chunkMethod=$chunkMethod, filters=$filters, colName=$colName, schema=$schema" // Print inner node's details for debugging override def curNodeText(level: Int): String = { diff --git a/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala b/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala index 48eafbfdbf..cc0a86cc59 100644 --- a/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala +++ b/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala @@ -48,18 +48,18 @@ final case class PeriodicSamplesMapper(start: Long, //scalastyle:off method.length def apply(source: Observable[RangeVector], - queryConfig: QueryConfig, + querySession: QuerySession, limit: Int, sourceSchema: ResultSchema, paramResponse: Seq[Observable[ScalarRangeVector]]): Observable[RangeVector] = { // enforcement of minimum step is good since we have a high limit on number of samples - if (step < queryConfig.minStepMs) - throw new BadQueryException(s"step should be at least ${queryConfig.minStepMs/1000}s") + if (step < querySession.queryConfig.minStepMs) + throw new BadQueryException(s"step should be at least ${querySession.queryConfig.minStepMs/1000}s") val valColType = RangeVectorTransformer.valueColumnType(sourceSchema) // If a max column is present, the ExecPlan's job is to put it into column 2 val hasMaxCol = valColType == ColumnType.HistogramColumn && sourceSchema.colIDs.length > 2 && sourceSchema.columns(2).name == "max" - val rangeFuncGen = RangeFunction.generatorFor(sourceSchema, functionId, valColType, queryConfig, + val rangeFuncGen = RangeFunction.generatorFor(sourceSchema, functionId, valColType, querySession.queryConfig, funcParams, rawSource) // Generate one range function to check if it is chunked @@ -67,7 +67,7 @@ final case class PeriodicSamplesMapper(start: Long, // Really, use the stale lookback window size, not 0 which doesn't make sense // Default value for window should be queryConfig.staleSampleAfterMs + 1 for empty functionId, // so that it returns value present at time - staleSampleAfterMs - val windowLength = window.getOrElse(if (isLastFn) queryConfig.staleSampleAfterMs + 1 else 0L) + val windowLength = window.getOrElse(if (isLastFn) querySession.queryConfig.staleSampleAfterMs + 1 else 0L) val rvs = sampleRangeFunc match { case c: ChunkedRangeFunction[_] if valColType == ColumnType.HistogramColumn => @@ -75,28 +75,28 @@ final case class PeriodicSamplesMapper(start: Long, val histRow = if (hasMaxCol) new TransientHistMaxRow() else new TransientHistRow() IteratorBackedRangeVector(rv.key, new ChunkedWindowIteratorH(rv.asInstanceOf[RawDataRangeVector], startWithOffset, step, endWithOffset, - windowLength, rangeFuncGen().asChunkedH, queryConfig, queryContext, histRow)) + windowLength, rangeFuncGen().asChunkedH, querySession, histRow)) } case c: ChunkedRangeFunction[_] => source.map { rv => qLogger.trace(s"Creating ChunkedWindowIterator for rv=${rv.key}, step=$step windowLength=$windowLength") IteratorBackedRangeVector(rv.key, new ChunkedWindowIteratorD(rv.asInstanceOf[RawDataRangeVector], startWithOffset, step, endWithOffset, - windowLength, rangeFuncGen().asChunkedD, queryConfig, queryContext)) + windowLength, rangeFuncGen().asChunkedD, querySession)) } // Iterator-based: Wrap long columns to yield a double value case f: RangeFunction if valColType == ColumnType.LongColumn => source.map { rv => IteratorBackedRangeVector(rv.key, new SlidingWindowIterator(new LongToDoubleIterator(rv.rows), startWithOffset, step, endWithOffset, - window.getOrElse(0L), rangeFuncGen().asSliding, queryConfig)) + window.getOrElse(0L), rangeFuncGen().asSliding, querySession.queryConfig)) } // Otherwise just feed in the double column case f: RangeFunction => source.map { rv => IteratorBackedRangeVector(rv.key, new SlidingWindowIterator(rv.rows, startWithOffset, step, endWithOffset, window.getOrElse(0L), - rangeFuncGen().asSliding, queryConfig)) + rangeFuncGen().asSliding, querySession.queryConfig)) } } @@ -157,13 +157,12 @@ abstract class ChunkedWindowIterator[R <: MutableRowReader]( end: Long, window: Long, rangeFunction: ChunkedRangeFunction[R], - queryConfig: QueryConfig, - queryContext: QueryContext) + querySession: QuerySession) extends Iterator[R] with StrictLogging { // Lazily open the iterator and obtain the lock. This allows one thread to create the // iterator, but the lock is owned by the thread actually performing the iteration. private lazy val windowIt = { - val it = new WindowedChunkIterator(rv, start, step, end, window, queryContext) + val it = new WindowedChunkIterator(rv, start, step, end, window, querySession.qContext) // Need to hold the shared lock explicitly, because the window iterator needs to // pre-fetch chunks to determine the window. This pre-fetching can force the internal // iterator to close, which would release the lock too soon. @@ -187,7 +186,7 @@ extends Iterator[R] with StrictLogging { try { rangeFunction.addChunks(nextInfo.getTsVectorAccessor, nextInfo.getTsVectorAddr, nextInfo.getTsReader, nextInfo.getValueVectorAccessor, nextInfo.getValueVectorAddr, nextInfo.getValueReader, - wit.curWindowStart, wit.curWindowEnd, nextInfo, queryConfig) + wit.curWindowStart, wit.curWindowEnd, nextInfo, querySession.queryConfig) } catch { case e: Exception => val tsReader = LongBinaryVector(nextInfo.getTsVectorAccessor, nextInfo.getTsVectorAddr) @@ -216,20 +215,18 @@ extends Iterator[R] with StrictLogging { class ChunkedWindowIteratorD(rv: RawDataRangeVector, start: Long, step: Long, end: Long, window: Long, rangeFunction: ChunkedRangeFunction[TransientRow], - queryConfig: QueryConfig, - queryContext: QueryContext = QueryContext(), + querySession: QuerySession, // put emitter here in constructor for faster access var sampleToEmit: TransientRow = new TransientRow()) extends -ChunkedWindowIterator[TransientRow](rv, start, step, end, window, rangeFunction, queryConfig, queryContext) +ChunkedWindowIterator[TransientRow](rv, start, step, end, window, rangeFunction, querySession) class ChunkedWindowIteratorH(rv: RawDataRangeVector, start: Long, step: Long, end: Long, window: Long, rangeFunction: ChunkedRangeFunction[TransientHistRow], - queryConfig: QueryConfig, - queryContext: QueryContext = QueryContext(), + querySession: QuerySession, // put emitter here in constructor for faster access var sampleToEmit: TransientHistRow = new TransientHistRow()) extends -ChunkedWindowIterator[TransientHistRow](rv, start, step, end, window, rangeFunction, queryConfig, queryContext) +ChunkedWindowIterator[TransientHistRow](rv, start, step, end, window, rangeFunction, querySession) class QueueBasedWindow(q: IndexedArrayQueue[TransientRow]) extends Window { def size: Int = q.size diff --git a/query/src/main/scala/filodb/query/exec/PlanDispatcher.scala b/query/src/main/scala/filodb/query/exec/PlanDispatcher.scala index 263c17eaf7..9036959e19 100644 --- a/query/src/main/scala/filodb/query/exec/PlanDispatcher.scala +++ b/query/src/main/scala/filodb/query/exec/PlanDispatcher.scala @@ -29,16 +29,18 @@ trait PlanDispatcher extends java.io.Serializable { 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 queryTimeElapsed = System.currentTimeMillis() - plan.queryContext.submitTime + val remainingTime = plan.queryContext.queryTimeoutMillis - queryTimeElapsed // Don't send if time left is very small - if (remainingTime < 1) throw QueryTimeoutException(remainingTime, this.getClass.getName) - val t = Timeout(FiniteDuration(remainingTime, TimeUnit.MILLISECONDS)) - val fut = (target ? plan)(t).map { - case resp: QueryResponse => resp - case e => throw new IllegalStateException(s"Received bad response $e") + if (remainingTime < 1) { + Task.raiseError(QueryTimeoutException(remainingTime, this.getClass.getName)) + } else { + val t = Timeout(FiniteDuration(remainingTime, TimeUnit.MILLISECONDS)) + val fut = (target ? plan)(t).map { + case resp: QueryResponse => resp + case e => throw new IllegalStateException(s"Received bad response $e") + } + Task.fromFuture(fut) } - Task.fromFuture(fut) } } diff --git a/query/src/main/scala/filodb/query/exec/PromQlExec.scala b/query/src/main/scala/filodb/query/exec/PromQlExec.scala index c42834abe0..af2a120ddd 100644 --- a/query/src/main/scala/filodb/query/exec/PromQlExec.scala +++ b/query/src/main/scala/filodb/query/exec/PromQlExec.scala @@ -37,11 +37,12 @@ case class PromQlExec(queryContext: QueryContext, * implementation of the operation represented by this exec plan * node */ - def doExecute(source: ChunkSource, queryConfig: QueryConfig) + def doExecute(source: ChunkSource, + querySession: QuerySession) (implicit sched: Scheduler): ExecResult = ??? override def execute(source: ChunkSource, - queryConfig: QueryConfig) + querySession: QuerySession) (implicit sched: Scheduler): Task[QueryResponse] = { val execPlan2Span = Kamon.spanBuilder(s"execute-${getClass.getSimpleName}") .asChildOf(Kamon.currentSpan()) diff --git a/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala b/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala index 1d1dfd41c4..e68f1ae758 100644 --- a/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala +++ b/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala @@ -11,7 +11,7 @@ import filodb.core.query.Filter.Equals import filodb.memory.format.{RowReader, ZeroCopyUTF8String} import filodb.memory.format.vectors.{HistogramBuckets, HistogramWithBuckets} import filodb.query._ -import filodb.query.{BinaryOperator, InstantFunctionId, MiscellaneousFunctionId, QueryConfig, SortFunctionId} +import filodb.query.{BinaryOperator, InstantFunctionId, MiscellaneousFunctionId, SortFunctionId} import filodb.query.InstantFunctionId.HistogramQuantile import filodb.query.MiscellaneousFunctionId.{LabelJoin, LabelReplace} import filodb.query.ScalarFunctionId.Scalar @@ -36,7 +36,7 @@ trait RangeVectorTransformer extends java.io.Serializable { def funcParams: Seq[FuncArgs] def apply(source: Observable[RangeVector], - queryConfig: QueryConfig, + querySession: QuerySession, limit: Int, sourceSchema: ResultSchema, paramsResponse: Seq[Observable[ScalarRangeVector]]): Observable[RangeVector] @@ -69,7 +69,7 @@ final case class InstantVectorFunctionMapper(function: InstantFunctionId, funcParams: Seq[FuncArgs] = Nil) extends RangeVectorTransformer { protected[exec] def args: String = s"function=$function" - def evaluate(source: Observable[RangeVector], scalarRangeVector: Seq[ScalarRangeVector], queryConfig: QueryConfig, + def evaluate(source: Observable[RangeVector], scalarRangeVector: Seq[ScalarRangeVector], querySession: QuerySession, limit: Int, sourceSchema: ResultSchema) : Observable[RangeVector] = { RangeVectorTransformer.valueColumnType(sourceSchema) match { case ColumnType.HistogramColumn => @@ -91,7 +91,7 @@ final case class InstantVectorFunctionMapper(function: InstantFunctionId, if (function == HistogramQuantile) { // Special mapper to pull all buckets together from different Prom-schema time series val mapper = HistogramQuantileMapper(funcParams) - mapper.apply(source, queryConfig, limit, sourceSchema, Nil) + mapper.apply(source, querySession, limit, sourceSchema, Nil) } else { val instantFunction = InstantFunction.double(function) source.map { rv => @@ -105,21 +105,21 @@ final case class InstantVectorFunctionMapper(function: InstantFunctionId, } def apply(source: Observable[RangeVector], - queryConfig: QueryConfig, + querySession: QuerySession, limit: Int, sourceSchema: ResultSchema, paramResponse: Seq[Observable[ScalarRangeVector]]): Observable[RangeVector] = { if (funcParams.isEmpty) { - evaluate(source, Nil, queryConfig, limit, sourceSchema) + evaluate(source, Nil, querySession, limit, sourceSchema) } else { // Multiple ExecPlanFunArgs not supported yet funcParams.head match { case s: StaticFuncArgs => evaluate(source, funcParams.map(x => x.asInstanceOf[StaticFuncArgs]). - map(x => ScalarFixedDouble(x.timeStepParams, x.scalar)), queryConfig, limit, + map(x => ScalarFixedDouble(x.timeStepParams, x.scalar)), querySession, limit, sourceSchema) case t: TimeFuncArgs => evaluate(source, funcParams.map(x => x.asInstanceOf[TimeFuncArgs]). - map(x => TimeScalar(x.timeStepParams)), queryConfig, limit, sourceSchema) + map(x => TimeScalar(x.timeStepParams)), querySession, limit, sourceSchema) case e: ExecPlanFuncArgs => paramResponse.head.map { param => - evaluate(source, Seq(param), queryConfig, limit, sourceSchema) + evaluate(source, Seq(param), querySession, limit, sourceSchema) }.flatten case _ => throw new IllegalArgumentException(s"Invalid function param") } @@ -198,7 +198,7 @@ final case class ScalarOperationMapper(operator: BinaryOperator, val operatorFunction = BinaryOperatorFunction.factoryMethod(operator) def apply(source: Observable[RangeVector], - queryConfig: QueryConfig, + querySession: QuerySession, limit: Int, sourceSchema: ResultSchema, paramResponse: Seq[Observable[ScalarRangeVector]] = Nil): Observable[RangeVector] = { @@ -250,7 +250,7 @@ final case class MiscellaneousFunctionMapper(function: MiscellaneousFunctionId, } def apply(source: Observable[RangeVector], - queryConfig: QueryConfig, + querySession: QuerySession, limit: Int, sourceSchema: ResultSchema, paramResponse: Seq[Observable[ScalarRangeVector]]): Observable[RangeVector] = { @@ -262,7 +262,7 @@ final case class SortFunctionMapper(function: SortFunctionId) extends RangeVecto protected[exec] def args: String = s"function=$function" def apply(source: Observable[RangeVector], - queryConfig: QueryConfig, + querySession: QuerySession, limit: Int, sourceSchema: ResultSchema, paramResponse: Seq[Observable[ScalarRangeVector]]): Observable[RangeVector] = { @@ -309,7 +309,7 @@ final case class ScalarFunctionMapper(function: ScalarFunctionId, Observable.fromTask(resultRv).flatten } def apply(source: Observable[RangeVector], - queryConfig: QueryConfig, + querySession: QuerySession, limit: Int, sourceSchema: ResultSchema, paramResponse: Seq[Observable[ScalarRangeVector]]): Observable[RangeVector] = { @@ -327,7 +327,7 @@ final case class VectorFunctionMapper() extends RangeVectorTransformer { protected[exec] def args: String = s"funcParams=$funcParams" def apply(source: Observable[RangeVector], - queryConfig: QueryConfig, + querySession: QuerySession, limit: Int, sourceSchema: ResultSchema, paramResponse: Seq[Observable[ScalarRangeVector]]): Observable[RangeVector] = { @@ -356,7 +356,7 @@ final case class AbsentFunctionMapper(columnFilter: Seq[ColumnFilter], rangePara } def apply(source: Observable[RangeVector], - queryConfig: QueryConfig, + querySession: QuerySession, limit: Int, sourceSchema: ResultSchema, paramResponse: Seq[Observable[ScalarRangeVector]]): Observable[RangeVector] = { @@ -414,7 +414,7 @@ final case class HistToPromSeriesMapper(sch: PartitionSchema) extends RangeVecto // NOTE: apply() is only called for explicit instantiation of conversion function. So this will error out if // the data source is not histogram. def apply(source: Observable[RangeVector], - queryConfig: QueryConfig, + querySession: QuerySession, limit: Int, sourceSchema: ResultSchema, paramResponse: Seq[Observable[ScalarRangeVector]]): Observable[RangeVector] = { diff --git a/query/src/main/scala/filodb/query/exec/ScalarBinaryOperationExec.scala b/query/src/main/scala/filodb/query/exec/ScalarBinaryOperationExec.scala index 8a0e590297..012f2c92d1 100644 --- a/query/src/main/scala/filodb/query/exec/ScalarBinaryOperationExec.scala +++ b/query/src/main/scala/filodb/query/exec/ScalarBinaryOperationExec.scala @@ -8,7 +8,7 @@ import filodb.core.DatasetRef import filodb.core.metadata.Column.ColumnType import filodb.core.query._ import filodb.core.store.ChunkSource -import filodb.query.{BinaryOperator, QueryConfig, QueryResponse, QueryResult} +import filodb.query.{BinaryOperator, QueryResponse, QueryResult} import filodb.query.exec.binaryOp.BinaryOperatorFunction /** @@ -47,14 +47,15 @@ case class ScalarBinaryOperationExec(queryContext: QueryContext, * implementation of the operation represented by this exec plan * node */ - override def doExecute(source: ChunkSource, queryConfig: QueryConfig) + override def doExecute(source: ChunkSource, + querySession: QuerySession) (implicit sched: Scheduler): ExecResult = { throw new IllegalStateException("doExecute should not be called for ScalarBinaryOperationExec since it represents" + "a static value") } override def execute(source: ChunkSource, - queryConfig: QueryConfig) + querySession: QuerySession) (implicit sched: Scheduler): Task[QueryResponse] = { val execPlan2Span = Kamon.spanBuilder(s"execute-${getClass.getSimpleName}") .asChildOf(Kamon.currentSpan()) diff --git a/query/src/main/scala/filodb/query/exec/ScalarFixedDoubleExec.scala b/query/src/main/scala/filodb/query/exec/ScalarFixedDoubleExec.scala index 4c18481d30..a6dd0fa3c9 100644 --- a/query/src/main/scala/filodb/query/exec/ScalarFixedDoubleExec.scala +++ b/query/src/main/scala/filodb/query/exec/ScalarFixedDoubleExec.scala @@ -9,7 +9,7 @@ import filodb.core.DatasetRef import filodb.core.metadata.Column.ColumnType import filodb.core.query._ import filodb.core.store.ChunkSource -import filodb.query.{QueryConfig, QueryResponse, QueryResult} +import filodb.query.{QueryResponse, QueryResult} /** @@ -29,7 +29,8 @@ case class ScalarFixedDoubleExec(queryContext: QueryContext, * implementation of the operation represented by this exec plan * node */ - override def doExecute(source: ChunkSource, queryConfig: QueryConfig) + override def doExecute(source: ChunkSource, + querySession: QuerySession) (implicit sched: Scheduler): ExecResult = { throw new IllegalStateException("doExecute should not be called for ScalarFixedDoubleExec since it represents a " + "readily available static value") @@ -43,7 +44,7 @@ case class ScalarFixedDoubleExec(queryContext: QueryContext, override def execute(source: ChunkSource, - queryConfig: QueryConfig) + querySession: QuerySession) (implicit sched: Scheduler): Task[QueryResponse] = { val execPlan2Span = Kamon.spanBuilder(s"execute-${getClass.getSimpleName}") .asChildOf(Kamon.currentSpan()) @@ -62,7 +63,8 @@ case class ScalarFixedDoubleExec(queryContext: QueryContext, .start() rangeVectorTransformers.foldLeft((Observable.fromIterable(rangeVectors), resultSchema)) { (acc, transf) => val paramRangeVector: Seq[Observable[ScalarRangeVector]] = transf.funcParams.map(_.getResult) - (transf.apply(acc._1, queryConfig, queryContext.sampleLimit, acc._2, paramRangeVector), transf.schema(acc._2)) + (transf.apply(acc._1, querySession, queryContext.sampleLimit, acc._2, + paramRangeVector), transf.schema(acc._2)) }._1.toListL.map({ span.finish() QueryResult(queryContext.queryId, resultSchema, _) diff --git a/query/src/main/scala/filodb/query/exec/SelectChunkInfosExec.scala b/query/src/main/scala/filodb/query/exec/SelectChunkInfosExec.scala index 1b067954e9..915624457d 100644 --- a/query/src/main/scala/filodb/query/exec/SelectChunkInfosExec.scala +++ b/query/src/main/scala/filodb/query/exec/SelectChunkInfosExec.scala @@ -9,7 +9,6 @@ import filodb.core.memstore.TimeSeriesShard import filodb.core.metadata.Column import filodb.core.query._ import filodb.core.store._ -import filodb.query.QueryConfig object SelectChunkInfosExec { import Column.ColumnType._ @@ -42,10 +41,10 @@ final case class SelectChunkInfosExec(queryContext: QueryContext, import SelectChunkInfosExec._ def doExecute(source: ChunkSource, - queryConfig: QueryConfig) + querySession: QuerySession) (implicit sched: Scheduler): ExecResult = { val partMethod = FilteredPartitionScan(ShardSplit(shard), filters) - val lookupRes = source.lookupPartitions(dataset, partMethod, chunkMethod) + val lookupRes = source.lookupPartitions(dataset, partMethod, chunkMethod, querySession) val schemas = source.schemas(dataset).get val dataSchema = schema.map { s => schemas.schemas(s) } @@ -55,7 +54,7 @@ final case class SelectChunkInfosExec(queryContext: QueryContext, val partCols = dataSchema.partitionInfos val numGroups = source.groupsInDataset(dataset) Kamon.currentSpan().mark("creating-scanpartitions") - val rvs = source.scanPartitions(dataset, lookupRes) + val rvs = source.scanPartitions(dataset, lookupRes, Seq.empty, querySession) .filter(_.hasChunks(chunkMethod)) .map { partition => source.stats.incrReadPartitions(1) diff --git a/query/src/main/scala/filodb/query/exec/SelectRawPartitionsExec.scala b/query/src/main/scala/filodb/query/exec/SelectRawPartitionsExec.scala index 6afbc59c05..a18a668baa 100644 --- a/query/src/main/scala/filodb/query/exec/SelectRawPartitionsExec.scala +++ b/query/src/main/scala/filodb/query/exec/SelectRawPartitionsExec.scala @@ -8,9 +8,9 @@ import monix.reactive.Observable import filodb.core.{DatasetRef, Types} import filodb.core.memstore.PartLookupResult import filodb.core.metadata.{Column, Schema, Schemas} -import filodb.core.query.{QueryContext, ResultSchema} +import filodb.core.query.{QueryContext, QuerySession, ResultSchema} import filodb.core.store._ -import filodb.query.{Query, QueryConfig} +import filodb.query.Query import filodb.query.Query.qLogger import filodb.query.exec.rangefn.RangeFunction @@ -122,7 +122,7 @@ final case class SelectRawPartitionsExec(queryContext: QueryContext, } def doExecute(source: ChunkSource, - queryConfig: QueryConfig) + querySession: QuerySession) (implicit sched: Scheduler): ExecResult = { val span = Kamon.spanBuilder(s"execute-${getClass.getSimpleName}") .asChildOf(Kamon.currentSpan()) @@ -131,7 +131,7 @@ final case class SelectRawPartitionsExec(queryContext: QueryContext, 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) + 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/SetOperatorExec.scala b/query/src/main/scala/filodb/query/exec/SetOperatorExec.scala index 3f72cf7067..9735d60cc6 100644 --- a/query/src/main/scala/filodb/query/exec/SetOperatorExec.scala +++ b/query/src/main/scala/filodb/query/exec/SetOperatorExec.scala @@ -51,7 +51,7 @@ final case class SetOperatorExec(queryContext: QueryContext, protected[exec] def compose(childResponses: Observable[(QueryResponse, Int)], firstSchema: Task[ResultSchema], - queryConfig: QueryConfig): Observable[RangeVector] = { + querySession: QuerySession): Observable[RangeVector] = { val taskOfResults = childResponses.map { case (QueryResult(_, _, result), i) => (result, i) case (QueryError(_, ex), _) => throw ex diff --git a/query/src/main/scala/filodb/query/exec/StitchRvsExec.scala b/query/src/main/scala/filodb/query/exec/StitchRvsExec.scala index 7cad5555b5..f68912e911 100644 --- a/query/src/main/scala/filodb/query/exec/StitchRvsExec.scala +++ b/query/src/main/scala/filodb/query/exec/StitchRvsExec.scala @@ -66,7 +66,7 @@ final case class StitchRvsExec(queryContext: QueryContext, protected def compose(childResponses: Observable[(QueryResponse, Int)], firstSchema: Task[ResultSchema], - queryConfig: QueryConfig): Observable[RangeVector] = { + querySession: QuerySession): Observable[RangeVector] = { qLogger.debug(s"StitchRvsExec: Stitching results:") val stitched = childResponses.map { case (QueryResult(_, _, result), _) => result @@ -103,7 +103,7 @@ final case class StitchRvsExec(queryContext: QueryContext, final case class StitchRvsMapper() extends RangeVectorTransformer { def apply(source: Observable[RangeVector], - queryConfig: QueryConfig, + querySession: QuerySession, limit: Int, sourceSchema: ResultSchema, paramResponse: Seq[Observable[ScalarRangeVector]]): Observable[RangeVector] = { qLogger.debug(s"StitchRvsMapper: Stitching results:") diff --git a/query/src/main/scala/filodb/query/exec/TimeScalarGeneratorExec.scala b/query/src/main/scala/filodb/query/exec/TimeScalarGeneratorExec.scala index f9c344de55..49d9212385 100644 --- a/query/src/main/scala/filodb/query/exec/TimeScalarGeneratorExec.scala +++ b/query/src/main/scala/filodb/query/exec/TimeScalarGeneratorExec.scala @@ -9,7 +9,7 @@ import filodb.core.DatasetRef import filodb.core.metadata.Column.ColumnType import filodb.core.query._ import filodb.core.store.ChunkSource -import filodb.query.{BadQueryException, QueryConfig, QueryResponse, QueryResult, ScalarFunctionId} +import filodb.query.{BadQueryException, QueryResponse, QueryResult, ScalarFunctionId} import filodb.query.ScalarFunctionId.{DayOfMonth, DayOfWeek, DaysInMonth, Hour, Minute, Month, Time, Year} /** @@ -28,7 +28,8 @@ case class TimeScalarGeneratorExec(queryContext: QueryContext, * implementation of the operation represented by this exec plan * node */ - override def doExecute(source: ChunkSource, queryConfig: QueryConfig) + override def doExecute(source: ChunkSource, + querySession: QuerySession) (implicit sched: Scheduler): ExecResult = { throw new IllegalStateException("doExecute should not be called for TimeScalarGeneratorExec since it represents" + "a readily available static value") @@ -41,7 +42,7 @@ case class TimeScalarGeneratorExec(queryContext: QueryContext, override protected def args: String = s"params = $params, function = $function" override def execute(source: ChunkSource, - queryConfig: QueryConfig) + querySession: QuerySession) (implicit sched: Scheduler): Task[QueryResponse] = { val execPlan2Span = Kamon.spanBuilder(s"execute-${getClass.getSimpleName}") .asChildOf(Kamon.currentSpan()) @@ -71,7 +72,8 @@ case class TimeScalarGeneratorExec(queryContext: QueryContext, rangeVectorTransformers.foldLeft((Observable.fromIterable(rangeVectors), resultSchema)) { (acc, transf) => span.mark(transf.getClass.getSimpleName) val paramRangeVector: Seq[Observable[ScalarRangeVector]] = transf.funcParams.map(_.getResult) - (transf.apply(acc._1, queryConfig, queryContext.sampleLimit, acc._2, paramRangeVector), transf.schema(acc._2)) + (transf.apply(acc._1, querySession, queryContext.sampleLimit, acc._2, + paramRangeVector), transf.schema(acc._2)) }._1.toListL.map({ span.finish() QueryResult(queryContext.queryId, resultSchema, _) diff --git a/query/src/main/scala/filodb/query/exec/rangefn/AggrOverTimeFunctions.scala b/query/src/main/scala/filodb/query/exec/rangefn/AggrOverTimeFunctions.scala index 994d3fbcc8..434d05efad 100644 --- a/query/src/main/scala/filodb/query/exec/rangefn/AggrOverTimeFunctions.scala +++ b/query/src/main/scala/filodb/query/exec/rangefn/AggrOverTimeFunctions.scala @@ -5,12 +5,11 @@ import java.lang.{Double => JLDouble} import debox.Buffer import java.util -import filodb.core.query.{TransientHistMaxRow, TransientHistRow, TransientRow} +import filodb.core.query.{QueryConfig, TransientHistMaxRow, TransientHistRow, TransientRow} import filodb.core.store.ChunkSetInfoReader import filodb.memory.format.{BinaryVector, MemoryReader, VectorDataReader} import filodb.memory.format.{vectors => bv} import filodb.memory.format.vectors.DoubleIterator -import filodb.query.QueryConfig import filodb.query.exec.{FuncArgs, StaticFuncArgs} class MinMaxOverTimeFunction(ord: Ordering[Double]) extends RangeFunction { 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 cf4b59cf47..356c03195c 100644 --- a/query/src/main/scala/filodb/query/exec/rangefn/RangeFunction.scala +++ b/query/src/main/scala/filodb/query/exec/rangefn/RangeFunction.scala @@ -2,11 +2,10 @@ package filodb.query.exec.rangefn import filodb.core.metadata.Column.ColumnType import filodb.core.metadata.Schema -import filodb.core.query.{MutableRowReader, ResultSchema, TransientHistMaxRow, TransientHistRow, TransientRow} +import filodb.core.query._ import filodb.core.store.ChunkSetInfoReader import filodb.memory.format.{vectors => bv, _} import filodb.memory.format.BinaryVector.BinaryVectorPtr -import filodb.query.QueryConfig import filodb.query.exec._ /** diff --git a/query/src/main/scala/filodb/query/exec/rangefn/RangeInstantFunctions.scala b/query/src/main/scala/filodb/query/exec/rangefn/RangeInstantFunctions.scala index e4453904df..3675029130 100644 --- a/query/src/main/scala/filodb/query/exec/rangefn/RangeInstantFunctions.scala +++ b/query/src/main/scala/filodb/query/exec/rangefn/RangeInstantFunctions.scala @@ -1,7 +1,6 @@ package filodb.query.exec.rangefn -import filodb.core.query.TransientRow -import filodb.query.QueryConfig +import filodb.core.query.{QueryConfig, TransientRow} object RangeInstantFunctions { def derivFunction(window: Window): Double = { diff --git a/query/src/main/scala/filodb/query/exec/rangefn/RateFunctions.scala b/query/src/main/scala/filodb/query/exec/rangefn/RateFunctions.scala index fd60a19361..417f653158 100644 --- a/query/src/main/scala/filodb/query/exec/rangefn/RateFunctions.scala +++ b/query/src/main/scala/filodb/query/exec/rangefn/RateFunctions.scala @@ -2,11 +2,10 @@ package filodb.query.exec.rangefn import scalaxy.loops._ -import filodb.core.query.{TransientHistRow, TransientRow} +import filodb.core.query.{QueryConfig, TransientHistRow, TransientRow} import filodb.memory.format.{CounterVectorReader, MemoryReader} import filodb.memory.format.{vectors => bv} import filodb.memory.format.BinaryVector.BinaryVectorPtr -import filodb.query.QueryConfig object RateFunctions { diff --git a/query/src/main/scala/filodb/query/util/LogicalPlanUtil.scala b/query/src/main/scala/filodb/query/util/LogicalPlanUtil.scala deleted file mode 100644 index a64dc70af8..0000000000 --- a/query/src/main/scala/filodb/query/util/LogicalPlanUtil.scala +++ /dev/null @@ -1,37 +0,0 @@ -package filodb.query.util - -import filodb.core.query.ColumnFilter -import filodb.query._ - -object LogicalPlanUtil { - - private def getLabelValueFromFilters(filters: Seq[ColumnFilter], labelName: String): Option[Set[String]] = { - val matchingFilters = filters.filter(_.column.equals(labelName)) - if (matchingFilters.isEmpty) - None - else - Some(matchingFilters.head.filter.valuesStrings.map(_.toString)) - } - - def getLabelValueFromLogicalPlan(logicalPlan: LogicalPlan, labelName: String): Option[Set[String]] = { - val labelValues = LogicalPlan.findLeafLogicalPlans(logicalPlan).flatMap { lp => - lp match { - case lp: LabelValues => lp.labelConstraints.get(labelName).map(Set(_)) - case lp: RawSeries => getLabelValueFromFilters(lp.filters, labelName) - case lp: RawChunkMeta => getLabelValueFromFilters(lp.filters, labelName) - case lp: SeriesKeysByFilters => getLabelValueFromFilters(lp.filters, labelName) - case lp: ScalarTimeBasedPlan => Nil // Plan does not have labels - case lp: ScalarFixedDoublePlan => Nil - case lp: ScalarBinaryOperation => Nil - case _ => throw new BadQueryException("Invalid logical plan") - } - } - if (labelValues.isEmpty) { - None - } else { - var res: Set[String] = Set() - // Concatenate results - Some(labelValues.foldLeft(res) { (acc, i) => i.union(acc) }) - } - } -} diff --git a/query/src/test/scala/filodb/query/LogicalPlanSpec.scala b/query/src/test/scala/filodb/query/LogicalPlanSpec.scala new file mode 100644 index 0000000000..57e9fd0f6f --- /dev/null +++ b/query/src/test/scala/filodb/query/LogicalPlanSpec.scala @@ -0,0 +1,152 @@ +package filodb.query + +import filodb.core.query.{ColumnFilter, RangeParams} +import filodb.core.query.Filter.{Equals, EqualsRegex, In, NotEquals, NotEqualsRegex} +import filodb.query.BinaryOperator.DIV +import filodb.query.Cardinality.OneToOne +import filodb.query.RangeFunctionId.SumOverTime +import org.scalatest.{FunSpec, Matchers} + +class LogicalPlanSpec extends FunSpec with Matchers { + + it("should get labelValueOps from logicalPlan") { + + val rawSeries = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("_name_", Equals("MetricName")), + ColumnFilter("instance", NotEquals("Inst-0"))), Seq("_name_", "instance"), Some(300000), None) + val periodicSeriesWithWindowing = PeriodicSeriesWithWindowing(rawSeries, 1000, 500, 5000, 100, SumOverTime) + + val res = LogicalPlan.getLabelValueOperatorsFromLogicalPlan(periodicSeriesWithWindowing) + res.get.size.shouldEqual(1) + res.get(0).labelValueOperators.size.shouldEqual(2) + res.get(0).labelValueOperators(0).columnName.shouldEqual("_name_") + res.get(0).labelValueOperators(0).value.shouldEqual(Seq("MetricName")) + res.get(0).labelValueOperators(0).operator.shouldEqual("=") + res.get(0).labelValueOperators(1).columnName.shouldEqual("instance") + res.get(0).labelValueOperators(1).value.shouldEqual(Seq("Inst-0")) + res.get(0).labelValueOperators(1).operator.shouldEqual("!=") + } + + it("should get labelValueOps from logicalPlan with filter In") { + + val rawSeries = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("_name_", Equals("MetricName")), + ColumnFilter("instance", In(Set("Inst-1", "Inst-0")))), Seq("_name_", "instance"), Some(300000), None) + val periodicSeriesWithWindowing = PeriodicSeriesWithWindowing(rawSeries, 1000, 500, 5000, 100, SumOverTime) + + val res = LogicalPlan.getLabelValueOperatorsFromLogicalPlan(periodicSeriesWithWindowing) + res.get.size.shouldEqual(1) + res.get(0).labelValueOperators.size.shouldEqual(2) + res.get(0).labelValueOperators(0).columnName.shouldEqual("_name_") + res.get(0).labelValueOperators(0).value.shouldEqual(Seq("MetricName")) + res.get(0).labelValueOperators(0).operator.shouldEqual("=") + res.get(0).labelValueOperators(1).columnName.shouldEqual("instance") + res.get(0).labelValueOperators(1).value.shouldEqual(Seq("Inst-0", "Inst-1")) + res.get(0).labelValueOperators(1).operator.shouldEqual("in") + } + + it("should get labelValueOps from BinaryJoin LogicalPlan") { + + val rawSeriesLhs = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("_name_", Equals("MetricName1")), + ColumnFilter("instance", EqualsRegex("Inst-0"))), Seq("_name_", "instance"), Some(300000), None) + val lhs = PeriodicSeries(rawSeriesLhs, 1000, 500, 50000) + + val rawSeriesRhs = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("job", Equals("MetricName2")), + ColumnFilter("instance", NotEqualsRegex("Inst-1"))), Seq("job", "instance"), Some(300000), None) + val rhs = PeriodicSeries(rawSeriesRhs, 1000, 500, 50000) + + val binaryJoin = BinaryJoin(lhs, DIV, OneToOne, rhs) + + val res = LogicalPlan.getLabelValueOperatorsFromLogicalPlan(binaryJoin) + res.get.size.shouldEqual(2) + res.get(0).labelValueOperators.size.shouldEqual(2) + res.get(0).labelValueOperators(0).columnName.shouldEqual("_name_") + res.get(0).labelValueOperators(0).value.shouldEqual(Seq("MetricName1")) + res.get(0).labelValueOperators(0).operator.shouldEqual("=") + res.get(0).labelValueOperators(1).columnName.shouldEqual("instance") + res.get(0).labelValueOperators(1).value.shouldEqual(Seq("Inst-0")) + res.get(0).labelValueOperators(1).operator.shouldEqual("=~") + res.get(1).labelValueOperators.size.shouldEqual(2) + res.get(1).labelValueOperators(0).columnName.shouldEqual("job") + res.get(1).labelValueOperators(0).value.shouldEqual(Seq("MetricName2")) + res.get(1).labelValueOperators(0).operator.shouldEqual("=") + res.get(1).labelValueOperators(1).columnName.shouldEqual("instance") + res.get(1).labelValueOperators(1).value.shouldEqual(Seq("Inst-1")) + res.get(1).labelValueOperators(1).operator.shouldEqual("!~") + } + + it("should get labelValueOps fail for scalar logicalPlan") { + val periodicSeriesWithWindowing = ScalarTimeBasedPlan(ScalarFunctionId.Year, RangeParams(1000, 500, 5000)) + val res = LogicalPlan.getLabelValueOperatorsFromLogicalPlan(periodicSeriesWithWindowing) + res.isEmpty should be (true) + intercept[NoSuchElementException] { res.get } + } + + it("should get MetricName fail for scalar logicalPlan") { + val periodicSeriesWithWindowing = ScalarTimeBasedPlan(ScalarFunctionId.Year, RangeParams(1000, 500, 5000)) + val res = LogicalPlan.getLabelValueFromLogicalPlan(periodicSeriesWithWindowing, "_name_") + res.isEmpty should be (true) + intercept[NoSuchElementException] { res.get } + } + + it("should get MetricName from logicalPlan") { + + val rawSeries = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("_name_", Equals("MetricName")), + ColumnFilter("instance", Equals("Inst-0"))), Seq("_name_", "instance"), Some(300000), None) + val periodicSeriesWithWindowing = PeriodicSeriesWithWindowing(rawSeries, 1000, 500, 5000, 100, SumOverTime) + + val res = LogicalPlan.getLabelValueFromLogicalPlan(periodicSeriesWithWindowing, "_name_") + res.get.shouldEqual(Seq("MetricName")) + } + + it("should get LabelName from logicalPlan with filter In") { + + val rawSeries = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("_name_", Equals("MetricName")), + ColumnFilter("instance", In(Set("Inst-0", "Inst-1")))), Seq("_name_", "instance"), Some(300000), None) + val periodicSeriesWithWindowing = PeriodicSeriesWithWindowing(rawSeries, 1000, 500, 5000, 100, SumOverTime) + + val res = LogicalPlan.getLabelValueFromLogicalPlan(periodicSeriesWithWindowing, "instance") + res.get.shouldEqual(Seq("Inst-0", "Inst-1")) + } + + it("should get MetricName from BinaryJoin LogicalPlan") { + + val rawSeriesLhs = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("_name_", Equals("MetricName1")), + ColumnFilter("instance", Equals("Inst-0"))), Seq("_name_", "instance"), Some(300000), None) + val lhs = PeriodicSeries(rawSeriesLhs, 1000, 500, 50000) + + val rawSeriesRhs = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("job", Equals("MetricName2")), + ColumnFilter("instance", Equals("Inst-1"))), Seq("job", "instance"), Some(300000), None) + val rhs = PeriodicSeries(rawSeriesRhs, 1000, 500, 50000) + + val binaryJoin = BinaryJoin(lhs, DIV, OneToOne, rhs) + + val res = LogicalPlan.getLabelValueFromLogicalPlan(binaryJoin, "_name_") + res.get.shouldEqual(Seq("MetricName1")) + } + + it("should return None if label value is not present in logicalPlan") { + + val rawSeries = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("_name_", Equals("MetricName")), + ColumnFilter("instance", Equals("Inst-0"))), Seq("_name_", "instance"), Some(300000), None) + val periodicSeriesWithWindowing = PeriodicSeriesWithWindowing(rawSeries, 1000, 500, 5000, 100, SumOverTime) + + val res = LogicalPlan.getLabelValueFromLogicalPlan(periodicSeriesWithWindowing, "_name") + res.isEmpty shouldEqual(true) + } + + it("should concatenate results from lhs and rhs for BinaryJoin LogicalPlan") { + + val rawSeriesLhs = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("_name_", Equals("MetricName1")), + ColumnFilter("instance", Equals("Inst-0"))), Seq("_name_", "instance"), Some(300000), None) + val lhs = PeriodicSeries(rawSeriesLhs, 1000, 500, 50000) + + val rawSeriesRhs = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("job", Equals("MetricName2")), + ColumnFilter("instance", Equals("Inst-1"))), Seq("job", "instance"), Some(300000), None) + val rhs = PeriodicSeries(rawSeriesRhs, 1000, 500, 50000) + + val binaryJoin = BinaryJoin(lhs, DIV, OneToOne, rhs) + + val res = LogicalPlan.getLabelValueFromLogicalPlan(binaryJoin, "instance") + res.get.shouldEqual(Seq("Inst-0", "Inst-1")) + } + +} diff --git a/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala b/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala index 9a3525eb71..54be905e64 100644 --- a/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala +++ b/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala @@ -308,8 +308,8 @@ class AggrOverRangeVectorsSpec extends RawDataWindowingSpec with ScalaFutures { val agg = RowAggregator(AggregationOperator.Avg, Nil, tvSchema) val aggMR = AggregateMapReduce(AggregationOperator.Avg, Nil, Nil, Nil) - val mapped1 = aggMR(Observable.fromIterable(Seq(toRv(s1))), queryConfig, 1000, tvSchema) - val mapped2 = aggMR(Observable.fromIterable(Seq(toRv(s2))), queryConfig, 1000, tvSchema) + val mapped1 = aggMR(Observable.fromIterable(Seq(toRv(s1))), querySession, 1000, tvSchema) + val mapped2 = aggMR(Observable.fromIterable(Seq(toRv(s2))), querySession, 1000, tvSchema) val resultObs4 = RangeVectorAggregator.mapReduce(agg, true, mapped1 ++ mapped2, rv=>rv.key) val result4 = resultObs4.toListL.runAsync.futureValue diff --git a/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala b/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala index aa4a067b78..6c6dec9928 100644 --- a/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala @@ -21,6 +21,7 @@ class BinaryJoinExecSpec extends FunSpec with Matchers with ScalaFutures { val config = ConfigFactory.load("application_test.conf").getConfig("filodb") val queryConfig = new QueryConfig(config.getConfig("query")) + val querySession = QuerySession(QueryContext(), queryConfig) val rand = new Random() val error = 0.00000001d @@ -94,7 +95,7 @@ class BinaryJoinExecSpec extends FunSpec with Matchers with ScalaFutures { val rhs = QueryResult("someId", null, samplesRhs2.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on // note below that order of lhs and rhs is reversed, but index is right. Join should take that into account - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), tvSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), tvSchemaTask, querySession) .toListL.runAsync.futureValue result.foreach { rv => @@ -123,7 +124,7 @@ class BinaryJoinExecSpec extends FunSpec with Matchers with ScalaFutures { val lhs = QueryResult("someId", null, samplesLhs.map(rv => SerializedRangeVector(rv, schema))) val rhs = QueryResult("someId", null, samplesRhs2.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val result = execPlan.compose(Observable.fromIterable(Seq((lhs, 0), (rhs, 1))), tvSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((lhs, 0), (rhs, 1))), tvSchemaTask, querySession) .toListL.runAsync.futureValue result.foreach { rv => @@ -160,7 +161,7 @@ class BinaryJoinExecSpec extends FunSpec with Matchers with ScalaFutures { val rhs = QueryResult("someId", null, samplesRhs2.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val fut = execPlan.compose(Observable.fromIterable(Seq((lhs, 0), (rhs, 1))), tvSchemaTask, queryConfig) + val fut = execPlan.compose(Observable.fromIterable(Seq((lhs, 0), (rhs, 1))), tvSchemaTask, querySession) .toListL.runAsync ScalaFutures.whenReady(fut.failed) { e => e shouldBe a[BadQueryException] @@ -191,7 +192,7 @@ class BinaryJoinExecSpec extends FunSpec with Matchers with ScalaFutures { val rhs = QueryResult("someId", null, samplesRhs.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val fut = execPlan.compose(Observable.fromIterable(Seq((lhs, 0), (rhs, 1))), tvSchemaTask, queryConfig) + val fut = execPlan.compose(Observable.fromIterable(Seq((lhs, 0), (rhs, 1))), tvSchemaTask, querySession) .toListL.runAsync ScalaFutures.whenReady(fut.failed) { e => e.printStackTrace() @@ -213,7 +214,7 @@ class BinaryJoinExecSpec extends FunSpec with Matchers with ScalaFutures { val rhs = QueryResult("someId", null, samplesRhsGrouping.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on // note below that order of lhs and rhs is reversed, but index is right. Join should take that into account - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), tvSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), tvSchemaTask, querySession) .toListL.runAsync.futureValue result.foreach { rv => @@ -241,7 +242,7 @@ class BinaryJoinExecSpec extends FunSpec with Matchers with ScalaFutures { val rhs = QueryResult("someId", null, samplesRhsGrouping.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on // note below that order of lhs and rhs is reversed, but index is right. Join should take that into account - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), tvSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), tvSchemaTask, querySession) .toListL.runAsync.futureValue result.foreach { rv => @@ -289,7 +290,7 @@ class BinaryJoinExecSpec extends FunSpec with Matchers with ScalaFutures { val rhs = QueryResult("someId", null, samplesRhs2.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on // note below that order of lhs and rhs is reversed, but index is right. Join should take that into account - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), tvSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), tvSchemaTask, querySession) .toListL.runAsync.futureValue result.foreach { rv => @@ -337,7 +338,7 @@ class BinaryJoinExecSpec extends FunSpec with Matchers with ScalaFutures { val rhs = QueryResult("someId", null, samplesRhs.map(rv => SerializedRangeVector (rv, schema))) // scalastyle:on // note below that order of lhs and rhs is reversed, but index is right. Join should take that into account - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), tvSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), tvSchemaTask, querySession) .toListL.runAsync.futureValue result.foreach { rv => diff --git a/query/src/test/scala/filodb/query/exec/BinaryJoinGroupingSpec.scala b/query/src/test/scala/filodb/query/exec/BinaryJoinGroupingSpec.scala index d88f1a1109..e64dc08436 100644 --- a/query/src/test/scala/filodb/query/exec/BinaryJoinGroupingSpec.scala +++ b/query/src/test/scala/filodb/query/exec/BinaryJoinGroupingSpec.scala @@ -1,6 +1,7 @@ package filodb.query.exec import scala.util.Random + import com.typesafe.config.ConfigFactory import monix.eval.Task import monix.execution.Scheduler @@ -22,6 +23,8 @@ class BinaryJoinGroupingSpec extends FunSpec with Matchers with ScalaFutures { val config = ConfigFactory.load("application_test.conf").getConfig("filodb") val queryConfig = new QueryConfig(config.getConfig("query")) + val querySession = QuerySession(QueryContext(), queryConfig) + val tvSchema = ResultSchema(Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), ColumnInfo("value", ColumnType.DoubleColumn)), 1) val schema = Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), @@ -123,7 +126,7 @@ class BinaryJoinGroupingSpec extends FunSpec with Matchers with ScalaFutures { val lhs = QueryResult("someId", null, sampleNodeCpu.map(rv => SerializedRangeVector(rv, schema))) val rhs = QueryResult("someId", null, samplesRhs2.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), tvSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), tvSchemaTask, querySession) .toListL.runAsync.futureValue val expectedLabels = List(Map(ZeroCopyUTF8String("instance") -> ZeroCopyUTF8String("abc"), @@ -152,13 +155,13 @@ class BinaryJoinGroupingSpec extends FunSpec with Matchers with ScalaFutures { new Array[ExecPlan](1), BinaryOperator.MUL, Cardinality.ManyToOne, - Nil, Seq("role", "mode"), Seq("role"), "__name__") + Nil, Seq("role", "mode"), Seq("role"), "__name__") // scalastyle:off val lhs = QueryResult("someId", null, sampleNodeCpu.map(rv => SerializedRangeVector(rv, schema))) val rhs = QueryResult("someId", null, samplesRhs2.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), tvSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), tvSchemaTask, querySession) .toListL.runAsync.futureValue val expectedLabels = List(Map(ZeroCopyUTF8String("instance") -> ZeroCopyUTF8String("abc"), @@ -182,7 +185,7 @@ class BinaryJoinGroupingSpec extends FunSpec with Matchers with ScalaFutures { val agg = RowAggregator(AggregationOperator.Sum, Nil, tvSchema) val aggMR = AggregateMapReduce(AggregationOperator.Sum, Nil, Nil, Seq("instance", "job")) - val mapped = aggMR(Observable.fromIterable(sampleNodeCpu), queryConfig, 1000, tvSchema) + val mapped = aggMR(Observable.fromIterable(sampleNodeCpu), querySession, 1000, tvSchema) val resultObs4 = RangeVectorAggregator.mapReduce(agg, true, mapped, rv=>rv.key) val samplesRhs = resultObs4.toListL.runAsync.futureValue @@ -198,7 +201,7 @@ class BinaryJoinGroupingSpec extends FunSpec with Matchers with ScalaFutures { val lhs = QueryResult("someId", null, sampleNodeCpu.map(rv => SerializedRangeVector(rv, schema))) val rhs = QueryResult("someId", null, samplesRhs.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), tvSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), tvSchemaTask, querySession) .toListL.runAsync.futureValue val expectedLabels = List(Map(ZeroCopyUTF8String("instance") -> ZeroCopyUTF8String("abc"), @@ -243,7 +246,7 @@ class BinaryJoinGroupingSpec extends FunSpec with Matchers with ScalaFutures { val lhs = QueryResult("someId", null, sampleNodeRole.map(rv => SerializedRangeVector(rv, schema))) val rhs = QueryResult("someId", null, samplesRhs2.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), tvSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), tvSchemaTask, querySession) .toListL.runAsync.futureValue val expectedLabels = List(Map(ZeroCopyUTF8String("instance") -> ZeroCopyUTF8String("abc"), @@ -261,7 +264,7 @@ class BinaryJoinGroupingSpec extends FunSpec with Matchers with ScalaFutures { val agg = RowAggregator(AggregationOperator.Sum, Nil, tvSchema) val aggMR = AggregateMapReduce(AggregationOperator.Sum, Nil, Nil, Seq("instance", "job")) - val mapped = aggMR(Observable.fromIterable(sampleNodeCpu), queryConfig, 1000, tvSchema) + val mapped = aggMR(Observable.fromIterable(sampleNodeCpu), querySession, 1000, tvSchema) val resultObs4 = RangeVectorAggregator.mapReduce(agg, true, mapped, rv=>rv.key) val samplesRhs = resultObs4.toListL.runAsync.futureValue @@ -277,7 +280,7 @@ class BinaryJoinGroupingSpec extends FunSpec with Matchers with ScalaFutures { val lhs = QueryResult("someId", null, sampleNodeCpu.map(rv => SerializedRangeVector(rv, schema))) val rhs = QueryResult("someId", null, samplesRhs.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), tvSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), tvSchemaTask, querySession) .toListL.runAsync.futureValue val expectedLabels = List(Map(ZeroCopyUTF8String("instance") -> ZeroCopyUTF8String("abc"), @@ -357,7 +360,7 @@ class BinaryJoinGroupingSpec extends FunSpec with Matchers with ScalaFutures { val lhs = QueryResult("someId", null, sampleLhs.map(rv => SerializedRangeVector(rv, schema))) val rhs = QueryResult("someId", null, sampleRhs.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), tvSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), tvSchemaTask, querySession) .toListL.runAsync.futureValue val expectedLabels = List(Map(ZeroCopyUTF8String("instance") -> ZeroCopyUTF8String("abc"), diff --git a/query/src/test/scala/filodb/query/exec/BinaryJoinSetOperatorSpec.scala b/query/src/test/scala/filodb/query/exec/BinaryJoinSetOperatorSpec.scala index 07d47254a5..83019579a0 100644 --- a/query/src/test/scala/filodb/query/exec/BinaryJoinSetOperatorSpec.scala +++ b/query/src/test/scala/filodb/query/exec/BinaryJoinSetOperatorSpec.scala @@ -24,6 +24,8 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures val config = ConfigFactory.load("application_test.conf").getConfig("filodb") val queryConfig = new QueryConfig(config.getConfig("query")) + val querySession = QuerySession(QueryContext(), queryConfig) + val tvSchema = ResultSchema(Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), ColumnInfo("value", ColumnType.DoubleColumn)), 1) val schema = Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), @@ -186,7 +188,7 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures val lhs = QueryResult("someId", null, sampleCanary.map(rv => SerializedRangeVector(rv, schema))) val rhs = QueryResult("someId", null, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue @@ -217,13 +219,13 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures BinaryOperator.LAND, Nil, Nil, "__name__") - val canaryPlusOne = scalarOpMapper(Observable.fromIterable(sampleCanary), queryConfig, 1000, resultSchema). + val canaryPlusOne = scalarOpMapper(Observable.fromIterable(sampleCanary), querySession, 1000, resultSchema). toListL.runAsync.futureValue // scalastyle:off val lhs = QueryResult("someId", null, canaryPlusOne.map(rv => SerializedRangeVector(rv, schema))) val rhs = QueryResult("someId", null, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue val expectedLabels = List(Map(ZeroCopyUTF8String("__name__") -> ZeroCopyUTF8String("http_requests"), @@ -254,13 +256,13 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures BinaryOperator.LAND, Seq("instance", "job"), Nil, "__name__") - val canaryPlusOne = scalarOpMapper(Observable.fromIterable(sampleCanary), queryConfig, 1000, resultSchema). + val canaryPlusOne = scalarOpMapper(Observable.fromIterable(sampleCanary), querySession, 1000, resultSchema). toListL.runAsync.futureValue // scalastyle:off val lhs = QueryResult("someId", null, canaryPlusOne.map(rv => SerializedRangeVector(rv, schema))) val rhs = QueryResult("someId", null, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue val expectedLabels = List(Map(ZeroCopyUTF8String("__name__") -> ZeroCopyUTF8String("http_requests"), @@ -291,13 +293,13 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures BinaryOperator.LAND, Seq("instance"), Nil, "__name__") - val canaryPlusOne = scalarOpMapper(Observable.fromIterable(sampleCanary), queryConfig, 1000, resultSchema). + val canaryPlusOne = scalarOpMapper(Observable.fromIterable(sampleCanary), querySession, 1000, resultSchema). toListL.runAsync.futureValue // scalastyle:off val lhs = QueryResult("someId", null, canaryPlusOne.map(rv => SerializedRangeVector(rv, schema))) val rhs = QueryResult("someId", null, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue val expectedLabels = List(Map(ZeroCopyUTF8String("__name__") -> ZeroCopyUTF8String("http_requests"), @@ -327,13 +329,13 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures BinaryOperator.LAND, Nil, Seq("group"), "__name__") - val canaryPlusOne = scalarOpMapper(Observable.fromIterable(sampleCanary), queryConfig, 1000, resultSchema). + val canaryPlusOne = scalarOpMapper(Observable.fromIterable(sampleCanary), querySession, 1000, resultSchema). toListL.runAsync.futureValue // scalastyle:off val lhs = QueryResult("someId", null, canaryPlusOne.map(rv => SerializedRangeVector(rv, schema))) val rhs = QueryResult("someId", null, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue val expectedLabels = List(Map(ZeroCopyUTF8String("__name__") -> ZeroCopyUTF8String("http_requests"), @@ -362,13 +364,13 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures BinaryOperator.LAND, Nil, Seq("group", "job"), "__name__") - val canaryPlusOne = scalarOpMapper(Observable.fromIterable(sampleCanary), queryConfig, 1000, resultSchema). + val canaryPlusOne = scalarOpMapper(Observable.fromIterable(sampleCanary), querySession, 1000, resultSchema). toListL.runAsync.futureValue // scalastyle:off val lhs = QueryResult("someId", null, canaryPlusOne.map(rv => SerializedRangeVector(rv, schema))) val rhs = QueryResult("someId", null, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue val expectedLabels = List(Map(ZeroCopyUTF8String("__name__") -> ZeroCopyUTF8String("http_requests"), @@ -401,7 +403,7 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures val lhs = QueryResult("someId", null, sampleHttpRequests.map(rv => SerializedRangeVector(rv, schema))) val rhs = QueryResult("someId", null, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue result.size shouldEqual 8 @@ -423,7 +425,7 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures val lhs = QueryResult("someId", null, sampleHttpRequests.map(rv => SerializedRangeVector(rv, schema))) val rhs = QueryResult("someId", null, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue result.size shouldEqual 8 @@ -445,7 +447,7 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures val lhs = QueryResult("someId", null, sampleCanary.map(rv => SerializedRangeVector(rv, schema))) val rhs = QueryResult("someId", null, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue result.size shouldEqual 8 @@ -469,13 +471,13 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures BinaryOperator.LOR, Nil, Nil, "__name__") - val canaryPlusOne = scalarOpMapper(Observable.fromIterable(sampleCanary), queryConfig, 1000, resultSchema). + val canaryPlusOne = scalarOpMapper(Observable.fromIterable(sampleCanary), querySession, 1000, resultSchema). toListL.runAsync.futureValue // scalastyle:off val lhs = QueryResult("someId", null, canaryPlusOne.map(rv => SerializedRangeVector(rv, schema))) val rhs = QueryResult("someId", null, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue val expectedResult = (canaryPlusOne.toArray ++ sampleInstance1).distinct @@ -529,13 +531,13 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures BinaryOperator.LOR, Nil, Nil, "__name__") - val canaryPlusOne = scalarOpMapper(Observable.fromIterable(sampleCanary), queryConfig, 1000, resultSchema). + val canaryPlusOne = scalarOpMapper(Observable.fromIterable(sampleCanary), querySession, 1000, resultSchema). toListL.runAsync.futureValue // scalastyle:off val lhs1 = QueryResult("someId", null, sampleHttpRequests.map(rv => SerializedRangeVector(rv, schema))) val rhs1 = QueryResult("someId", null, sampleVectorMatching.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val result1 = execPlan1.compose(Observable.fromIterable(Seq((rhs1, 1), (lhs1, 0))), resSchemaTask, queryConfig) + val result1 = execPlan1.compose(Observable.fromIterable(Seq((rhs1, 1), (lhs1, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue val execPlan2 = SetOperatorExec(QueryContext(), dummyDispatcher, @@ -548,7 +550,7 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures val lhs2 = QueryResult("someId", null, canaryPlusOne.map(rv => SerializedRangeVector(rv, schema))) val rhs2 = QueryResult("someId", null, result1.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val result2 = execPlan2.compose(Observable.fromIterable(Seq((rhs2, 1), (lhs2, 0))), resSchemaTask, queryConfig) + val result2 = execPlan2.compose(Observable.fromIterable(Seq((rhs2, 1), (lhs2, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue val expectedLabels = List(Map(ZeroCopyUTF8String("__name__") -> ZeroCopyUTF8String("http_requests"), @@ -600,13 +602,13 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures BinaryOperator.LOR, Nil, Nil, "__name__") - val canaryPlusOne = scalarOpMapper(Observable.fromIterable(sampleCanary), queryConfig, 1000, resultSchema). + val canaryPlusOne = scalarOpMapper(Observable.fromIterable(sampleCanary), querySession, 1000, resultSchema). toListL.runAsync.futureValue // scalastyle:off val lhs1 = QueryResult("someId", null, sampleHttpRequests.map(rv => SerializedRangeVector(rv, schema))) val rhs1 = QueryResult("someId", null, sampleVectorMatching.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val result1 = execPlan1.compose(Observable.fromIterable(Seq((rhs1, 1), (lhs1, 0))), resSchemaTask, queryConfig) + val result1 = execPlan1.compose(Observable.fromIterable(Seq((rhs1, 1), (lhs1, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue val execPlan2 = SetOperatorExec(QueryContext(), dummyDispatcher, @@ -619,7 +621,7 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures val lhs2 = QueryResult("someId", null, canaryPlusOne.map(rv => SerializedRangeVector(rv, schema))) val rhs2 = QueryResult("someId", null, result1.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val result2 = execPlan2.compose(Observable.fromIterable(Seq((rhs2, 1), (lhs2, 0))), resSchemaTask, queryConfig) + val result2 = execPlan2.compose(Observable.fromIterable(Seq((rhs2, 1), (lhs2, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue val expectedLabels = List(Map(ZeroCopyUTF8String("__name__") -> ZeroCopyUTF8String("http_requests"), @@ -674,7 +676,7 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures val lhs = QueryResult("someId", null, sampleCanary.map(rv => SerializedRangeVector(rv, schema))) val rhs = QueryResult("someId", null, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue val expectedLabels = List(Map(ZeroCopyUTF8String("__name__") -> ZeroCopyUTF8String("http_requests"), @@ -708,7 +710,7 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures val lhs = QueryResult("someId", null, sampleCanary.map(rv => SerializedRangeVector(rv, schema))) val rhs = QueryResult("someId", null, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue val expectedLabels = List(Map(ZeroCopyUTF8String("__name__") -> ZeroCopyUTF8String("http_requests"), @@ -739,7 +741,7 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures val lhs = QueryResult("someId", null, sampleCanary.map(rv => SerializedRangeVector(rv, schema))) val rhs = QueryResult("someId", null, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue val expectedLabels = List(Map(ZeroCopyUTF8String("__name__") -> ZeroCopyUTF8String("http_requests"), @@ -774,7 +776,7 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures val lhs = QueryResult("someId", null, sampleCanary.map(rv => SerializedRangeVector(rv, schema))) val rhs = QueryResult("someId", null, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue val expectedLabels = List(Map(ZeroCopyUTF8String("__name__") -> ZeroCopyUTF8String("http_requests"), @@ -806,7 +808,7 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures val lhs = QueryResult("someId", null, sampleCanary.map(rv => SerializedRangeVector(rv, schema))) val rhs = QueryResult("someId", null, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue val expectedLabels = List(Map(ZeroCopyUTF8String("__name__") -> ZeroCopyUTF8String("http_requests"), diff --git a/query/src/test/scala/filodb/query/exec/HistToPromSeriesMapperSpec.scala b/query/src/test/scala/filodb/query/exec/HistToPromSeriesMapperSpec.scala index 6ea08e747a..0b9fcaca99 100644 --- a/query/src/test/scala/filodb/query/exec/HistToPromSeriesMapperSpec.scala +++ b/query/src/test/scala/filodb/query/exec/HistToPromSeriesMapperSpec.scala @@ -10,11 +10,11 @@ import filodb.core.metadata.Column.ColumnType import filodb.core.query._ import filodb.memory.format.{ZeroCopyUTF8String => ZCUTF8} import filodb.memory.format.vectors.HistogramWithBuckets -import filodb.query.QueryConfig class HistToPromSeriesMapperSpec extends FunSpec with Matchers with ScalaFutures { val config = ConfigFactory.load("application_test.conf").getConfig("filodb") val queryConfig = new QueryConfig(config.getConfig("query")) + val querySession = QuerySession(QueryContext(), queryConfig) import monix.execution.Scheduler.Implicits.global @@ -36,7 +36,7 @@ class HistToPromSeriesMapperSpec extends FunSpec with Matchers with ScalaFutures mapper.schema(sourceSchema).columns shouldEqual Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), ColumnInfo("value", ColumnType.DoubleColumn)) - val destObs = mapper.apply(sourceObs, queryConfig, 1000, sourceSchema, Nil) + val destObs = mapper.apply(sourceObs, querySession, 1000, sourceSchema, Nil) val destRvs = destObs.toListL.runAsync.futureValue // Should be 8 time series since there are 8 buckets @@ -71,7 +71,7 @@ class HistToPromSeriesMapperSpec extends FunSpec with Matchers with ScalaFutures mapper.schema(sourceSchema).columns shouldEqual Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), ColumnInfo("value", ColumnType.DoubleColumn)) - val destObs = mapper.apply(sourceObs, queryConfig, 1000, sourceSchema, Nil) + val destObs = mapper.apply(sourceObs, querySession, 1000, sourceSchema, Nil) val destRvs = destObs.toListL.runAsync.futureValue // Should be 10 time series since there are up to 10 buckets diff --git a/query/src/test/scala/filodb/query/exec/HistogramQuantileMapperSpec.scala b/query/src/test/scala/filodb/query/exec/HistogramQuantileMapperSpec.scala index 74a52b4ed1..52810a192a 100644 --- a/query/src/test/scala/filodb/query/exec/HistogramQuantileMapperSpec.scala +++ b/query/src/test/scala/filodb/query/exec/HistogramQuantileMapperSpec.scala @@ -11,12 +11,13 @@ import org.scalatest.concurrent.ScalaFutures import filodb.core.metadata.Column.ColumnType import filodb.core.query._ import filodb.memory.format.ZeroCopyUTF8String -import filodb.query.QueryConfig class HistogramQuantileMapperSpec extends FunSpec with Matchers with ScalaFutures { val config = ConfigFactory.load("application_test.conf").getConfig("filodb") val queryConfig = new QueryConfig(config.getConfig("query")) + val querySession = QuerySession(QueryContext(), queryConfig) + val rangeParams = RangeParams(100, 20, 200) import HistogramQuantileMapper._ import ZeroCopyUTF8String._ @@ -57,7 +58,7 @@ class HistogramQuantileMapperSpec extends FunSpec with Matchers with ScalaFuture expectedResult: Seq[(Map[ZeroCopyUTF8String, ZeroCopyUTF8String], Seq[(Int, Double)])]): Unit = { val hqMapper = HistogramQuantileMapper(Seq(StaticFuncArgs(q, rangeParams))) - val result = hqMapper.apply(Observable.fromIterable(histRvs), queryConfig, 10, + val result = hqMapper.apply(Observable.fromIterable(histRvs), querySession, 10, new ResultSchema(Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), ColumnInfo("value", ColumnType.DoubleColumn)), 1), Nil) .toListL.runAsync.futureValue diff --git a/query/src/test/scala/filodb/query/exec/InProcessPlanDispatcherSpec.scala b/query/src/test/scala/filodb/query/exec/InProcessPlanDispatcherSpec.scala index cab2054b0d..c82695a2f4 100644 --- a/query/src/test/scala/filodb/query/exec/InProcessPlanDispatcherSpec.scala +++ b/query/src/test/scala/filodb/query/exec/InProcessPlanDispatcherSpec.scala @@ -5,18 +5,20 @@ import java.util.concurrent.{Executors, TimeUnit} import scala.collection.immutable import scala.concurrent.ExecutionContext import scala.concurrent.duration.FiniteDuration + import com.typesafe.config.{Config, ConfigFactory} import monix.eval.Task import monix.execution.Scheduler import org.scalatest.{BeforeAndAfterAll, FunSpec, Matchers} import org.scalatest.concurrent.ScalaFutures import org.scalatest.time.{Millis, Seconds, Span} + import filodb.core.MetricsTestData.{builder, timeseriesDataset, timeseriesSchema} import filodb.core.TestData import filodb.core.binaryrecord2.{RecordBuilder, RecordContainer} import filodb.core.memstore.{FixedMaxPartitionsEvictionPolicy, SomeData, TimeSeriesMemStore} import filodb.core.metadata.{Column, Dataset, Schemas} -import filodb.core.query.{ColumnFilter, Filter, QueryContext} +import filodb.core.query.{ColumnFilter, Filter, QueryConfig, QueryContext, QuerySession} import filodb.core.store.{AllChunkScan, InMemoryMetaStore, NullColumnStore} import filodb.memory.MemFactory import filodb.memory.format.{SeqRowReader, ZeroCopyUTF8String} @@ -59,6 +61,7 @@ class InProcessPlanDispatcherSpec extends FunSpec with Matchers with ScalaFuture val config: Config = ConfigFactory.load("application_test.conf").getConfig("filodb") val queryConfig = new QueryConfig(config.getConfig("query")) + val querySession = QuerySession(QueryContext(), queryConfig) val policy = new FixedMaxPartitionsEvictionPolicy(20) val memStore = new TimeSeriesMemStore(config, new NullColumnStore, new InMemoryMetaStore(), Some(policy)) @@ -91,7 +94,7 @@ class InProcessPlanDispatcherSpec extends FunSpec with Matchers with ScalaFuture val dispatcher: PlanDispatcher = InProcessPlanDispatcher - val dummyDispatcher = DummyDispatcher(memStore, queryConfig) + val dummyDispatcher = DummyDispatcher(memStore, querySession) val execPlan1 = MultiSchemaPartitionsExec(QueryContext(), dummyDispatcher, timeseriesDataset.ref, 0, filters, AllChunkScan) @@ -119,7 +122,7 @@ class InProcessPlanDispatcherSpec extends FunSpec with Matchers with ScalaFuture val dispatcher: PlanDispatcher = InProcessPlanDispatcher - val dummyDispatcher = DummyDispatcher(memStore, queryConfig) + val dummyDispatcher = DummyDispatcher(memStore, querySession) val execPlan1 = MultiSchemaPartitionsExec(QueryContext(), dummyDispatcher, timeseriesDataset.ref, 0, filters, AllChunkScan) @@ -162,10 +165,10 @@ class InProcessPlanDispatcherSpec extends FunSpec with Matchers with ScalaFuture } } -case class DummyDispatcher(memStore: TimeSeriesMemStore, queryConfig: QueryConfig) extends PlanDispatcher { +case class DummyDispatcher(memStore: TimeSeriesMemStore, querySession: QuerySession) extends PlanDispatcher { // run locally withing any check. override def dispatch(plan: ExecPlan) (implicit sched: Scheduler): Task[QueryResponse] = { - plan.execute(memStore, queryConfig) + plan.execute(memStore, querySession) } } \ No newline at end of file diff --git a/query/src/test/scala/filodb/query/exec/LastSampleFunctionSpec.scala b/query/src/test/scala/filodb/query/exec/LastSampleFunctionSpec.scala index ca66feeafa..38f53f6ab0 100644 --- a/query/src/test/scala/filodb/query/exec/LastSampleFunctionSpec.scala +++ b/query/src/test/scala/filodb/query/exec/LastSampleFunctionSpec.scala @@ -28,7 +28,7 @@ class LastSampleFunctionSpec extends RawDataWindowingSpec { val lastSamplesIter = new SlidingWindowIterator(rv.rows, start, step, end, 0, LastSampleFunction, queryConfig) validateLastSamples(samples, lastSamplesIter, start, end, step) - val chunkedIter = new ChunkedWindowIteratorD(rv, start, step, end, w, chunkedLSFunc, queryConfig) + val chunkedIter = new ChunkedWindowIteratorD(rv, start, step, end, w, chunkedLSFunc, querySession) validateLastSamples(samples, chunkedIter, start, end, step) } } @@ -40,7 +40,7 @@ class LastSampleFunctionSpec extends RawDataWindowingSpec { val lastSamplesIter = new SlidingWindowIterator(rv.rows, start, step, end, 0, LastSampleFunction, queryConfig) validateLastSamples(samples, lastSamplesIter, start, end, step) - val chunkedIter = new ChunkedWindowIteratorD(rv, start, step, end, w, chunkedLSFunc, queryConfig) + val chunkedIter = new ChunkedWindowIteratorD(rv, start, step, end, w, chunkedLSFunc, querySession) validateLastSamples(samples, chunkedIter, start, end, step) } } @@ -53,7 +53,7 @@ class LastSampleFunctionSpec extends RawDataWindowingSpec { val lastSamplesIter = new SlidingWindowIterator(rv.rows, start, step, end, 0, LastSampleFunction, queryConfig) validateLastSamples(samples, lastSamplesIter, start, end, step) - val chunkedIter = new ChunkedWindowIteratorD(rv, start, step, end, w, chunkedLSFunc, queryConfig) + val chunkedIter = new ChunkedWindowIteratorD(rv, start, step, end, w, chunkedLSFunc, querySession) validateLastSamples(samples, chunkedIter, start, end, step) } } @@ -70,7 +70,7 @@ class LastSampleFunctionSpec extends RawDataWindowingSpec { start, step, end, 0, LastSampleFunction, queryConfig) validateLastSamples(samplesWithLongGap, lastSamplesIter, start, end, step) - val chunkedIter = new ChunkedWindowIteratorD(rvWithLongGap, start, step, end, w, chunkedLSFunc, queryConfig) + val chunkedIter = new ChunkedWindowIteratorD(rvWithLongGap, start, step, end, w, chunkedLSFunc, querySession) validateLastSamples(samplesWithLongGap, chunkedIter, start, end, step) } @@ -85,7 +85,7 @@ class LastSampleFunctionSpec extends RawDataWindowingSpec { start, step, end, 0, LastSampleFunction, queryConfig) validateLastSamples(samplesWithLongGap, lastSamplesIter, start, end, step) - val chunkedIter = new ChunkedWindowIteratorD(rvWithLongGap, start, step, end, w, chunkedLSFunc, queryConfig) + val chunkedIter = new ChunkedWindowIteratorD(rvWithLongGap, start, step, end, w, chunkedLSFunc, querySession) validateLastSamples(samplesWithLongGap, chunkedIter, start, end, step) } @@ -100,7 +100,7 @@ class LastSampleFunctionSpec extends RawDataWindowingSpec { start, step, end, 0, LastSampleFunction, queryConfig) validateLastSamples(samplesWithLongGap, lastSamplesIter, start, end, step) - val chunkedIter = new ChunkedWindowIteratorD(rvWithLongGap, start, step, end, w, chunkedLSFunc, queryConfig) + val chunkedIter = new ChunkedWindowIteratorD(rvWithLongGap, start, step, end, w, chunkedLSFunc, querySession) validateLastSamples(samplesWithLongGap, chunkedIter, start, end, step) } diff --git a/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala b/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala index 26e7920e46..a360807b1e 100644 --- a/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala @@ -16,7 +16,7 @@ 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, QueryContext, SerializedRangeVector} +import filodb.core.query.{ColumnFilter, Filter, QueryConfig, QueryContext, QuerySession, SerializedRangeVector} import filodb.core.store.{InMemoryMetaStore, NullColumnStore} import filodb.memory.format.{SeqRowReader, ZeroCopyUTF8String} import filodb.query._ @@ -28,6 +28,8 @@ class MetadataExecSpec extends FunSpec with Matchers with ScalaFutures with Befo val config = ConfigFactory.load("application_test.conf").getConfig("filodb") val queryConfig = new QueryConfig(config.getConfig("query")) + val querySession = QuerySession(QueryContext(), queryConfig) + val policy = new FixedMaxPartitionsEvictionPolicy(20) val memStore = new TimeSeriesMemStore(config, new NullColumnStore, new InMemoryMetaStore(), Some(policy)) @@ -86,7 +88,7 @@ class MetadataExecSpec extends FunSpec with Matchers with ScalaFutures with Befo val execPlan = LabelValuesExec(QueryContext(), dummyDispatcher, timeseriesDataset.ref, 0, filters, Seq("job"), 10) - val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue + val resp = execPlan.execute(memStore, querySession).runAsync.futureValue val result = resp match { case QueryResult(id, _, response) => { val rv = response(0) @@ -98,7 +100,6 @@ class MetadataExecSpec extends FunSpec with Matchers with ScalaFutures with Befo result shouldEqual jobQueryResult1 } - it ("should not return any rows for wrong column filters") { import ZeroCopyUTF8String._ val filters = Seq (ColumnFilter("__name__", Filter.Equals("http_req_total1".utf8)), @@ -107,7 +108,7 @@ class MetadataExecSpec extends FunSpec with Matchers with ScalaFutures with Befo val execPlan = PartKeysExec(QueryContext(), dummyDispatcher, timeseriesDataset.ref, 0, Schemas.promCounter.partition, filters, false, now-5000, now) - val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue + val resp = execPlan.execute(memStore, querySession).runAsync.futureValue resp match { case QueryResult(_, _, results) => results.size shouldEqual 1 results(0).rows.size shouldEqual 0 @@ -121,7 +122,7 @@ class MetadataExecSpec extends FunSpec with Matchers with ScalaFutures with Befo val execPlan = PartKeysExec(QueryContext(), dummyDispatcher, timeseriesDataset.ref, 0, Schemas.promCounter.partition, filters, false, now-5000, now) - val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue + val resp = execPlan.execute(memStore, querySession).runAsync.futureValue val result = resp match { case QueryResult(id, _, response) => response.size shouldEqual 1 @@ -142,7 +143,7 @@ class MetadataExecSpec extends FunSpec with Matchers with ScalaFutures with Befo val execPlan = PartKeysExec(QueryContext(sampleLimit = limit-1), dummyDispatcher, timeseriesDataset.ref, 0, Schemas.promCounter.partition, filters, false, now-5000, now) - val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue + val resp = execPlan.execute(memStore, querySession).runAsync.futureValue val result = resp match { case QueryResult(id, _, response) => { response.size shouldEqual 1 diff --git a/query/src/test/scala/filodb/query/exec/MultiSchemaPartitionsExecSpec.scala b/query/src/test/scala/filodb/query/exec/MultiSchemaPartitionsExecSpec.scala index bc31a90f90..9a712af2e3 100644 --- a/query/src/test/scala/filodb/query/exec/MultiSchemaPartitionsExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/MultiSchemaPartitionsExecSpec.scala @@ -1,24 +1,26 @@ package filodb.query.exec import scala.concurrent.duration._ + import com.typesafe.config.ConfigFactory import monix.eval.Task +import monix.execution.Scheduler import monix.execution.Scheduler.Implicits.global import org.scalatest.{BeforeAndAfterAll, FunSpec, Matchers} import org.scalatest.concurrent.ScalaFutures +import org.scalatest.exceptions.TestFailedException import org.scalatest.time.{Millis, Seconds, Span} + import filodb.core.{DatasetRef, QueryTimeoutException, TestData, Types} import filodb.core.binaryrecord2.RecordBuilder import filodb.core.memstore.{FixedMaxPartitionsEvictionPolicy, SchemaMismatch, SomeData, TimeSeriesMemStore} -import filodb.core.metadata.Schemas import filodb.core.metadata.Column.ColumnType.{DoubleColumn, HistogramColumn, LongColumn, TimestampColumn} +import filodb.core.metadata.Schemas import filodb.core.query._ import filodb.core.store.{AllChunkScan, InMemoryMetaStore, NullColumnStore, TimeRangeChunkScan} import filodb.memory.MemFactory import filodb.memory.format.{SeqRowReader, ZeroCopyUTF8String} import filodb.query._ -import monix.execution.Scheduler -import org.scalatest.exceptions.TestFailedException object MultiSchemaPartitionsExecSpec { val dummyDispatcher = new PlanDispatcher { @@ -42,6 +44,7 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu val config = ConfigFactory.load("application_test.conf").getConfig("filodb") val queryConfig = new QueryConfig(config.getConfig("query")) + val querySession = QuerySession(QueryContext(), queryConfig) val policy = new FixedMaxPartitionsEvictionPolicy(20) val memStore = new TimeSeriesMemStore(config, new NullColumnStore, new InMemoryMetaStore(), Some(policy)) @@ -101,7 +104,7 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu val execPlan = MultiSchemaPartitionsExec(QueryContext(), dummyDispatcher, dsRef, 0, filters, AllChunkScan) - val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue + val resp = execPlan.execute(memStore, querySession).runAsync.futureValue val result = resp.asInstanceOf[QueryResult] result.resultSchema.columns.map(_.colType) shouldEqual Seq(TimestampColumn, DoubleColumn) result.result.size shouldEqual 1 @@ -122,7 +125,7 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu val execPlan = MultiSchemaPartitionsExec(QueryContext(), dummyDispatcher, dsRef, 0, filters, TimeRangeChunkScan(startTime, endTime)) - val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue + val resp = execPlan.execute(memStore, querySession).runAsync.futureValue val result = resp.asInstanceOf[QueryResult] result.result.size shouldEqual 1 val dataRead = result.result(0).rows.map(r=>(r.getLong(0), r.getDouble(1))).toList @@ -138,7 +141,7 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu val execPlan = MultiSchemaPartitionsExec(QueryContext(), dummyDispatcher, dsRef, 0, filters, AllChunkScan) - val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue + val resp = execPlan.execute(memStore, querySession).runAsync.futureValue val result = resp.asInstanceOf[QueryResult] result.resultSchema.columns.isEmpty shouldEqual true result.result.size shouldEqual 0 @@ -152,7 +155,7 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu val execPlan = MultiSchemaPartitionsExec(QueryContext(), dummyDispatcher, MMD.dataset1.ref, 0, filters, TimeRangeChunkScan(100000L, 150000L), colName = Some("count")) - val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue + val resp = execPlan.execute(memStore, querySession).runAsync.futureValue val result = resp.asInstanceOf[QueryResult] result.resultSchema.columns.map(_.colType) shouldEqual Seq(TimestampColumn, LongColumn) result.result.size shouldEqual 1 @@ -168,7 +171,7 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu val execPlan = MultiSchemaPartitionsExec(QueryContext(), dummyDispatcher, dsRef, 0, filters, TimeRangeChunkScan(100000L, 150000L), colName=Some("h")) - val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue + val resp = execPlan.execute(memStore, querySession).runAsync.futureValue val result = resp.asInstanceOf[QueryResult] result.resultSchema.columns.map(_.colType) shouldEqual Seq(TimestampColumn, HistogramColumn) result.result.size shouldEqual 1 @@ -188,7 +191,7 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu val end = now - (numRawSamples-100) * reportingInterval execPlan.addRangeVectorTransformer(new PeriodicSamplesMapper(start, step, end, None, None, QueryContext())) - val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue + val resp = execPlan.execute(memStore, querySession).runAsync.futureValue val result = resp.asInstanceOf[QueryResult] result.resultSchema.columns.map(_.colType) shouldEqual Seq(TimestampColumn, DoubleColumn) // PSM should rename the double column to value always @@ -224,7 +227,7 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu val end = 185000L execPlan.addRangeVectorTransformer(new PeriodicSamplesMapper(start, step, end, None, None, QueryContext())) - val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue + val resp = execPlan.execute(memStore, querySession).runAsync.futureValue val result = resp.asInstanceOf[QueryResult] result.resultSchema.columns.map(_.colType) shouldEqual Seq(TimestampColumn, DoubleColumn) result.result.size shouldEqual 1 @@ -245,7 +248,7 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu val end = 185000L execPlan.addRangeVectorTransformer(new PeriodicSamplesMapper(start, step, end, None, None, QueryContext())) - val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue + val resp = execPlan.execute(memStore, querySession).runAsync.futureValue val result = resp.asInstanceOf[QueryResult] result.resultSchema.columns.map(_.colType) shouldEqual Seq(TimestampColumn, HistogramColumn) result.result.size shouldEqual 1 @@ -272,7 +275,7 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu execPlan.addRangeVectorTransformer(new PeriodicSamplesMapper(start, step, end, Some(300 * 1000), // [5m] Some(InternalRangeFunction.Rate), QueryContext(), rawSource = false)) - val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue + val resp = execPlan.execute(memStore, querySession).runAsync.futureValue val result = resp.asInstanceOf[QueryResult] result.resultSchema.columns.map(_.colType) shouldEqual Seq(TimestampColumn, DoubleColumn) result.result.size shouldEqual 1 @@ -289,7 +292,7 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu val execPlan = MultiSchemaPartitionsExec(QueryContext(), dummyDispatcher, dsRef, 0, Nil, AllChunkScan) - val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue + val resp = execPlan.execute(memStore, querySession).runAsync.futureValue val result = resp.asInstanceOf[QueryError] result.t.getClass shouldEqual classOf[SchemaMismatch] } @@ -298,7 +301,7 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu val execPlan = MultiSchemaPartitionsExec(QueryContext(), dummyDispatcher, dsRef, 0, Nil, AllChunkScan, schema = Some("prom-counter")) - val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue + val resp = execPlan.execute(memStore, querySession).runAsync.futureValue val result = resp.asInstanceOf[QueryResult] result.resultSchema.columns.map(_.colType) shouldEqual Seq(TimestampColumn, DoubleColumn) result.result.size shouldEqual 1 @@ -319,7 +322,7 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu Some(InternalRangeFunction.SumOverTime), QueryContext())) execPlan.addRangeVectorTransformer(AggregateMapReduce(AggregationOperator.Sum, Nil, Nil, Nil)) - val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue + val resp = execPlan.execute(memStore, querySession).runAsync.futureValue info(execPlan.printTree()) // Check that the "inner" SelectRawPartitionsExec has the right schema/columnIDs execPlan.finalPlan shouldBe a[SelectRawPartitionsExec] @@ -342,7 +345,7 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu // Add the histogram_max_quantile function to ExecPlan and make sure results are OK execPlan.addRangeVectorTransformer( exec.InstantVectorFunctionMapper(InstantFunctionId.HistogramMaxQuantile, Seq(StaticFuncArgs(0.99, RangeParams(0,0,0))))) - val resp2 = execPlan.execute(memStore, queryConfig).runAsync.futureValue + val resp2 = execPlan.execute(memStore, querySession).runAsync.futureValue val result2 = resp2.asInstanceOf[QueryResult] result2.resultSchema.columns.map(_.colType) shouldEqual Seq(TimestampColumn, DoubleColumn) result2.result.size shouldEqual 1 @@ -363,7 +366,7 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu val end = 185000L execPlan.addRangeVectorTransformer(new PeriodicSamplesMapper(start, step, end, None, None, QueryContext())) - val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue + val resp = execPlan.execute(memStore, querySession).runAsync.futureValue val result = resp.asInstanceOf[QueryResult] result.resultSchema.columns.map(_.colType) shouldEqual Seq(TimestampColumn, HistogramColumn, DoubleColumn) result.result.size shouldEqual 1 @@ -387,7 +390,7 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu // TODO: SelectChunkInfos should not require a raw schema val execPlan = SelectChunkInfosExec(QueryContext(), dummyDispatcher, dsRef, 0, filters, AllChunkScan, colName = Some("timestamp")) - val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue + val resp = execPlan.execute(memStore, querySession).runAsync.futureValue info(s"resp = $resp") val result = resp.asInstanceOf[QueryResult] result.result.size shouldEqual 1 @@ -419,7 +422,7 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu val execPlan = MultiSchemaPartitionsExec(QueryContext(sampleLimit = 999), dummyDispatcher, dsRef, 0, filters, AllChunkScan) - val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue + val resp = execPlan.execute(memStore, querySession).runAsync.futureValue val result = resp.asInstanceOf[QueryError] result.t.getClass shouldEqual classOf[BadQueryException] } @@ -432,7 +435,7 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu dummyDispatcher, dsRef, 0, filters, AllChunkScan) val thrown = intercept[TestFailedException] { - val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue + val resp = execPlan.execute(memStore, querySession).runAsync.futureValue val result = resp.asInstanceOf[QueryResult] } thrown.getCause.getClass shouldEqual classOf[QueryTimeoutException] diff --git a/query/src/test/scala/filodb/query/exec/PeriodicSamplesMapperSpec.scala b/query/src/test/scala/filodb/query/exec/PeriodicSamplesMapperSpec.scala index c61166c22d..c9b1ee4921 100644 --- a/query/src/test/scala/filodb/query/exec/PeriodicSamplesMapperSpec.scala +++ b/query/src/test/scala/filodb/query/exec/PeriodicSamplesMapperSpec.scala @@ -31,7 +31,8 @@ class PeriodicSamplesMapperSpec extends FunSpec with Matchers with ScalaFutures 500000L -> 200d ) val periodicSamplesVectorFnMapper = exec.PeriodicSamplesMapper(100000L, 100000, 600000L, None, None, QueryContext()) - val resultObs = periodicSamplesVectorFnMapper(Observable.fromIterable(Seq(rv)), queryConfig, 1000, resultSchema, Nil) + val resultObs = periodicSamplesVectorFnMapper(Observable.fromIterable(Seq(rv)), + querySession, 1000, resultSchema, Nil) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map (r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN)) @@ -54,7 +55,8 @@ class PeriodicSamplesMapperSpec extends FunSpec with Matchers with ScalaFutures val periodicSamplesVectorFnMapper = exec.PeriodicSamplesMapper(100100L, 100000, 600100L, None, None, QueryContext(), Nil, Some(100)) - val resultObs = periodicSamplesVectorFnMapper(Observable.fromIterable(Seq(rv)), queryConfig, 1000, resultSchema, Nil) + val resultObs = periodicSamplesVectorFnMapper(Observable.fromIterable(Seq(rv)), querySession, + 1000, resultSchema, Nil) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map (r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN)) diff --git a/query/src/test/scala/filodb/query/exec/TimestampFunctionSpec.scala b/query/src/test/scala/filodb/query/exec/TimestampFunctionSpec.scala index 86934d50ef..e2bac263ae 100644 --- a/query/src/test/scala/filodb/query/exec/TimestampFunctionSpec.scala +++ b/query/src/test/scala/filodb/query/exec/TimestampFunctionSpec.scala @@ -13,7 +13,7 @@ class TimestampFunctionSpec extends RawDataWindowingSpec { val list = rv.rows.map(x => (x.getLong(0), x.getDouble(1))).toList val chunkedIt = new ChunkedWindowIteratorD(rv, 100000, 5000, 120000, w, - new TimestampChunkedFunction, queryConfig) + new TimestampChunkedFunction, querySession) val aggregated = chunkedIt.map(x => (x.getLong(0), x.getDouble(1))).toList val expectedResult = List((100000, 100), (105000, 100), (110000, 110), (115000, 110), (120000, 120)) @@ -27,7 +27,7 @@ class TimestampFunctionSpec extends RawDataWindowingSpec { val list = rv.rows.map(x => (x.getLong(0), x.getDouble(1))).toList val chunkedIt = new ChunkedWindowIteratorD(rv, 95000, 50000, 450000, w, - new TimestampChunkedFunction, queryConfig) + new TimestampChunkedFunction, querySession) val aggregated = chunkedIt.map(x => (x.getLong(0), x.getDouble(1))).toList val expectedResult = List((95000, Double.NaN), (145000, 120d), (195000, 120d), (245000, 120d), (295000, 120d), (345000, 120d), (395000, 120d), (445000, Double.NaN)) diff --git a/query/src/test/scala/filodb/query/exec/WindowIteratorSpec.scala b/query/src/test/scala/filodb/query/exec/WindowIteratorSpec.scala index 3ec4a0e3e1..cd3b1f809a 100644 --- a/query/src/test/scala/filodb/query/exec/WindowIteratorSpec.scala +++ b/query/src/test/scala/filodb/query/exec/WindowIteratorSpec.scala @@ -210,7 +210,7 @@ class WindowIteratorSpec extends RawDataWindowingSpec { val chunkedIt = new ChunkedWindowIteratorD(rv, 50000L, 100000, 1100000L, 100000, RangeFunction(tsResSchema, Some(InternalRangeFunction.SumOverTime), ColumnType.DoubleColumn, queryConfig, - useChunked = true).asChunkedD, queryConfig) + useChunked = true).asChunkedD, querySession) chunkedIt.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual windowResults } @@ -245,7 +245,7 @@ class WindowIteratorSpec extends RawDataWindowingSpec { val rv = timeValueRV(samples) val chunkedIt = new ChunkedWindowIteratorD(rv, 1548191496000L, 15000, 1548191796000L, 300000, RangeFunction(tsResSchema, - Some(Rate), ColumnType.DoubleColumn, queryConfig, useChunked = true).asChunkedD, queryConfig) + Some(Rate), ColumnType.DoubleColumn, queryConfig, useChunked = true).asChunkedD, querySession) chunkedIt.foreach { v => windowResults.find(a => a._1 == v.timestamp).foreach(b => v.value shouldEqual b._2 +- 0.0000000001) } @@ -292,7 +292,7 @@ class WindowIteratorSpec extends RawDataWindowingSpec { val chunkedWinIt = new ChunkedWindowIteratorD(rv, 1540845090000L, 15000, 1540855905000L, 180000, RangeFunction(tsResSchema, - Some(Last), ColumnType.DoubleColumn, queryConfig, useChunked = true).asChunkedD, queryConfig) + Some(Last), ColumnType.DoubleColumn, queryConfig, useChunked = true).asChunkedD, querySession) chunkedWinIt.map(r => (r.getLong(0), r.getDouble(1))).toList.filter(!_._2.isNaN) shouldEqual windowResults } @@ -359,7 +359,7 @@ class WindowIteratorSpec extends RawDataWindowingSpec { val chunkedWinIt = new ChunkedWindowIteratorD(rv, 1540845090000L, 15000, 1540855905000L, queryConfig.staleSampleAfterMs, RangeFunction(tsResSchema, - None, ColumnType.DoubleColumn, queryConfig, useChunked = true).asChunkedD, queryConfig) + None, ColumnType.DoubleColumn, queryConfig, useChunked = true).asChunkedD, querySession) chunkedWinIt.map(r => (r.getLong(0), r.getDouble(1))).toList.filter(!_._2.isNaN) shouldEqual windowResults } @@ -389,7 +389,7 @@ class WindowIteratorSpec extends RawDataWindowingSpec { val chunkedWinIt = new ChunkedWindowIteratorD(rv, 100000L, 100000, 600000L, queryConfig.staleSampleAfterMs + 1, RangeFunction(tsResSchema, - None, ColumnType.DoubleColumn, queryConfig, useChunked = true).asChunkedD, queryConfig) + None, ColumnType.DoubleColumn, queryConfig, useChunked = true).asChunkedD, querySession) chunkedWinIt.map(r => (r.getLong(0), r.getDouble(1))).toList.filter(!_._2.isNaN) shouldEqual windowResults } @@ -425,7 +425,7 @@ class WindowIteratorSpec extends RawDataWindowingSpec { val chunkedIt = new ChunkedWindowIteratorD(rv, 50000L, 100000, 700000L, 100000, RangeFunction(tsResSchema, Some(InternalRangeFunction.AvgOverTime), ColumnType.DoubleColumn, queryConfig, - useChunked = true).asChunkedD, queryConfig) + useChunked = true).asChunkedD, querySession) chunkedIt.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual windowResults } @@ -461,7 +461,7 @@ class WindowIteratorSpec extends RawDataWindowingSpec { val chunkedIt = new ChunkedWindowIteratorD(rv, 50000L, 100000, 700000L, 100000, RangeFunction(tsResSchema, Some(InternalRangeFunction.CountOverTime), ColumnType.DoubleColumn, queryConfig, - useChunked = true).asChunkedD, queryConfig) + useChunked = true).asChunkedD, querySession) chunkedIt.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual windowResults } @@ -500,7 +500,7 @@ class WindowIteratorSpec extends RawDataWindowingSpec { val chunkedItAvg = new ChunkedWindowIteratorD(rvAvg, 50000L, 100000, 750000L, 100000, RangeFunction(dsResSchema, rangeFunc, ColumnType.DoubleColumn, queryConfig, - useChunked = true).asChunkedD, queryConfig) + useChunked = true).asChunkedD, querySession) chunkedItAvg.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual avgWindowResults val rvCnt = timeValueRvDownsample(samples, Array(0, 4)) @@ -515,7 +515,7 @@ class WindowIteratorSpec extends RawDataWindowingSpec { val cntFunc = RangeFunction.downsampleRangeFunction(Some(InternalRangeFunction.CountOverTime)) val chunkedItCnt = new ChunkedWindowIteratorD(rvCnt, 50000L, 100000, 750000L, 100000, RangeFunction(dsResSchema, cntFunc, - ColumnType.DoubleColumn, queryConfig, useChunked = true).asChunkedD, queryConfig) + ColumnType.DoubleColumn, queryConfig, useChunked = true).asChunkedD, querySession) chunkedItCnt.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual countWindowResults } @@ -550,7 +550,7 @@ class WindowIteratorSpec extends RawDataWindowingSpec { val chunkedIt = new ChunkedWindowIteratorD(rv, 50000L, 100000, 700000L, 100000, RangeFunction(tsResSchema, Some(InternalRangeFunction.MinOverTime), ColumnType.DoubleColumn, queryConfig, - useChunked = true).asChunkedD, queryConfig) + useChunked = true).asChunkedD, querySession) chunkedIt.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual windowResults } @@ -586,7 +586,7 @@ class WindowIteratorSpec extends RawDataWindowingSpec { val chunkedIt = new ChunkedWindowIteratorD(rv, 50000L, 100000, 700000L, 100000, RangeFunction(tsResSchema, Some(InternalRangeFunction.MaxOverTime), ColumnType.DoubleColumn, queryConfig, - useChunked = true).asChunkedD, queryConfig) + useChunked = true).asChunkedD, querySession) chunkedIt.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual windowResults } } diff --git a/query/src/test/scala/filodb/query/exec/rangefn/AbsentFunctionSpec.scala b/query/src/test/scala/filodb/query/exec/rangefn/AbsentFunctionSpec.scala index ce871f1462..7a43ae816a 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/AbsentFunctionSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/AbsentFunctionSpec.scala @@ -2,21 +2,23 @@ package filodb.query.exec.rangefn import com.typesafe.config.{Config, ConfigFactory} -import filodb.core.MetricsTestData -import filodb.core.query.Filter.{Equals, NotEqualsRegex} -import filodb.core.query._ -import filodb.memory.format.{RowReader, ZeroCopyUTF8String} -import filodb.query.{QueryConfig, exec} import monix.execution.Scheduler.Implicits.global import monix.reactive.Observable -import org.scalatest.concurrent.ScalaFutures import org.scalatest.{FunSpec, Matchers} +import org.scalatest.concurrent.ScalaFutures + +import filodb.core.MetricsTestData +import filodb.core.query._ +import filodb.core.query.Filter.{Equals, NotEqualsRegex} +import filodb.memory.format.{RowReader, ZeroCopyUTF8String} +import filodb.query.exec class AbsentFunctionSpec extends FunSpec with Matchers with ScalaFutures { val config: Config = ConfigFactory.load("application_test.conf").getConfig("filodb") val resultSchema = ResultSchema(MetricsTestData.timeseriesSchema.infosFromIDs(0 to 1), 1) val queryConfig = new QueryConfig(config.getConfig("query")) - + val querySession = QuerySession(QueryContext(), queryConfig) + val testKey1 = CustomRangeVectorKey( Map(ZeroCopyUTF8String("metric") -> ZeroCopyUTF8String("test1"), ZeroCopyUTF8String("src") -> ZeroCopyUTF8String("source-value"), @@ -67,7 +69,7 @@ class AbsentFunctionSpec extends FunSpec with Matchers with ScalaFutures { ZeroCopyUTF8String("instance") -> ZeroCopyUTF8String("instance1")) val expectedRows = List(1.0, 1.0, 1.0, 1.0, 1.0, 1.0) val absentFunctionMapper = exec.AbsentFunctionMapper(columnFilter, RangeParams(1, 2, 11), "metric") - val resultObs = absentFunctionMapper(Observable.fromIterable(emptySample), queryConfig, 1000, resultSchema, Nil) + val resultObs = absentFunctionMapper(Observable.fromIterable(emptySample), querySession, 1000, resultSchema, Nil) val result = resultObs.toListL.runAsync.futureValue result.size shouldEqual (1) val keys = result.map(_.key.labelValues) @@ -79,7 +81,7 @@ class AbsentFunctionSpec extends FunSpec with Matchers with ScalaFutures { it("should not generate range vector when sample is present") { val columnFilter = Seq(ColumnFilter("host", Equals("host1")), ColumnFilter("instance", Equals("instance1"))) val absentFunctionMapper = exec.AbsentFunctionMapper(columnFilter, RangeParams(1, 20, 1), "metric") - val resultObs = absentFunctionMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema, Nil) + val resultObs = absentFunctionMapper(Observable.fromIterable(testSample), querySession, 1000, resultSchema, Nil) val result = resultObs.toListL.runAsync.futureValue val keys = result.map(_.key.labelValues) val rows = result.flatMap(_.rows.map(_.getDouble(1)).toList) @@ -90,7 +92,7 @@ class AbsentFunctionSpec extends FunSpec with Matchers with ScalaFutures { val columnFilter = Seq(ColumnFilter("host", NotEqualsRegex("host1"))) val expectedRows = List(1.0, 1.0, 1.0, 1.0, 1.0, 1.0) val absentFunctionMapper = exec.AbsentFunctionMapper(columnFilter, RangeParams(1, 2, 11), "metric") - val resultObs = absentFunctionMapper(Observable.fromIterable(emptySample), queryConfig, 1000, resultSchema, Nil) + val resultObs = absentFunctionMapper(Observable.fromIterable(emptySample), querySession, 1000, resultSchema, Nil) val result = resultObs.toListL.runAsync.futureValue result.size shouldEqual (1) val keys = result.map(_.key.labelValues) @@ -105,7 +107,7 @@ class AbsentFunctionSpec extends FunSpec with Matchers with ScalaFutures { ZeroCopyUTF8String("instance") -> ZeroCopyUTF8String("instance1")) val expectedRows = List(1.0, 1.0, 1.0, 1.0, 1.0, 1.0) val absentFunctionMapper = exec.AbsentFunctionMapper(columnFilter, RangeParams(1, 2, 11), "metric") - val resultObs = absentFunctionMapper(Observable.fromIterable(emptySample), queryConfig, 1000, resultSchema, Nil) + val resultObs = absentFunctionMapper(Observable.fromIterable(emptySample), querySession, 1000, resultSchema, Nil) val result = resultObs.toListL.runAsync.futureValue result.size shouldEqual (1) val keys = result.map(_.key.labelValues) @@ -119,7 +121,7 @@ class AbsentFunctionSpec extends FunSpec with Matchers with ScalaFutures { ZeroCopyUTF8String("instance") -> ZeroCopyUTF8String("instance1")) val expectedRows = List((3000, 1.0)) val absentFunctionMapper = exec.AbsentFunctionMapper(columnFilter, RangeParams(1, 1, 3), "metric") - val resultObs = absentFunctionMapper(Observable.fromIterable(testSampleNan), queryConfig, 1000, resultSchema, Nil) + val resultObs = absentFunctionMapper(Observable.fromIterable(testSampleNan), querySession, 1000, resultSchema, Nil) val result = resultObs.toListL.runAsync.futureValue result.size shouldEqual (1) val keys = result.map(_.key.labelValues) 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 5fd6809a4d..84407eeaed 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala @@ -1,16 +1,17 @@ package filodb.query.exec.rangefn -import scala.util.Random import scala.collection.mutable.ArrayBuffer +import scala.util.Random + import com.typesafe.config.ConfigFactory import org.scalatest.{BeforeAndAfterAll, FunSpec, Matchers} + +import filodb.core.{MetricsTestData, QueryTimeoutException, TestData, MachineMetricsData => MMD} import filodb.core.memstore.{TimeSeriesPartition, TimeSeriesPartitionSpec, WriteBufferPool} -import filodb.core.query.{QueryContext, RangeParams, RawDataRangeVector, TransientHistMaxRow, TransientHistRow, TransientRow} +import filodb.core.query._ import filodb.core.store.AllChunkScan -import filodb.core.{MetricsTestData, QueryTimeoutException, TestData, MachineMetricsData => MMD} import filodb.memory._ import filodb.memory.format.{TupleRowReader, vectors => bv} -import filodb.query.QueryConfig import filodb.query.exec._ /** @@ -133,6 +134,7 @@ trait RawDataWindowingSpec extends FunSpec with Matchers with BeforeAndAfterAll val config = ConfigFactory.load("application_test.conf").getConfig("filodb") val queryConfig = new QueryConfig(config.getConfig("query")) + val querySession = QuerySession(QueryContext(), queryConfig) // windowSize and step are in number of elements of the data def numWindows(data: Seq[Any], windowSize: Int, step: Int): Int = data.sliding(windowSize, step).length @@ -193,7 +195,7 @@ trait RawDataWindowingSpec extends FunSpec with Matchers with BeforeAndAfterAll val windowStartTS = defaultStartTS + windowTime val stepTimeMillis = step.toLong * pubFreq val windowEndTS = windowStartTS + (numWindows(data, windowSize, step) - 1) * stepTimeMillis - new ChunkedWindowIteratorD(rv, windowStartTS, stepTimeMillis, windowEndTS, windowTime, func, queryConfig) + new ChunkedWindowIteratorD(rv, windowStartTS, stepTimeMillis, windowEndTS, windowTime, func, querySession) } def chunkedWindowItHist[R <: TransientHistRow](data: Seq[Seq[Any]], @@ -207,7 +209,7 @@ trait RawDataWindowingSpec extends FunSpec with Matchers with BeforeAndAfterAll val stepTimeMillis = step.toLong * pubFreq val windowEndTS = windowStartTS + (numWindows(data, windowSize, step) - 1) * stepTimeMillis new ChunkedWindowIteratorH(rv, windowStartTS, stepTimeMillis, windowEndTS, windowTime, - func.asInstanceOf[ChunkedRangeFunction[TransientHistRow]], queryConfig, QueryContext(), row) + func.asInstanceOf[ChunkedRangeFunction[TransientHistRow]], querySession, row) } def chunkedWindowItHist(data: Seq[Seq[Any]], @@ -396,7 +398,7 @@ class AggrOverTimeFunctionsSpec extends RawDataWindowingSpec { val step = 20 val chunkedIt = new ChunkedWindowIteratorD(rv, 100000, 20000, 150000, 30000, - new ChangesChunkedFunctionD(), queryConfig) + new ChangesChunkedFunctionD(), querySession) val aggregated = chunkedIt.map(x => (x.getLong(0), x.getDouble(1))).toList aggregated shouldEqual List((100000, 0.0), (120000, 2.0), (140000, 2.0)) } @@ -416,26 +418,26 @@ class AggrOverTimeFunctionsSpec extends RawDataWindowingSpec { for (i <- 0 until n) { var rv = timeValueRV(twoSampleData) val chunkedItTwoSample = new ChunkedWindowIteratorD(rv, 110000, 120000, 150000, 30000, - new QuantileOverTimeChunkedFunctionD(Seq(quantiles(i))), queryConfig) + new QuantileOverTimeChunkedFunctionD(Seq(quantiles(i))), querySession) val aggregated2 = chunkedItTwoSample.map(_.getDouble(1)).toBuffer aggregated2(0) shouldEqual twoSampleDataResponses(i) +- 0.0000000001 rv = timeValueRV(threeSampleData) val chunkedItThreeSample = new ChunkedWindowIteratorD(rv, 120000, 20000, 130000, 50000, - new QuantileOverTimeChunkedFunctionD(Seq(quantiles(i))), queryConfig) + new QuantileOverTimeChunkedFunctionD(Seq(quantiles(i))), querySession) val aggregated3 = chunkedItThreeSample.map(_.getDouble(1)).toBuffer aggregated3(0) shouldEqual threeSampleDataResponses(i) +- 0.0000000001 rv = timeValueRV(unevenSampleData) val chunkedItUnevenSample = new ChunkedWindowIteratorD(rv, 120000, 20000, 130000, 30000, - new QuantileOverTimeChunkedFunctionD(Seq(quantiles(i))), queryConfig) + new QuantileOverTimeChunkedFunctionD(Seq(quantiles(i))), querySession) val aggregatedUneven = chunkedItUnevenSample.map(_.getDouble(1)).toBuffer aggregatedUneven(0) shouldEqual unevenSampleDataResponses(i) +- 0.0000000001 } val emptyData = Seq() var rv = timeValueRV(emptyData) val chunkedItNoSample = new ChunkedWindowIteratorD(rv, 110000, 120000, 150000, 30000, - new QuantileOverTimeChunkedFunctionD(Seq(StaticFuncArgs(0.5, rangeParams))), queryConfig) + new QuantileOverTimeChunkedFunctionD(Seq(StaticFuncArgs(0.5, rangeParams))), querySession) val aggregatedEmpty = chunkedItNoSample.map(_.getDouble(1)).toBuffer aggregatedEmpty(0) isNaN @@ -519,25 +521,25 @@ class AggrOverTimeFunctionsSpec extends RawDataWindowingSpec { var rv = timeValueRV(positiveTrendData2) val chunkedIt2 = new ChunkedWindowIteratorD(rv, 160000, 100000, 180000, 100000, - new HoltWintersChunkedFunctionD(params), queryConfig) + new HoltWintersChunkedFunctionD(params), querySession) val aggregated2 = chunkedIt2.map(_.getDouble(1)).toBuffer aggregated2(0) shouldEqual holt_winters(positiveTrendData2) rv = timeValueRV(positiveTrendData3) val chunkedIt3 = new ChunkedWindowIteratorD(rv, 160000, 100000, 180000, 100000, - new HoltWintersChunkedFunctionD(params), queryConfig) + new HoltWintersChunkedFunctionD(params), querySession) val aggregated3 = chunkedIt3.map(_.getDouble(1)).toBuffer aggregated3(0) shouldEqual holt_winters(positiveTrendData3) rv = timeValueRV(positiveTrendData4) val chunkedIt4 = new ChunkedWindowIteratorD(rv, 160000, 100000, 180000, 100000, - new HoltWintersChunkedFunctionD(params), queryConfig) + new HoltWintersChunkedFunctionD(params), querySession) val aggregated4 = chunkedIt4.map(_.getDouble(1)).toBuffer aggregated4(0) shouldEqual holt_winters(positiveTrendData4) rv = timeValueRV(negativeTrendData2) val chunkedItNeg2 = new ChunkedWindowIteratorD(rv, 160000, 100000, 180000, 100000, - new HoltWintersChunkedFunctionD(params), queryConfig) + new HoltWintersChunkedFunctionD(params), querySession) val aggregatedNeg2 = chunkedItNeg2.map(_.getDouble(1)).toBuffer aggregatedNeg2(0) shouldEqual holt_winters(negativeTrendData2) @@ -627,7 +629,8 @@ class AggrOverTimeFunctionsSpec extends RawDataWindowingSpec { } } - it("it should correctly calculate sum_over_time, avg_over_time, stddev_over_time & zscore when the sequence contains NaNs or is empty") { + it("it should correctly calculate sum_over_time, avg_over_time, stddev_over_time & " + + "zscore when the sequence contains NaNs or is empty") { val test_data = Seq( Seq(15900.0, 15920.0, 15940.0, 15960.0, 15980.0, 16000.0, 16020.0), Seq(-15900.0, -15920.0, -15940.0, -15960.0, -15980.0, -16000.0), @@ -644,43 +647,44 @@ class AggrOverTimeFunctionsSpec extends RawDataWindowingSpec { val rv = timeValueRV(data) // sum_over_time - val chunkedItSumOverTime = new ChunkedWindowIteratorD(rv, 160000, 100000, 180000, 100000, new SumOverTimeChunkedFunctionD(), queryConfig) + val chunkedItSumOverTime = new ChunkedWindowIteratorD(rv, 160000, 100000, 180000, 100000, new SumOverTimeChunkedFunctionD(), querySession) val aggregatedSumOverTime = chunkedItSumOverTime.map(_.getDouble(1)).toBuffer - if (aggregatedSumOverTime(0).isNaN) aggregatedSumOverTime(0).isNaN shouldBe true else aggregatedSumOverTime(0) shouldBe sumWithNaN(data) + if (aggregatedSumOverTime(0).isNaN) aggregatedSumOverTime(0).isNaN shouldBe true + else aggregatedSumOverTime(0) shouldBe sumWithNaN(data) // avg_over_time - val chunkedItAvgOverTime = new ChunkedWindowIteratorD(rv, 160000, 100000, 180000, 100000, new AvgOverTimeChunkedFunctionD(), queryConfig) + val chunkedItAvgOverTime = new ChunkedWindowIteratorD(rv, 160000, 100000, 180000, 100000, new AvgOverTimeChunkedFunctionD(), querySession) val aggregatedAvgOverTime = chunkedItAvgOverTime.map(_.getDouble(1)).toBuffer - if (aggregatedAvgOverTime(0).isNaN) aggregatedAvgOverTime(0).isNaN shouldBe true else aggregatedAvgOverTime(0) shouldBe avgWithNaN(data) + if (aggregatedAvgOverTime(0).isNaN) aggregatedAvgOverTime(0).isNaN shouldBe true + else aggregatedAvgOverTime(0) shouldBe avgWithNaN(data) // stdvar_over_time - val chunkedItStdVarOverTime = new ChunkedWindowIteratorD(rv, 160000, 100000, 180000, 100000, new StdVarOverTimeChunkedFunctionD(), queryConfig) + val chunkedItStdVarOverTime = new ChunkedWindowIteratorD(rv, 160000, 100000, 180000, 100000, new StdVarOverTimeChunkedFunctionD(), querySession) val aggregatedStdVarOverTime = chunkedItStdVarOverTime.map(_.getDouble(1)).toBuffer - if (aggregatedStdVarOverTime(0).isNaN) aggregatedStdVarOverTime(0).isNaN shouldBe true else aggregatedStdVarOverTime(0) shouldBe stdVarWithNaN(data) + if (aggregatedStdVarOverTime(0).isNaN) aggregatedStdVarOverTime(0).isNaN shouldBe true + else aggregatedStdVarOverTime(0) shouldBe stdVarWithNaN(data) // stddev_over_time - val chunkedItStdDevOverTime = new ChunkedWindowIteratorD(rv, 160000, 100000, 180000, 100000, new StdDevOverTimeChunkedFunctionD(), queryConfig) + val chunkedItStdDevOverTime = new ChunkedWindowIteratorD(rv, 160000, 100000, 180000, 100000, new StdDevOverTimeChunkedFunctionD(), querySession) val aggregatedStdDevOverTime = chunkedItStdDevOverTime.map(_.getDouble(1)).toBuffer - if (aggregatedStdDevOverTime(0).isNaN) aggregatedStdDevOverTime(0).isNaN shouldBe true else aggregatedStdDevOverTime(0) shouldBe Math.sqrt(stdVarWithNaN(data)) + if (aggregatedStdDevOverTime(0).isNaN) aggregatedStdDevOverTime(0).isNaN shouldBe true + else aggregatedStdDevOverTime(0) shouldBe Math.sqrt(stdVarWithNaN(data)) // zscore - val chunkedItZscore = new ChunkedWindowIteratorD(rv, 160000, 100000, 180000, 100000, new ZScoreChunkedFunctionD(), queryConfig) + val chunkedItZscore = new ChunkedWindowIteratorD(rv, 160000, 100000, 180000, 100000, new ZScoreChunkedFunctionD(), querySession) val aggregatedZscore = chunkedItZscore.map(_.getDouble(1)).toBuffer - if (aggregatedZscore(0).isNaN) aggregatedZscore(0).isNaN shouldBe true else aggregatedZscore(0) shouldBe z_score(data) + if (aggregatedZscore(0).isNaN) aggregatedZscore(0).isNaN shouldBe true + else aggregatedZscore(0) shouldBe z_score(data) } } it("should throw QueryTimeoutException when query processing time is greater than timeout") { the[QueryTimeoutException] thrownBy { - var data = Seq(1.5, 2.5, 3.5, 4.5, 5.5) + val data = Seq(1.5, 2.5, 3.5, 4.5, 5.5) val rv = timeValueRV(data) - val list = rv.rows.map(x => (x.getLong(0), x.getDouble(1))).toList - - val windowSize = 100 - val step = 20 - val chunkedIt = new ChunkedWindowIteratorD(rv, 100000, 20000, 150000, 30000, - new ChangesChunkedFunctionD(), queryConfig, QueryContext(submitTime = System.currentTimeMillis() - 180000)) + new ChangesChunkedFunctionD(), + QuerySession(QueryContext(submitTime = System.currentTimeMillis() - 180000), queryConfig)) val aggregated = chunkedIt.map(x => (x.getLong(0), x.getDouble(1))).toList aggregated shouldEqual List((100000, 0.0), (120000, 2.0), (140000, 2.0)) } should have message "Query timeout in filodb.core.store.WindowedChunkIterator after 180 seconds" diff --git a/query/src/test/scala/filodb/query/exec/rangefn/BinaryOperatorSpec.scala b/query/src/test/scala/filodb/query/exec/rangefn/BinaryOperatorSpec.scala index 7d1fbfe905..dffffc4b47 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/BinaryOperatorSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/BinaryOperatorSpec.scala @@ -7,7 +7,7 @@ import monix.reactive.Observable import org.scalatest.{FunSpec, Matchers} import org.scalatest.concurrent.ScalaFutures import filodb.core.MetricsTestData -import filodb.core.query.{CustomRangeVectorKey, RangeParams, RangeVector, RangeVectorKey, ResultSchema, TransientRow} +import filodb.core.query._ import filodb.memory.format.{RowReader, ZeroCopyUTF8String} import filodb.query._ import filodb.query.exec.{StaticFuncArgs, TimeFuncArgs} @@ -32,6 +32,8 @@ class BinaryOperatorSpec extends FunSpec with Matchers with ScalaFutures { new TransientRow(4L, 94935.1523d)).iterator }) val queryConfig = new QueryConfig(config.getConfig("query")) + val querySession = QuerySession(QueryContext(), queryConfig) + val rand = new Random() val error = 0.00000001d val scalar = 5.0 @@ -233,7 +235,7 @@ class BinaryOperatorSpec extends FunSpec with Matchers with ScalaFutures { // ceil val expectedVal = sampleBase.map(_.rows.map(v => scala.math.floor(v.getDouble(1)))) val binaryOpMapper = exec.ScalarOperationMapper(BinaryOperator.ADD, true, Seq(StaticFuncArgs(scalar, RangeParams(0,0,0)))) - val resultObs = binaryOpMapper(Observable.fromIterable(sampleBase), queryConfig, 1000, resultSchema) + val resultObs = binaryOpMapper(Observable.fromIterable(sampleBase), querySession, 1000, resultSchema) val result = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) expectedVal.zip(result).foreach { case (ex, res) => { @@ -269,7 +271,7 @@ class BinaryOperatorSpec extends FunSpec with Matchers with ScalaFutures { ) val expectedVal = samples.map(_.rows.map(v => v.getDouble(1) * 2)) val binaryOpMapper = exec.ScalarOperationMapper(BinaryOperator.ADD, true, Seq(TimeFuncArgs(RangeParams(1,1,4)))) - val resultObs = binaryOpMapper(Observable.fromIterable(samples), queryConfig, 1000, resultSchema) + val resultObs = binaryOpMapper(Observable.fromIterable(samples), querySession, 1000, resultSchema) val result = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) result.foreach(x=> println(x.toList)) expectedVal.zip(result).foreach { @@ -293,7 +295,7 @@ class BinaryOperatorSpec extends FunSpec with Matchers with ScalaFutures { ) val expectedVal = samples.map(_.rows.map(v => scala.math.floor(v.getDouble(1)))) val binaryOpMapper = exec.ScalarOperationMapper(BinaryOperator.ADD, true, Seq(StaticFuncArgs(1571267260, RangeParams(0,0,0)))) - val resultObs = binaryOpMapper(Observable.fromIterable(samples), queryConfig, 1000, resultSchema) + val resultObs = binaryOpMapper(Observable.fromIterable(samples), querySession, 1000, resultSchema) val result = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) expectedVal.zip(result).foreach { case (ex, res) => { @@ -308,7 +310,7 @@ class BinaryOperatorSpec extends FunSpec with Matchers with ScalaFutures { private def applyBinaryOperationAndAssertResult(samples: Array[RangeVector], expectedVal: Array[Iterator[Double]], binOp: BinaryOperator, scalar: Double, scalarOnLhs: Boolean): Unit = { val scalarOpMapper = exec.ScalarOperationMapper(binOp, scalarOnLhs, Seq(StaticFuncArgs(scalar, RangeParams(0,0,0)))) - val resultObs = scalarOpMapper(Observable.fromIterable(samples), queryConfig, 1000, resultSchema) + val resultObs = scalarOpMapper(Observable.fromIterable(samples), querySession, 1000, resultSchema) val result = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) expectedVal.zip(result).foreach { diff --git a/query/src/test/scala/filodb/query/exec/rangefn/InstantFunctionSpec.scala b/query/src/test/scala/filodb/query/exec/rangefn/InstantFunctionSpec.scala index e24b601eb9..be21ac981e 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/InstantFunctionSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/InstantFunctionSpec.scala @@ -160,28 +160,28 @@ class InstantFunctionSpec extends RawDataWindowingSpec with ScalaFutures { // clamp_max the[IllegalArgumentException] thrownBy { val instantVectorFnMapper1 = exec.InstantVectorFunctionMapper(InstantFunctionId.ClampMax) - val resultObs = instantVectorFnMapper1(Observable.fromIterable(sampleBase), queryConfig, 1000, resultSchema, Nil) + val resultObs = instantVectorFnMapper1(Observable.fromIterable(sampleBase), querySession, 1000, resultSchema, Nil) val result = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1)).toList) } should have message "requirement failed: Cannot use ClampMax without providing a upper limit of max." // clamp_min the[IllegalArgumentException] thrownBy { val instantVectorFnMapper3 = exec.InstantVectorFunctionMapper(InstantFunctionId.ClampMin) - val resultObs = instantVectorFnMapper3(Observable.fromIterable(sampleBase), queryConfig, 1000, resultSchema, Nil) + val resultObs = instantVectorFnMapper3(Observable.fromIterable(sampleBase), querySession, 1000, resultSchema, Nil) resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1)).toList) } should have message "requirement failed: Cannot use ClampMin without providing a lower limit of min." the[IllegalArgumentException] thrownBy { val instantVectorFnMapper5 = exec.InstantVectorFunctionMapper(InstantFunctionId.Sqrt, Seq(StaticFuncArgs(1, rangeParams))) - val resultObs = instantVectorFnMapper5(Observable.fromIterable(sampleBase), queryConfig, 1000, resultSchema, Nil) + val resultObs = instantVectorFnMapper5(Observable.fromIterable(sampleBase), querySession, 1000, resultSchema, Nil) resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1)).toList) } should have message "requirement failed: No additional parameters required for the instant function." the[IllegalArgumentException] thrownBy { val instantVectorFnMapper5 = exec.InstantVectorFunctionMapper(InstantFunctionId.Round, Seq(StaticFuncArgs(1, rangeParams), StaticFuncArgs(2, rangeParams))) - val resultObs = instantVectorFnMapper5(Observable.fromIterable(sampleBase), queryConfig, 1000, resultSchema, Nil) + val resultObs = instantVectorFnMapper5(Observable.fromIterable(sampleBase), querySession, 1000, resultSchema, Nil) resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1)).toList) } should have message "requirement failed: Only one optional parameters allowed for Round." @@ -189,7 +189,7 @@ class InstantFunctionSpec extends RawDataWindowingSpec with ScalaFutures { the[IllegalArgumentException] thrownBy { val (data, histRV) = histogramRV(numSamples = 10) val ivMapper = exec.InstantVectorFunctionMapper(InstantFunctionId.HistogramQuantile) - val resultObs = ivMapper(Observable.fromIterable(Array(histRV)), queryConfig, 1000, histSchema, Nil) + val resultObs = ivMapper(Observable.fromIterable(Array(histRV)), querySession, 1000, histSchema, Nil) resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1)).toList) } should have message "requirement failed: Quantile (between 0 and 1) required for histogram quantile" @@ -197,7 +197,7 @@ class InstantFunctionSpec extends RawDataWindowingSpec with ScalaFutures { the[IllegalArgumentException] thrownBy { val (data, histRV) = histogramRV(numSamples = 10) val ivMapper = exec.InstantVectorFunctionMapper(InstantFunctionId.HistogramBucket) - val resultObs = ivMapper(Observable.fromIterable(Array(histRV)), queryConfig, 1000, histSchema, Nil) + val resultObs = ivMapper(Observable.fromIterable(Array(histRV)), querySession, 1000, histSchema, Nil) resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1)).toList) } should have message "requirement failed: Bucket/le required for histogram bucket" } @@ -206,7 +206,7 @@ class InstantFunctionSpec extends RawDataWindowingSpec with ScalaFutures { // ceil val expectedVal = sampleBase.map(_.rows.map(v => scala.math.floor(v.getDouble(1)))) val instantVectorFnMapper = exec.InstantVectorFunctionMapper(InstantFunctionId.Ceil) - val resultObs = instantVectorFnMapper(Observable.fromIterable(sampleBase), queryConfig, 1000, resultSchema, Nil) + val resultObs = instantVectorFnMapper(Observable.fromIterable(sampleBase), querySession, 1000, resultSchema, Nil) val result = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) expectedVal.zip(result).foreach { case (ex, res) => { @@ -314,7 +314,7 @@ class InstantFunctionSpec extends RawDataWindowingSpec with ScalaFutures { instantFunctionId: InstantFunctionId, funcParams: Seq[Double] = Nil, schema: ResultSchema = resultSchema): Unit = { val instantVectorFnMapper = exec.InstantVectorFunctionMapper(instantFunctionId, funcParams.map(x => StaticFuncArgs(x, RangeParams(100,10,200)))) - val resultObs = instantVectorFnMapper(Observable.fromIterable(samples), queryConfig, 1000, schema, Nil) + val resultObs = instantVectorFnMapper(Observable.fromIterable(samples), querySession, 1000, schema, Nil) val result = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) expectedVal.zip(result).foreach { case (ex, res) => { diff --git a/query/src/test/scala/filodb/query/exec/rangefn/LabelReplaceSpec.scala b/query/src/test/scala/filodb/query/exec/rangefn/LabelReplaceSpec.scala index ed24592067..db0c662aad 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/LabelReplaceSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/LabelReplaceSpec.scala @@ -5,8 +5,9 @@ import monix.execution.Scheduler.Implicits.global import monix.reactive.Observable import org.scalatest.{FunSpec, Matchers} import org.scalatest.concurrent.ScalaFutures + import filodb.core.MetricsTestData -import filodb.core.query.{CustomRangeVectorKey, RangeVector, RangeVectorKey, ResultSchema, TransientRow} +import filodb.core.query.{CustomRangeVectorKey, QueryConfig, QueryContext, QuerySession, RangeVector, RangeVectorKey, ResultSchema, TransientRow} import filodb.memory.format.{RowReader, ZeroCopyUTF8String} import filodb.query._ @@ -42,6 +43,7 @@ class LabelReplaceSpec extends FunSpec with Matchers with ScalaFutures { }) val queryConfig = new QueryConfig(config.getConfig("query")) + val querySession = QuerySession(QueryContext(), queryConfig) it("should replace label only when match is found in label replace") { val sampleKey1 = CustomRangeVectorKey( @@ -72,7 +74,7 @@ class LabelReplaceSpec extends FunSpec with Matchers with ScalaFutures { val funcParams = Seq("instance", "$1 new Label Value $2", "instance", "(.*):90(.*)") val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, funcParams) - val resultObs = labelVectorFnMapper(Observable.fromIterable(sampleWithKey), queryConfig, 1000, resultSchema, Nil) + val resultObs = labelVectorFnMapper(Observable.fromIterable(sampleWithKey), querySession, 1000, resultSchema, Nil) val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) @@ -118,7 +120,7 @@ class LabelReplaceSpec extends FunSpec with Matchers with ScalaFutures { val funcParams = Seq("instanceNew", "$1-$1", "instance", "(.*)\\d") val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, funcParams) - val resultObs = labelVectorFnMapper(Observable.fromIterable(sampleWithKey), queryConfig, 1000, resultSchema, Nil) + val resultObs = labelVectorFnMapper(Observable.fromIterable(sampleWithKey), querySession, 1000, resultSchema, Nil) val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) @@ -159,7 +161,7 @@ class LabelReplaceSpec extends FunSpec with Matchers with ScalaFutures { val funcParams = Seq("instance", "$1", "instance", "(.*)9") val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, funcParams) - val resultObs = labelVectorFnMapper(Observable.fromIterable(sampleWithKey), queryConfig, 1000, resultSchema, Nil) + val resultObs = labelVectorFnMapper(Observable.fromIterable(sampleWithKey), querySession, 1000, resultSchema, Nil) val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) @@ -182,13 +184,13 @@ class LabelReplaceSpec extends FunSpec with Matchers with ScalaFutures { the[IllegalArgumentException] thrownBy { val miscellaneousFunctionMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, funcParams) - miscellaneousFunctionMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema, Nil) + miscellaneousFunctionMapper(Observable.fromIterable(testSample), querySession, 1000, resultSchema, Nil) } should have message "Invalid Regular Expression for label_replace" the[IllegalArgumentException] thrownBy { val miscellaneousFunctionMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, Seq("instance", "$1")) - miscellaneousFunctionMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema, Nil) + miscellaneousFunctionMapper(Observable.fromIterable(testSample), querySession, 1000, resultSchema, Nil) } should have message "requirement failed: " + "Cannot use LabelReplace without function parameters: " + "instant-vector, dst_label string, replacement string, src_label string, regex string" @@ -196,7 +198,7 @@ class LabelReplaceSpec extends FunSpec with Matchers with ScalaFutures { the[IllegalArgumentException] thrownBy { val miscellaneousFunctionMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, Seq("$instance", "$1", "instance", "(.*)9(")) - miscellaneousFunctionMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema, Nil) + miscellaneousFunctionMapper(Observable.fromIterable(testSample), querySession, 1000, resultSchema, Nil) } should have message "requirement failed: Invalid destination label name" } @@ -210,7 +212,7 @@ class LabelReplaceSpec extends FunSpec with Matchers with ScalaFutures { val funcParams = Seq("dst", "destination-value-$1", "src", "source-value-(.*)") val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, funcParams) - val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema, Nil) + val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), querySession, 1000, resultSchema, Nil) val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) @@ -236,7 +238,7 @@ class LabelReplaceSpec extends FunSpec with Matchers with ScalaFutures { val funcParams = Seq("dst", "destination-value-$1", "src", "value-(.*)") val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, funcParams) - val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema, Nil) + val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), querySession, 1000, resultSchema, Nil) val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) @@ -262,7 +264,7 @@ class LabelReplaceSpec extends FunSpec with Matchers with ScalaFutures { val funcParams = Seq("dst", "$1-value-$2 $3$67", "src", "(.*)-value-(.*)") val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, funcParams) - val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema, Nil) + val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), querySession, 1000, resultSchema, Nil) val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) @@ -288,7 +290,7 @@ class LabelReplaceSpec extends FunSpec with Matchers with ScalaFutures { val funcParams = Seq("dst", "value-$1", "nonexistent-src", "source-value-(.*)") val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, funcParams) - val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema, Nil) + val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), querySession, 1000, resultSchema, Nil) val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) @@ -314,7 +316,7 @@ class LabelReplaceSpec extends FunSpec with Matchers with ScalaFutures { val funcParams = Seq("dst", "value-$1", "nonexistent-src", ".*") val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, funcParams) - val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema, Nil) + val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), querySession, 1000, resultSchema, Nil) val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) @@ -340,7 +342,7 @@ class LabelReplaceSpec extends FunSpec with Matchers with ScalaFutures { val funcParams = Seq("dst", "value-$1", "src", "dummy-regex") val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, funcParams) - val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema, Nil) + val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), querySession, 1000, resultSchema, Nil) val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) @@ -364,7 +366,7 @@ class LabelReplaceSpec extends FunSpec with Matchers with ScalaFutures { val funcParams = Seq("dst", "", "dst", ".*") val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, funcParams) - val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema, Nil) + val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), querySession, 1000, resultSchema, Nil) val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) @@ -388,7 +390,7 @@ class LabelReplaceSpec extends FunSpec with Matchers with ScalaFutures { val funcParams = Seq("src", "", "", "") val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, funcParams) - val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema, Nil) + val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), querySession, 1000, resultSchema, Nil) val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) diff --git a/query/src/test/scala/filodb/query/exec/rangefn/LableJoinSpec.scala b/query/src/test/scala/filodb/query/exec/rangefn/LableJoinSpec.scala index e0254b1b4d..0429c7039c 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/LableJoinSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/LableJoinSpec.scala @@ -5,8 +5,9 @@ import monix.execution.Scheduler.Implicits.global import monix.reactive.Observable import org.scalatest.{FunSpec, Matchers} import org.scalatest.concurrent.ScalaFutures + import filodb.core.MetricsTestData -import filodb.core.query.{CustomRangeVectorKey, RangeVector, RangeVectorKey, ResultSchema, TransientRow} +import filodb.core.query._ import filodb.memory.format.{RowReader, ZeroCopyUTF8String} import filodb.query._ @@ -83,6 +84,7 @@ class LableJoinSpec extends FunSpec with Matchers with ScalaFutures { }) val queryConfig = new QueryConfig(config.getConfig("query")) + val querySession = QuerySession(QueryContext(), queryConfig) it("label_join joins all src values in order") { @@ -99,7 +101,7 @@ class LableJoinSpec extends FunSpec with Matchers with ScalaFutures { val funcParams = Seq("dst", "-", "src", "src1", "src2") val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelJoin, funcParams) - val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema, Nil) + val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), querySession, 1000, resultSchema, Nil) val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) @@ -130,7 +132,7 @@ class LableJoinSpec extends FunSpec with Matchers with ScalaFutures { val funcParams = Seq("dst", "-", "src", "src3", "src1") val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelJoin, funcParams) - val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema, Nil) + val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), querySession, 1000, resultSchema, Nil) val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) @@ -160,7 +162,7 @@ class LableJoinSpec extends FunSpec with Matchers with ScalaFutures { val funcParams = Seq("dst", "", "emptysrc", "emptysrc1", "emptysrc2") val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelJoin, funcParams) - val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema, Nil) + val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), querySession, 1000, resultSchema, Nil) val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) @@ -191,7 +193,7 @@ class LableJoinSpec extends FunSpec with Matchers with ScalaFutures { val funcParams = Seq("dst", "-", "src", "src1", "src2") val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelJoin, funcParams) - val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema, Nil) + val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), querySession, 1000, resultSchema, Nil) val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) @@ -221,7 +223,7 @@ class LableJoinSpec extends FunSpec with Matchers with ScalaFutures { val funcParams = Seq("dst", "-") val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelJoin, funcParams) - val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema, Nil) + val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), querySession, 1000, resultSchema, Nil) val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) @@ -246,19 +248,19 @@ class LableJoinSpec extends FunSpec with Matchers with ScalaFutures { the[IllegalArgumentException] thrownBy { val miscellaneousFunctionMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelJoin, funcParams1) - miscellaneousFunctionMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema, Nil) + miscellaneousFunctionMapper(Observable.fromIterable(testSample), querySession, 1000, resultSchema, Nil) } should have message "requirement failed: Invalid source label name in label_join()" the[IllegalArgumentException] thrownBy { val miscellaneousFunctionMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelJoin, funcParams2) - miscellaneousFunctionMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema, Nil) + miscellaneousFunctionMapper(Observable.fromIterable(testSample), querySession, 1000, resultSchema, Nil) } should have message "requirement failed: Invalid destination label name in label_join()" the[IllegalArgumentException] thrownBy { val miscellaneousFunctionMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelJoin, Seq("dst")) - miscellaneousFunctionMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema, Nil) + miscellaneousFunctionMapper(Observable.fromIterable(testSample), querySession, 1000, resultSchema, Nil) } should have message "requirement failed: expected at least 3 argument(s) in call to label_join" } } 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 abb100917f..c6efe1f78b 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/RateFunctionsSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/RateFunctionsSpec.scala @@ -64,7 +64,7 @@ class RateFunctionsSpec extends RawDataWindowingSpec { // One window, start=end=endTS val it = new ChunkedWindowIteratorD(counterRV, endTs, 10000, endTs, endTs - startTs, - new ChunkedRateFunction, queryConfig) + new ChunkedRateFunction, querySession) it.next.getDouble(1) shouldEqual expected +- errorOk } @@ -86,7 +86,7 @@ class RateFunctionsSpec extends RawDataWindowingSpec { // One window, start=end=endTS val it = new ChunkedWindowIteratorD(rv, endTs, 10000, endTs, endTs - startTs, - new ChunkedRateFunction, queryConfig) + new ChunkedRateFunction, querySession) it.next.getDouble(1) shouldEqual expected +- errorOk } @@ -123,13 +123,13 @@ class RateFunctionsSpec extends RawDataWindowingSpec { // One window, start=end=endTS val it = new ChunkedWindowIteratorD(rv, endTs, 10000, endTs, endTs - startTs, - new ChunkedRateFunction, queryConfig) + new ChunkedRateFunction, querySession) it.next.getDouble(1) shouldEqual expected +- errorOk // Two drops in one chunk val rv2 = timeValueRV(resetChunk1 ++ resetChunk2) val it2 = new ChunkedWindowIteratorD(rv2, endTs, 10000, endTs, endTs - startTs, - new ChunkedRateFunction, queryConfig) + new ChunkedRateFunction, querySession) it2.next.getDouble(1) shouldEqual expected +- errorOk } @@ -138,7 +138,7 @@ class RateFunctionsSpec extends RawDataWindowingSpec { val endTs = 8103215L val it = new ChunkedWindowIteratorD(counterRV, endTs, 10000, endTs, endTs - startTs, - new ChunkedRateFunction, queryConfig) + new ChunkedRateFunction, querySession) it.next.getDouble(1).isNaN shouldEqual true } @@ -150,7 +150,7 @@ class RateFunctionsSpec extends RawDataWindowingSpec { // One window, start=end=endTS val it = new ChunkedWindowIteratorD(flatRV, endTs, 10000, endTs, endTs - startTs, - new ChunkedRateFunction, queryConfig) + new ChunkedRateFunction, querySession) it.next.getDouble(1) shouldEqual 0.0 } @@ -206,7 +206,7 @@ class RateFunctionsSpec extends RawDataWindowingSpec { // One window, start=end=endTS val it = new ChunkedWindowIteratorH(rv, endTs, 100000, endTs, endTs - startTs, - new HistRateFunction, queryConfig) + new HistRateFunction, querySession) // Scheme should have remained the same val answer = it.next.getHistogram(1) answer.numBuckets shouldEqual expected.numBuckets @@ -245,7 +245,7 @@ class RateFunctionsSpec extends RawDataWindowingSpec { // One window, start=end=endTS val it = new ChunkedWindowIteratorH(rv, endTs, 110000, endTs, endTs - startTs, - new HistRateFunction, queryConfig) + new HistRateFunction, querySession) // Scheme should have remained the same val answer = it.next.getHistogram(1) answer.numBuckets shouldEqual expected.numBuckets @@ -374,12 +374,12 @@ class RateFunctionsSpec extends RawDataWindowingSpec { val endTs = 8163070L val expected = (q.last.value - q.head.value) / (q.last.timestamp - q.head.timestamp) * (endTs - startTs) val toEmit = new TransientRow - IncreaseFunction.apply(startTs,endTs, counterWindow, toEmit, queryConfig) + IncreaseFunction.apply(startTs, endTs, counterWindow, toEmit, queryConfig) toEmit.value shouldEqual expected +- errorOk // One window, start=end=endTS val it = new ChunkedWindowIteratorD(counterRV, endTs, 10000, endTs, endTs - startTs, - new ChunkedIncreaseFunction, queryConfig) + new ChunkedIncreaseFunction, querySession) it.next.getDouble(1) shouldEqual expected +- errorOk } @@ -388,13 +388,13 @@ class RateFunctionsSpec extends RawDataWindowingSpec { val endTs = 8163070L val expected = (q2.last.value - q2.head.value) / (q2.last.timestamp - q2.head.timestamp) * (endTs - startTs) val toEmit = new TransientRow - DeltaFunction.apply(startTs,endTs, gaugeWindow, toEmit, queryConfig) + DeltaFunction.apply(startTs, endTs, gaugeWindow, toEmit, queryConfig) toEmit.value shouldEqual expected +- errorOk // One window, start=end=endTS val gaugeRV = timeValueRV(gaugeSamples) val it = new ChunkedWindowIteratorD(gaugeRV, endTs, 10000, endTs, endTs - startTs, - new ChunkedDeltaFunction, queryConfig) + new ChunkedDeltaFunction, querySession) it.next.getDouble(1) shouldEqual expected +- errorOk } diff --git a/query/src/test/scala/filodb/query/exec/rangefn/ScalarFunctionSpec.scala b/query/src/test/scala/filodb/query/exec/rangefn/ScalarFunctionSpec.scala index 244f08f46a..aa62e4e599 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/ScalarFunctionSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/ScalarFunctionSpec.scala @@ -10,7 +10,7 @@ import filodb.core.query._ import filodb.core.store.{InMemoryMetaStore, NullColumnStore} import filodb.memory.format.{RowReader, ZeroCopyUTF8String} import filodb.query.exec.TimeScalarGeneratorExec -import filodb.query.{QueryConfig, QueryResult, ScalarFunctionId, exec} +import filodb.query.{QueryResult, ScalarFunctionId, exec} import monix.execution.Scheduler.Implicits.global import monix.reactive.Observable import org.scalatest.concurrent.ScalaFutures @@ -26,6 +26,7 @@ class ScalarFunctionSpec extends FunSpec with Matchers with ScalaFutures { val config: Config = ConfigFactory.load("application_test.conf").getConfig("filodb") val queryConfig = new QueryConfig(config.getConfig("query")) + val querySession = QuerySession(QueryContext(), queryConfig) val policy = new FixedMaxPartitionsEvictionPolicy(20) val memStore = new TimeSeriesMemStore(config, new NullColumnStore, new InMemoryMetaStore(), Some(policy)) val resultSchema = ResultSchema(MetricsTestData.timeseriesSchema.infosFromIDs(0 to 1), 1) @@ -100,7 +101,7 @@ class ScalarFunctionSpec extends FunSpec with Matchers with ScalaFutures { it("should generate scalar") { val scalarFunctionMapper = exec.ScalarFunctionMapper(ScalarFunctionId.Scalar, RangeParams(1,1,1)) - val resultObs = scalarFunctionMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema, Nil) + val resultObs = scalarFunctionMapper(Observable.fromIterable(testSample), querySession, 1000, resultSchema, Nil) val resultRangeVectors = resultObs.toListL.runAsync.futureValue resultRangeVectors.forall(x => x.isInstanceOf[ScalarFixedDouble]) shouldEqual (true) val resultRows = resultRangeVectors.flatMap(_.rows.map(_.getDouble(1)).toList) @@ -110,7 +111,7 @@ class ScalarFunctionSpec extends FunSpec with Matchers with ScalaFutures { it("should generate scalar values when there is one range vector") { val scalarFunctionMapper = exec.ScalarFunctionMapper(ScalarFunctionId.Scalar, RangeParams(1,1,1)) - val resultObs = scalarFunctionMapper(Observable.fromIterable(oneSample), queryConfig, 1000, resultSchema, Nil) + val resultObs = scalarFunctionMapper(Observable.fromIterable(oneSample), querySession, 1000, resultSchema, Nil) val resultRangeVectors = resultObs.toListL.runAsync.futureValue resultRangeVectors.forall(x => x.isInstanceOf[ScalarVaryingDouble]) shouldEqual (true) val resultRows = resultRangeVectors.flatMap(_.rows.map(_.getDouble(1)).toList) @@ -121,7 +122,7 @@ class ScalarFunctionSpec extends FunSpec with Matchers with ScalaFutures { val execPlan = TimeScalarGeneratorExec(QueryContext(), timeseriesDataset.ref, RangeParams(10, 10, 100), ScalarFunctionId.Time) implicit val timeout: FiniteDuration = FiniteDuration(5, TimeUnit.SECONDS) import monix.execution.Scheduler.Implicits.global - val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue + val resp = execPlan.execute(memStore, querySession).runAsync.futureValue val result = resp match { case QueryResult(id, _, response) => { val rv = response(0) @@ -136,7 +137,7 @@ class ScalarFunctionSpec extends FunSpec with Matchers with ScalaFutures { val execPlan = TimeScalarGeneratorExec(QueryContext(), timeseriesDataset.ref, RangeParams(1565627710, 10, 1565627790), ScalarFunctionId.Hour) implicit val timeout: FiniteDuration = FiniteDuration(5, TimeUnit.SECONDS) import monix.execution.Scheduler.Implicits.global - val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue + val resp = execPlan.execute(memStore, querySession).runAsync.futureValue val result = resp match { case QueryResult(id, _, response) => { val rv = response(0) @@ -152,7 +153,7 @@ class ScalarFunctionSpec extends FunSpec with Matchers with ScalaFutures { val execPlan = TimeScalarGeneratorExec(QueryContext(), timeseriesDataset.ref, RangeParams(1583682900, 100, 1583683400), ScalarFunctionId.DayOfWeek) implicit val timeout: FiniteDuration = FiniteDuration(5, TimeUnit.SECONDS) import monix.execution.Scheduler.Implicits.global - val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue + val resp = execPlan.execute(memStore, querySession).runAsync.futureValue val result = resp match { case QueryResult(id, _, response) => { val rv = response(0) diff --git a/query/src/test/scala/filodb/query/exec/rangefn/SortFunctionSpec.scala b/query/src/test/scala/filodb/query/exec/rangefn/SortFunctionSpec.scala index ec5d9eeb86..660d04707b 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/SortFunctionSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/SortFunctionSpec.scala @@ -4,11 +4,11 @@ import com.typesafe.config.{Config, ConfigFactory} import filodb.core.MetricsTestData import filodb.core.metadata.Column.ColumnType -import filodb.core.query.{ColumnInfo, CustomRangeVectorKey, RangeVector, RangeVectorKey, ResultSchema, TransientRow} +import filodb.core.query._ import filodb.memory.format.{RowReader, ZeroCopyUTF8String} import filodb.query.exec.RangeVectorAggregator import filodb.query.exec.aggregator.RowAggregator -import filodb.query.{exec, AggregationOperator, QueryConfig, SortFunctionId} +import filodb.query.{exec, AggregationOperator, SortFunctionId} import monix.execution.Scheduler.Implicits.global import monix.reactive.Observable @@ -19,6 +19,7 @@ class SortFunctionSpec extends FunSpec with Matchers with ScalaFutures { val config: Config = ConfigFactory.load("application_test.conf").getConfig("filodb") val resultSchema = ResultSchema(MetricsTestData.timeseriesSchema.infosFromIDs(0 to 1), 1) val queryConfig = new QueryConfig(config.getConfig("query")) + val querySession = QuerySession(QueryContext(), queryConfig) val ignoreKey = CustomRangeVectorKey( Map(ZeroCopyUTF8String("ignore") -> ZeroCopyUTF8String("ignore"))) @@ -85,21 +86,21 @@ class SortFunctionSpec extends FunSpec with Matchers with ScalaFutures { it("should sort instant vectors in ascending order") { val sortFunctionMapper = exec.SortFunctionMapper(SortFunctionId.Sort) - val resultObs = sortFunctionMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema, Nil) + val resultObs = sortFunctionMapper(Observable.fromIterable(testSample), querySession, 1000, resultSchema, Nil) val resultRows = resultObs.toListL.runAsync.futureValue.flatMap(_.rows.map(_.getDouble(1)).toList) resultRows.shouldEqual(List(0.0, 1.0, 2.0, 3.0, 4.0, 5.0, 6.0)) } it("should sort instant vectors in descending order") { val sortFunctionMapper = exec.SortFunctionMapper(SortFunctionId.SortDesc) - val resultObs = sortFunctionMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema, Nil) + val resultObs = sortFunctionMapper(Observable.fromIterable(testSample), querySession, 1000, resultSchema, Nil) val resultRows = resultObs.toListL.runAsync.futureValue.flatMap(_.rows.map(_.getDouble(1)).toList) resultRows.shouldEqual(List(6.0, 5.0, 4.0, 3.0, 2.0, 1.0, 0.0)) } it("should return empty rangeVector when sorting empty sample") { val sortFunctionMapper = exec.SortFunctionMapper(SortFunctionId.Sort) - val resultObs = sortFunctionMapper(Observable.fromIterable(emptySample), queryConfig, 1000, resultSchema, Nil) + val resultObs = sortFunctionMapper(Observable.fromIterable(emptySample), querySession, 1000, resultSchema, Nil) val resultRows = resultObs.toListL.runAsync.futureValue.flatMap(_.rows.map(_.getDouble(1)).toList) resultRows.isEmpty shouldEqual(true) } @@ -138,7 +139,7 @@ class SortFunctionSpec extends FunSpec with Matchers with ScalaFutures { resultAgg.flatMap(_.rows.map(_.getDouble(1)).toList) shouldEqual(List(5.0, 1.0)) val sortFunctionMapper = exec.SortFunctionMapper(SortFunctionId.Sort) - val resultObs = sortFunctionMapper(resultObs2, queryConfig, 1000, resultSchema, Nil) + val resultObs = sortFunctionMapper(resultObs2, querySession, 1000, resultSchema, Nil) val resultRows = resultObs.toListL.runAsync.futureValue.flatMap(_.rows.map(_.getDouble(1)).toList) resultRows.shouldEqual(List(1.0, 5.0)) } diff --git a/query/src/test/scala/filodb/query/util/LogicalPlanUtilSpec.scala b/query/src/test/scala/filodb/query/util/LogicalPlanUtilSpec.scala deleted file mode 100644 index 398f800356..0000000000 --- a/query/src/test/scala/filodb/query/util/LogicalPlanUtilSpec.scala +++ /dev/null @@ -1,76 +0,0 @@ -package filodb.query.util - -import org.scalatest.{FunSpec, Matchers} - -import filodb.core.query.ColumnFilter -import filodb.core.query.Filter.{Equals, In} -import filodb.query._ -import filodb.query.BinaryOperator.DIV -import filodb.query.Cardinality.OneToOne -import filodb.query.RangeFunctionId.SumOverTime - -class LogicalPlanUtilSpec extends FunSpec with Matchers { - - it("should get MetricName from logicalPlan") { - - val rawSeries = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("_name_", Equals("MetricName")), - ColumnFilter("instance", Equals("Inst-0"))), Seq("_name_", "instance"), Some(300000), None) - val periodicSeriesWithWindowing = PeriodicSeriesWithWindowing(rawSeries, 1000, 500, 5000, 100, SumOverTime) - - val res = LogicalPlanUtil.getLabelValueFromLogicalPlan(periodicSeriesWithWindowing, "_name_") - res.get.shouldEqual(Set("MetricName")) - } - - it("should get LabelName from logicalPlan with filter In") { - - val rawSeries = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("_name_", Equals("MetricName")), - ColumnFilter("instance", In(Set("Inst-0", "Inst-1")))), Seq("_name_", "instance"), Some(300000), None) - val periodicSeriesWithWindowing = PeriodicSeriesWithWindowing(rawSeries, 1000, 500, 5000, 100, SumOverTime) - - val res = LogicalPlanUtil.getLabelValueFromLogicalPlan(periodicSeriesWithWindowing, "instance") - res.get.shouldEqual(Set("Inst-0", "Inst-1")) - } - - it("should get MetricName from BinaryJoin LogicalPlan") { - - val rawSeriesLhs = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("_name_", Equals("MetricName1")), - ColumnFilter("instance", Equals("Inst-0"))), Seq("_name_", "instance"), Some(300000), None) - val lhs = PeriodicSeries(rawSeriesLhs, 1000, 500, 50000) - - val rawSeriesRhs = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("job", Equals("MetricName2")), - ColumnFilter("instance", Equals("Inst-1"))), Seq("job", "instance"), Some(300000), None) - val rhs = PeriodicSeries(rawSeriesRhs, 1000, 500, 50000) - - val binaryJoin = BinaryJoin(lhs, DIV, OneToOne, rhs) - - val res = LogicalPlanUtil.getLabelValueFromLogicalPlan(binaryJoin, "_name_") - res.get.shouldEqual(Set("MetricName1")) - } - - it("should return None if label value is not present in logicalPlan") { - - val rawSeries = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("_name_", Equals("MetricName")), - ColumnFilter("instance", Equals("Inst-0"))), Seq("_name_", "instance"), Some(300000), None) - val periodicSeriesWithWindowing = PeriodicSeriesWithWindowing(rawSeries, 1000, 500, 5000, 100, SumOverTime) - - val res = LogicalPlanUtil.getLabelValueFromLogicalPlan(periodicSeriesWithWindowing, "_name") - res.isEmpty shouldEqual(true) - } - - it("should concatenate results from lhs and rhs for BinaryJoin LogicalPlan") { - - val rawSeriesLhs = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("_name_", Equals("MetricName1")), - ColumnFilter("instance", Equals("Inst-0"))), Seq("_name_", "instance"), Some(300000), None) - val lhs = PeriodicSeries(rawSeriesLhs, 1000, 500, 50000) - - val rawSeriesRhs = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("job", Equals("MetricName2")), - ColumnFilter("instance", Equals("Inst-1"))), Seq("job", "instance"), Some(300000), None) - val rhs = PeriodicSeries(rawSeriesRhs, 1000, 500, 50000) - - val binaryJoin = BinaryJoin(lhs, DIV, OneToOne, rhs) - - val res = LogicalPlanUtil.getLabelValueFromLogicalPlan(binaryJoin, "instance") - res.get.shouldEqual(Set("Inst-1", "Inst-0")) - } - -} diff --git a/run_benchmarks.sh b/run_benchmarks.sh index 01e72d416e..cdbe31c166 100755 --- a/run_benchmarks.sh +++ b/run_benchmarks.sh @@ -1,6 +1,7 @@ #!/bin/bash sbt "jmh/jmh:run -rf json -i 15 -wi 10 -f3 -jvmArgsAppend -XX:MaxInlineLevel=20 \ -jvmArgsAppend -Xmx4g -jvmArgsAppend -XX:MaxInlineSize=99 \ + -prof jmh.extras.JFR:dir=/tmp/filo-jmh \ filodb.jmh.QueryHiCardInMemoryBenchmark \ filodb.jmh.QueryInMemoryBenchmark \ filodb.jmh.QueryAndIngestBenchmark \ diff --git a/spark-jobs/src/main/scala/filodb/cardbuster/PerShardCardinalityBuster.scala b/spark-jobs/src/main/scala/filodb/cardbuster/PerShardCardinalityBuster.scala index f3fb20d796..da32df6a53 100644 --- a/spark-jobs/src/main/scala/filodb/cardbuster/PerShardCardinalityBuster.scala +++ b/spark-jobs/src/main/scala/filodb/cardbuster/PerShardCardinalityBuster.scala @@ -1,5 +1,8 @@ package filodb.cardbuster +import java.time.Instant +import java.time.format.DateTimeFormatter + import scala.concurrent.Await import kamon.Kamon @@ -37,21 +40,54 @@ class PerShardCardinalityBuster(dsSettings: DownsamplerSettings, @transient lazy val deleteFilter = dsSettings.filodbConfig .as[Seq[Map[String, String]]]("cardbuster.delete-pk-filters").map(_.toSeq) + @transient lazy val startTimeGTE = dsSettings.filodbConfig + .as[Option[String]]("cardbuster.delete-startTimeGTE").map { str => + Instant.from(DateTimeFormatter.ISO_OFFSET_DATE_TIME.parse(str)).toEpochMilli + } + @transient lazy val startTimeLTE = dsSettings.filodbConfig + .as[Option[String]]("cardbuster.delete-startTimeLTE").map { str => + Instant.from(DateTimeFormatter.ISO_OFFSET_DATE_TIME.parse(str)).toEpochMilli + } + @transient lazy val endTimeGTE = dsSettings.filodbConfig + .as[Option[String]]("cardbuster.delete-endTimeGTE").map { str => + Instant.from(DateTimeFormatter.ISO_OFFSET_DATE_TIME.parse(str)).toEpochMilli + } + @transient lazy val endTimeLTE = dsSettings.filodbConfig + .as[Option[String]]("cardbuster.delete-endTimeLTE").map { str => + Instant.from(DateTimeFormatter.ISO_OFFSET_DATE_TIME.parse(str)).toEpochMilli + } def bustIndexRecords(shard: Int): Unit = { - BusterContext.log.info(s"Busting cardinality in shard=$shard with filter=$deleteFilter") - val toDelete = colStore.scanPartKeys(dataset, shard).map(_.partKey) - .filter { pk => - val rawSchemaId = RecordSchema.schemaID(pk, UnsafeUtils.arayOffset) - val schema = schemas(rawSchemaId) - val pkPairs = schema.partKeySchema.toStringPairs(pk, UnsafeUtils.arayOffset) - val willDelete = deleteFilter.exists(filter => filter.forall(pkPairs.contains)) - if (willDelete) { - BusterContext.log.debug(s"Deleting part key ${schema.partKeySchema.stringify(pk)}") - numPartKeysDeleting.increment() + BusterContext.log.info(s"Busting cardinality in shard=$shard with " + + s"filter=$deleteFilter " + + s"inDownsampleTables=$inDownsampleTables " + + s"startTimeGTE=$startTimeGTE " + + s"startTimeLTE=$startTimeLTE " + + s"endTimeGTE=$endTimeGTE " + + s"endTimeLTE=$endTimeLTE " + ) + val toDelete = colStore.scanPartKeys(dataset, shard) + .filter { pkr => + val timeOk = startTimeGTE.forall(pkr.startTime >= _) && + startTimeLTE.forall(pkr.startTime <= _) && + endTimeGTE.forall(pkr.endTime >= _) && + endTimeLTE.forall(pkr.endTime <= _) + + if (timeOk) { + val pk = pkr.partKey + val rawSchemaId = RecordSchema.schemaID(pk, UnsafeUtils.arayOffset) + val schema = schemas(rawSchemaId) + val pkPairs = schema.partKeySchema.toStringPairs(pk, UnsafeUtils.arayOffset) + val willDelete = deleteFilter.exists(filter => filter.forall(pkPairs.contains)) + if (willDelete) { + BusterContext.log.debug(s"Deleting part key ${schema.partKeySchema.stringify(pk)}") + numPartKeysDeleting.increment() + } + willDelete + } else { + false } - willDelete - } + }.map(_.partKey) val fut = colStore.deletePartKeys(dataset, shard, toDelete) val numKeysDeleted = Await.result(fut, dsSettings.cassWriteTimeout) BusterContext.log.info(s"Deleted keys from shard shard=$shard numKeysDeleted=$numKeysDeleted") 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 d9f83c9273..ca69d75bc0 100644 --- a/spark-jobs/src/main/scala/filodb/downsampler/chunk/BatchDownsampler.scala +++ b/spark-jobs/src/main/scala/filodb/downsampler/chunk/BatchDownsampler.scala @@ -259,6 +259,8 @@ class BatchDownsampler(settings: DownsamplerSettings) extends Instance with Seri val timestampCol = 0 val rawChunksets = rawPartToDownsample.infos(AllChunkScan) + require(downsamplers.size > 1, s"Number of downsamplers for ${rawPartToDownsample.stringPartition} should be > 1") + // for each chunk while (rawChunksets.hasNext) { val chunkset = rawChunksets.nextInfoReader diff --git a/spark-jobs/src/main/scala/filodb/downsampler/chunk/DownsamplerMain.scala b/spark-jobs/src/main/scala/filodb/downsampler/chunk/DownsamplerMain.scala index 8f5a5431ab..f6e8632d49 100644 --- a/spark-jobs/src/main/scala/filodb/downsampler/chunk/DownsamplerMain.scala +++ b/spark-jobs/src/main/scala/filodb/downsampler/chunk/DownsamplerMain.scala @@ -7,6 +7,7 @@ import kamon.Kamon import org.apache.spark.SparkConf import org.apache.spark.sql.SparkSession +import filodb.coordinator.KamonShutdownHook import filodb.downsampler.DownsamplerContext /** @@ -100,10 +101,13 @@ class Downsampler(settings: DownsamplerSettings, batchDownsampler: BatchDownsamp DownsamplerContext.dsLogger.info(s"Cassandra split size: ${splits.size}. We will have this many spark " + s"partitions. Tune splitsPerNode which was ${settings.splitsPerNode} if parallelism is low") + KamonShutdownHook.registerShutdownHook() + spark.sparkContext .makeRDD(splits) .mapPartitions { splitIter => - Kamon.init() // kamon init should be first thing in worker jvm + Kamon.init() + KamonShutdownHook.registerShutdownHook() import filodb.core.Iterators._ val rawDataSource = batchDownsampler.rawCassandraColStore val batchReadSpan = Kamon.spanBuilder("cassandra-raw-data-read-latency").start() @@ -117,7 +121,8 @@ class Downsampler(settings: DownsamplerSettings, batchDownsampler: BatchDownsamp batchIter // iterator of batches } .foreach { rawPartsBatch => - Kamon.init() // kamon init should be first thing in worker jvm + Kamon.init() + KamonShutdownHook.registerShutdownHook() batchDownsampler.downsampleBatch(rawPartsBatch, userTimeStart, userTimeEndExclusive) } 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 96b5e205a8..3aa7830e1c 100644 --- a/spark-jobs/src/main/scala/filodb/downsampler/index/DSIndexJob.scala +++ b/spark-jobs/src/main/scala/filodb/downsampler/index/DSIndexJob.scala @@ -76,7 +76,7 @@ class DSIndexJob(dsSettings: DownsamplerSettings, shard = shard.toInt, updateHour = epochHour) count += migrateWithDownsamplePartKeys(partKeys, shard) } - DownsamplerContext.dsLogger.info(s"Successfully Completed Full PartKey Migration for shard=$shard " + + DownsamplerContext.dsLogger.info(s"Successfully Completed Partial PartKey Migration for shard=$shard " + s"count=$count fromHour=$fromHour toHourExcl=$toHourExcl") } sparkForeachTasksCompleted.increment() diff --git a/spark-jobs/src/main/scala/filodb/downsampler/index/DSIndexJobMain.scala b/spark-jobs/src/main/scala/filodb/downsampler/index/DSIndexJobMain.scala index 1200dffe4e..f889599255 100644 --- a/spark-jobs/src/main/scala/filodb/downsampler/index/DSIndexJobMain.scala +++ b/spark-jobs/src/main/scala/filodb/downsampler/index/DSIndexJobMain.scala @@ -7,6 +7,7 @@ import kamon.Kamon import org.apache.spark.SparkConf import org.apache.spark.sql.SparkSession +import filodb.coordinator.KamonShutdownHook import filodb.downsampler.DownsamplerContext import filodb.downsampler.chunk.DownsamplerSettings @@ -94,13 +95,16 @@ class IndexJobDriver(dsSettings: DownsamplerSettings, dsIndexJobSettings: DSInde val numShards = dsIndexJobSettings.numShards + KamonShutdownHook.registerShutdownHook() + DownsamplerContext.dsLogger.info(s"Spark Job Properties: ${spark.sparkContext.getConf.toDebugString}") val startHour = fromHour val endHourExcl = toHourExcl spark.sparkContext .makeRDD(0 until numShards) .foreach { shard => - Kamon.init() // kamon init should be first thing in worker jvm + Kamon.init() + KamonShutdownHook.registerShutdownHook() job.updateDSPartKeyIndex(shard, startHour, endHourExcl, doFullMigration) } DownsamplerContext.dsLogger.info(s"IndexUpdater Driver completed successfully") diff --git a/spark-jobs/src/test/scala/filodb/downsampler/DownsamplerMainSpec.scala b/spark-jobs/src/test/scala/filodb/downsampler/DownsamplerMainSpec.scala index eb81798f80..4d768ebae9 100644 --- a/spark-jobs/src/test/scala/filodb/downsampler/DownsamplerMainSpec.scala +++ b/spark-jobs/src/test/scala/filodb/downsampler/DownsamplerMainSpec.scala @@ -22,7 +22,7 @@ import filodb.core.downsample.DownsampledTimeSeriesStore import filodb.core.memstore.{PagedReadablePartition, TimeSeriesPartition} import filodb.core.memstore.FiloSchedulers.QuerySchedName import filodb.core.metadata.{Dataset, Schemas} -import filodb.core.query.{ColumnFilter, CustomRangeVectorKey, QueryContext, RawDataRangeVector} +import filodb.core.query._ import filodb.core.query.Filter.Equals import filodb.core.store.{AllChunkScan, PartKeyRecord, SinglePartitionScan, StoreConfig} import filodb.downsampler.chunk.{BatchDownsampler, Downsampler, DownsamplerSettings, OffHeapMemory} @@ -30,7 +30,7 @@ import filodb.downsampler.index.{DSIndexJobSettings, IndexJobDriver} import filodb.memory.format.{PrimitiveVectorReader, UnsafeUtils} import filodb.memory.format.ZeroCopyUTF8String._ import filodb.memory.format.vectors.{CustomBuckets, LongHistogram} -import filodb.query.{QueryConfig, QueryResult} +import filodb.query.QueryResult import filodb.query.exec.{InProcessPlanDispatcher, MultiSchemaPartitionsExec} /** @@ -44,6 +44,7 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w val conf = ConfigFactory.parseFile(new File("conf/timeseries-filodb-server.conf")) val settings = new DownsamplerSettings(conf) + val queryConfig = new QueryConfig(settings.filodbConfig.getConfig("query")) val dsIndexJobSettings = new DSIndexJobSettings(settings) val batchDownsampler = new BatchDownsampler(settings) @@ -77,7 +78,7 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w val gaugeLowFreqName = "my_gauge_low_freq" var gaugeLowFreqPartKeyBytes: Array[Byte] = _ - val lastSampleTime = 1574373042000L + val lastSampleTime = 74373042000L val pkUpdateHour = hour(lastSampleTime) val metricNames = Seq(gaugeName, gaugeLowFreqName, counterName, histName, untypedName) @@ -111,20 +112,20 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w untypedPartKeyBytes = part.partKeyBytes val rawSamples = Stream( - Seq(1574372801000L, 3d, untypedName, seriesTags), - Seq(1574372802000L, 5d, untypedName, seriesTags), + Seq(74372801000L, 3d, untypedName, seriesTags), + Seq(74372802000L, 5d, untypedName, seriesTags), - Seq(1574372861000L, 9d, untypedName, seriesTags), - Seq(1574372862000L, 11d, untypedName, seriesTags), + Seq(74372861000L, 9d, untypedName, seriesTags), + Seq(74372862000L, 11d, untypedName, seriesTags), - Seq(1574372921000L, 13d, untypedName, seriesTags), - Seq(1574372922000L, 15d, untypedName, seriesTags), + Seq(74372921000L, 13d, untypedName, seriesTags), + Seq(74372922000L, 15d, untypedName, seriesTags), - Seq(1574372981000L, 17d, untypedName, seriesTags), - Seq(1574372982000L, 15d, untypedName, seriesTags), + Seq(74372981000L, 17d, untypedName, seriesTags), + Seq(74372982000L, 15d, untypedName, seriesTags), - Seq(1574373041000L, 13d, untypedName, seriesTags), - Seq(1574373042000L, 11d, untypedName, seriesTags) + Seq(74373041000L, 13d, untypedName, seriesTags), + Seq(74373042000L, 11d, untypedName, seriesTags) ) MachineMetricsData.records(rawDataset, rawSamples).records.foreach { case (base, offset) => @@ -135,7 +136,7 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w val chunks = part.makeFlushChunks(offheapMem.blockMemFactory) rawColStore.write(rawDataset.ref, Observable.fromIterator(chunks)).futureValue - val pk = PartKeyRecord(untypedPartKeyBytes, 1574372801000L, 1574373042000L, Some(150)) + val pk = PartKeyRecord(untypedPartKeyBytes, 74372801000L, 74373042000L, Some(150)) rawColStore.writePartKeys(rawDataset.ref, 0, Observable.now(pk), 259200, pkUpdateHour).futureValue } @@ -153,20 +154,20 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w gaugePartKeyBytes = part.partKeyBytes val rawSamples = Stream( - Seq(1574372801000L, 3d, gaugeName, seriesTags), - Seq(1574372802000L, 5d, gaugeName, seriesTags), + Seq(74372801000L, 3d, gaugeName, seriesTags), + Seq(74372802000L, 5d, gaugeName, seriesTags), - Seq(1574372861000L, 9d, gaugeName, seriesTags), - Seq(1574372862000L, 11d, gaugeName, seriesTags), + Seq(74372861000L, 9d, gaugeName, seriesTags), + Seq(74372862000L, 11d, gaugeName, seriesTags), - Seq(1574372921000L, 13d, gaugeName, seriesTags), - Seq(1574372922000L, 15d, gaugeName, seriesTags), + Seq(74372921000L, 13d, gaugeName, seriesTags), + Seq(74372922000L, 15d, gaugeName, seriesTags), - Seq(1574372981000L, 17d, gaugeName, seriesTags), - Seq(1574372982000L, 15d, gaugeName, seriesTags), + Seq(74372981000L, 17d, gaugeName, seriesTags), + Seq(74372982000L, 15d, gaugeName, seriesTags), - Seq(1574373041000L, 13d, gaugeName, seriesTags), - Seq(1574373042000L, 11d, gaugeName, seriesTags) + Seq(74373041000L, 13d, gaugeName, seriesTags), + Seq(74373042000L, 11d, gaugeName, seriesTags) ) MachineMetricsData.records(rawDataset, rawSamples).records.foreach { case (base, offset) => @@ -177,7 +178,7 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w val chunks = part.makeFlushChunks(offheapMem.blockMemFactory) rawColStore.write(rawDataset.ref, Observable.fromIterator(chunks)).futureValue - val pk = PartKeyRecord(gaugePartKeyBytes, 1574372801000L, 1574373042000L, Some(150)) + val pk = PartKeyRecord(gaugePartKeyBytes, 74372801000L, 74373042000L, Some(150)) rawColStore.writePartKeys(rawDataset.ref, 0, Observable.now(pk), 259200, pkUpdateHour).futureValue } @@ -195,18 +196,18 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w gaugeLowFreqPartKeyBytes = part.partKeyBytes val rawSamples = Stream( - Seq(1574372801000L, 3d, gaugeName, seriesTags), - Seq(1574372802000L, 5d, gaugeName, seriesTags), + Seq(74372801000L, 3d, gaugeName, seriesTags), + Seq(74372802000L, 5d, gaugeName, seriesTags), // skip next minute - Seq(1574372921000L, 13d, gaugeName, seriesTags), - Seq(1574372922000L, 15d, gaugeName, seriesTags), + Seq(74372921000L, 13d, gaugeName, seriesTags), + Seq(74372922000L, 15d, gaugeName, seriesTags), // skip next minute - Seq(1574373041000L, 13d, gaugeName, seriesTags), - Seq(1574373042000L, 11d, gaugeName, seriesTags) + Seq(74373041000L, 13d, gaugeName, seriesTags), + Seq(74373042000L, 11d, gaugeName, seriesTags) ) MachineMetricsData.records(rawDataset, rawSamples).records.foreach { case (base, offset) => @@ -217,7 +218,7 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w val chunks = part.makeFlushChunks(offheapMem.blockMemFactory) rawColStore.write(rawDataset.ref, Observable.fromIterator(chunks)).futureValue - val pk = PartKeyRecord(gaugeLowFreqPartKeyBytes, 1574372801000L, 1574373042000L, Some(150)) + val pk = PartKeyRecord(gaugeLowFreqPartKeyBytes, 74372801000L, 74373042000L, Some(150)) rawColStore.writePartKeys(rawDataset.ref, 0, Observable.now(pk), 259200, pkUpdateHour).futureValue } @@ -235,24 +236,24 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w counterPartKeyBytes = part.partKeyBytes val rawSamples = Stream( - Seq(1574372801000L, 3d, counterName, seriesTags), - Seq(1574372801500L, 4d, counterName, seriesTags), - Seq(1574372802000L, 5d, counterName, seriesTags), + Seq(74372801000L, 3d, counterName, seriesTags), + Seq(74372801500L, 4d, counterName, seriesTags), + Seq(74372802000L, 5d, counterName, seriesTags), - Seq(1574372861000L, 9d, counterName, seriesTags), - Seq(1574372861500L, 10d, counterName, seriesTags), - Seq(1574372862000L, 11d, counterName, seriesTags), + Seq(74372861000L, 9d, counterName, seriesTags), + Seq(74372861500L, 10d, counterName, seriesTags), + Seq(74372862000L, 11d, counterName, seriesTags), - Seq(1574372921000L, 2d, counterName, seriesTags), - Seq(1574372921500L, 7d, counterName, seriesTags), - Seq(1574372922000L, 15d, counterName, seriesTags), + Seq(74372921000L, 2d, counterName, seriesTags), + Seq(74372921500L, 7d, counterName, seriesTags), + Seq(74372922000L, 15d, counterName, seriesTags), - Seq(1574372981000L, 17d, counterName, seriesTags), - Seq(1574372981500L, 1d, counterName, seriesTags), - Seq(1574372982000L, 15d, counterName, seriesTags), + Seq(74372981000L, 17d, counterName, seriesTags), + Seq(74372981500L, 1d, counterName, seriesTags), + Seq(74372982000L, 15d, counterName, seriesTags), - Seq(1574373041000L, 18d, counterName, seriesTags), - Seq(1574373042000L, 20d, counterName, seriesTags) + Seq(74373041000L, 18d, counterName, seriesTags), + Seq(74373042000L, 20d, counterName, seriesTags) ) MachineMetricsData.records(rawDataset, rawSamples).records.foreach { case (base, offset) => @@ -263,7 +264,7 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w val chunks = part.makeFlushChunks(offheapMem.blockMemFactory) rawColStore.write(rawDataset.ref, Observable.fromIterator(chunks)).futureValue - val pk = PartKeyRecord(counterPartKeyBytes, 1574372801000L, 1574373042000L, Some(1)) + val pk = PartKeyRecord(counterPartKeyBytes, 74372801000L, 74373042000L, Some(1)) rawColStore.writePartKeys(rawDataset.ref, 0, Observable.now(pk), 259200, pkUpdateHour).futureValue } @@ -282,24 +283,24 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w val bucketScheme = CustomBuckets(Array(3d, 10d, Double.PositiveInfinity)) val rawSamples = Stream( // time, sum, count, hist, name, tags - Seq(1574372801000L, 0d, 1d, LongHistogram(bucketScheme, Array(0L, 0, 1)), histName, seriesTags), - Seq(1574372801500L, 2d, 3d, LongHistogram(bucketScheme, Array(0L, 2, 3)), histName, seriesTags), - Seq(1574372802000L, 5d, 6d, LongHistogram(bucketScheme, Array(2L, 5, 6)), histName, seriesTags), + Seq(74372801000L, 0d, 1d, LongHistogram(bucketScheme, Array(0L, 0, 1)), histName, seriesTags), + Seq(74372801500L, 2d, 3d, LongHistogram(bucketScheme, Array(0L, 2, 3)), histName, seriesTags), + Seq(74372802000L, 5d, 6d, LongHistogram(bucketScheme, Array(2L, 5, 6)), histName, seriesTags), - Seq(1574372861000L, 9d, 9d, LongHistogram(bucketScheme, Array(2L, 5, 9)), histName, seriesTags), - Seq(1574372861500L, 10d, 10d, LongHistogram(bucketScheme, Array(2L, 5, 10)), histName, seriesTags), - Seq(1574372862000L, 11d, 14d, LongHistogram(bucketScheme, Array(2L, 8, 14)), histName, seriesTags), + Seq(74372861000L, 9d, 9d, LongHistogram(bucketScheme, Array(2L, 5, 9)), histName, seriesTags), + Seq(74372861500L, 10d, 10d, LongHistogram(bucketScheme, Array(2L, 5, 10)), histName, seriesTags), + Seq(74372862000L, 11d, 14d, LongHistogram(bucketScheme, Array(2L, 8, 14)), histName, seriesTags), - Seq(1574372921000L, 2d, 2d, LongHistogram(bucketScheme, Array(0L, 0, 2)), histName, seriesTags), - Seq(1574372921500L, 7d, 9d, LongHistogram(bucketScheme, Array(1L, 7, 9)), histName, seriesTags), - Seq(1574372922000L, 15d, 19d, LongHistogram(bucketScheme, Array(1L, 15, 19)), histName, seriesTags), + Seq(74372921000L, 2d, 2d, LongHistogram(bucketScheme, Array(0L, 0, 2)), histName, seriesTags), + Seq(74372921500L, 7d, 9d, LongHistogram(bucketScheme, Array(1L, 7, 9)), histName, seriesTags), + Seq(74372922000L, 15d, 19d, LongHistogram(bucketScheme, Array(1L, 15, 19)), histName, seriesTags), - Seq(1574372981000L, 17d, 21d, LongHistogram(bucketScheme, Array(2L, 16, 21)), histName, seriesTags), - Seq(1574372981500L, 1d, 1d, LongHistogram(bucketScheme, Array(0L, 1, 1)), histName, seriesTags), - Seq(1574372982000L, 15d, 15d, LongHistogram(bucketScheme, Array(0L, 15, 15)), histName, seriesTags), + Seq(74372981000L, 17d, 21d, LongHistogram(bucketScheme, Array(2L, 16, 21)), histName, seriesTags), + Seq(74372981500L, 1d, 1d, LongHistogram(bucketScheme, Array(0L, 1, 1)), histName, seriesTags), + Seq(74372982000L, 15d, 15d, LongHistogram(bucketScheme, Array(0L, 15, 15)), histName, seriesTags), - Seq(1574373041000L, 18d, 19d, LongHistogram(bucketScheme, Array(1L, 16, 19)), histName, seriesTags), - Seq(1574373042000L, 20d, 25d, LongHistogram(bucketScheme, Array(4L, 20, 25)), histName, seriesTags) + Seq(74373041000L, 18d, 19d, LongHistogram(bucketScheme, Array(1L, 16, 19)), histName, seriesTags), + Seq(74373042000L, 20d, 25d, LongHistogram(bucketScheme, Array(4L, 20, 25)), histName, seriesTags) ) MachineMetricsData.records(rawDataset, rawSamples).records.foreach { case (base, offset) => @@ -310,7 +311,7 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w val chunks = part.makeFlushChunks(offheapMem.blockMemFactory) rawColStore.write(rawDataset.ref, Observable.fromIterator(chunks)).futureValue - val pk = PartKeyRecord(histPartKeyBytes, 1574372801000L, 1574373042000L, Some(199)) + val pk = PartKeyRecord(histPartKeyBytes, 74372801000L, 74373042000L, Some(199)) rawColStore.writePartKeys(rawDataset.ref, 0, Observable.now(pk), 259200, pkUpdateHour).futureValue } @@ -320,15 +321,16 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w start until start + 6 } - it("should simulate bulk part key records being written for migration") { + it("should simulate bulk part key records being written into raw for migration") { val partBuilder = new RecordBuilder(offheapMem.nativeMemoryManager) val schemas = Seq(Schemas.promHistogram, Schemas.gauge, Schemas.promCounter) case class PkToWrite(pkr: PartKeyRecord, shard: Int, updateHour: Long) val pks = for { i <- 0 to 10000 } yield { val schema = schemas(i % schemas.size) - val partKey = partBuilder.partKeyFromObjects(schema, s"metric$i", bulkSeriesTags) + val partKey = partBuilder.partKeyFromObjects(schema, s"bulkmetric$i", bulkSeriesTags) val bytes = schema.partKeySchema.asByteArray(UnsafeUtils.ZeroPointer, partKey) - PkToWrite(PartKeyRecord(bytes, 0L, 1000L, Some(-i)), i % numShards, bulkPkUpdateHours(i % bulkPkUpdateHours.size)) + PkToWrite(PartKeyRecord(bytes, i, i + 500, Some(-i)), i % numShards, + bulkPkUpdateHours(i % bulkPkUpdateHours.size)) } val rawDataset = Dataset("prometheus", Schemas.promHistogram) @@ -390,7 +392,7 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w }.toSet // readKeys should not contain untyped part key - we dont downsample untyped - readKeys shouldEqual (0 to 10000).map(i => s"metric$i").toSet ++ (metricNames.toSet - untypedName) + readKeys shouldEqual (0 to 10000).map(i => s"bulkmetric$i").toSet ++ (metricNames.toSet - untypedName) } it("should read and verify gauge data in cassandra using PagedReadablePartition for 1-min downsampled data") { @@ -414,14 +416,19 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w // time, min, max, sum, count, avg downsampledData1 shouldEqual Seq( - (1574372802000L, 3.0, 5.0, 8.0, 2.0, 4.0), - (1574372862000L, 9.0, 11.0, 20.0, 2.0, 10.0), - (1574372922000L, 13.0, 15.0, 28.0, 2.0, 14.0), - (1574372982000L, 15.0, 17.0, 32.0, 2.0, 16.0), - (1574373042000L, 11.0, 13.0, 24.0, 2.0, 12.0) + (74372802000L, 3.0, 5.0, 8.0, 2.0, 4.0), + (74372862000L, 9.0, 11.0, 20.0, 2.0, 10.0), + (74372922000L, 13.0, 15.0, 28.0, 2.0, 14.0), + (74372982000L, 15.0, 17.0, 32.0, 2.0, 16.0), + (74373042000L, 11.0, 13.0, 24.0, 2.0, 12.0) ) } + /* + Tip: After running this spec, you can bring up the local downsample server and hit following URL on browser + http://localhost:9080/promql/prometheus/api/v1/query_range?query=my_gauge%7B_ws_%3D%27my_ws%27%2C_ns_%3D%27my_ns%27%7D&start=74372801&end=74373042&step=10&verbose=true&spread=2 + */ + it("should read and verify low freq gauge in cassandra using PagedReadablePartition for 1-min downsampled data") { val dsGaugeLowFreqPartKeyBytes = RecordBuilder.buildDownsamplePartKey(gaugeLowFreqPartKeyBytes, @@ -444,9 +451,9 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w // time, min, max, sum, count, avg downsampledData1 shouldEqual Seq( - (1574372802000L, 3.0, 5.0, 8.0, 2.0, 4.0), - (1574372922000L, 13.0, 15.0, 28.0, 2.0, 14.0), - (1574373042000L, 11.0, 13.0, 24.0, 2.0, 12.0) + (74372802000L, 3.0, 5.0, 8.0, 2.0, 4.0), + (74372922000L, 13.0, 15.0, 28.0, 2.0, 14.0), + (74373042000L, 11.0, 13.0, 24.0, 2.0, 12.0) ) } @@ -473,19 +480,19 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w // time, counter downsampledData1 shouldEqual Seq( - (1574372801000L, 3d), - (1574372802000L, 5d), + (74372801000L, 3d), + (74372802000L, 5d), - (1574372862000L, 11d), + (74372862000L, 11d), - (1574372921000L, 2d), - (1574372922000L, 15d), + (74372921000L, 2d), + (74372922000L, 15d), - (1574372981000L, 17d), - (1574372981500L, 1d), - (1574372982000L, 15d), + (74372981000L, 17d), + (74372981500L, 1d), + (74372982000L, 15d), - (1574373042000L, 20d) + (74373042000L, 20d) ) } @@ -518,19 +525,19 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w // time, sum, count, histogram downsampledData1 shouldEqual Seq( - (1574372801000L, 0d, 1d, Seq(0d, 0d, 1d)), - (1574372802000L, 5d, 6d, Seq(2d, 5d, 6d)), + (74372801000L, 0d, 1d, Seq(0d, 0d, 1d)), + (74372802000L, 5d, 6d, Seq(2d, 5d, 6d)), - (1574372862000L, 11d, 14d, Seq(2d, 8d, 14d)), + (74372862000L, 11d, 14d, Seq(2d, 8d, 14d)), - (1574372921000L, 2d, 2d, Seq(0d, 0d, 2d)), - (1574372922000L, 15d, 19d, Seq(1d, 15d, 19d)), + (74372921000L, 2d, 2d, Seq(0d, 0d, 2d)), + (74372922000L, 15d, 19d, Seq(1d, 15d, 19d)), - (1574372981000L, 17d, 21d, Seq(2d, 16d, 21d)), - (1574372981500L, 1d, 1d, Seq(0d, 1d, 1d)), - (1574372982000L, 15d, 15d, Seq(0d, 15d, 15d)), + (74372981000L, 17d, 21d, Seq(2d, 16d, 21d)), + (74372981500L, 1d, 1d, Seq(0d, 1d, 1d)), + (74372982000L, 15d, 15d, Seq(0d, 15d, 15d)), - (1574373042000L, 20d, 25d, Seq(4d, 20d, 25d)) + (74373042000L, 20d, 25d, Seq(4d, 20d, 25d)) ) } @@ -554,8 +561,8 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w // time, min, max, sum, count, avg downsampledData2 shouldEqual Seq( - (1574372982000L, 3.0, 17.0, 88.0, 8.0, 11.0), - (1574373042000L, 11.0, 13.0, 24.0, 2.0, 12.0) + (74372982000L, 3.0, 17.0, 88.0, 8.0, 11.0), + (74373042000L, 11.0, 13.0, 24.0, 2.0, 12.0) ) } @@ -582,18 +589,18 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w // time, counter downsampledData1 shouldEqual Seq( - (1574372801000L, 3d), + (74372801000L, 3d), - (1574372862000L, 11d), + (74372862000L, 11d), - (1574372921000L, 2d), + (74372921000L, 2d), - (1574372981000L, 17d), - (1574372981500L, 1d), + (74372981000L, 17d), + (74372981500L, 1d), - (1574372982000L, 15.0d), + (74372982000L, 15.0d), - (1574373042000L, 20.0d) + (74373042000L, 20.0d) ) } @@ -625,13 +632,13 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w // time, sum, count, histogram downsampledData1 shouldEqual Seq( - (1574372801000L, 0d, 1d, Seq(0d, 0d, 1d)), - (1574372862000L, 11d, 14d, Seq(2d, 8d, 14d)), - (1574372921000L, 2d, 2d, Seq(0d, 0d, 2d)), - (1574372981000L, 17d, 21d, Seq(2d, 16d, 21d)), - (1574372981500L, 1d, 1d, Seq(0d, 1d, 1d)), - (1574372982000L, 15.0d, 15.0d, Seq(0.0, 15.0, 15.0)), - (1574373042000L, 20.0d, 25.0d, Seq(4.0, 20.0, 25.0)) + (74372801000L, 0d, 1d, Seq(0d, 0d, 1d)), + (74372862000L, 11d, 14d, Seq(2d, 8d, 14d)), + (74372921000L, 2d, 2d, Seq(0d, 0d, 2d)), + (74372981000L, 17d, 21d, Seq(2d, 16d, 21d)), + (74372981500L, 1d, 1d, Seq(0d, 1d, 1d)), + (74372982000L, 15.0d, 15.0d, Seq(0.0, 15.0, 15.0)), + (74373042000L, 20.0d, 25.0d, Seq(4.0, 20.0, 25.0)) ) } @@ -652,9 +659,9 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w val exec = MultiSchemaPartitionsExec(QueryContext(sampleLimit = 1000), InProcessPlanDispatcher, batchDownsampler.rawDatasetRef, 0, queryFilters, AllChunkScan) - val queryConfig = new QueryConfig(settings.filodbConfig.getConfig("query")) + val querySession = QuerySession(QueryContext(), queryConfig) val queryScheduler = Scheduler.fixedPool(s"$QuerySchedName", 3) - val res = exec.execute(downsampleTSStore, queryConfig)(queryScheduler) + val res = exec.execute(downsampleTSStore, querySession)(queryScheduler) .runAsync(queryScheduler).futureValue.asInstanceOf[QueryResult] queryScheduler.shutdown() @@ -680,9 +687,9 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w val exec = MultiSchemaPartitionsExec(QueryContext(sampleLimit = 1000), InProcessPlanDispatcher, batchDownsampler.rawDatasetRef, 0, queryFilters, AllChunkScan) - val queryConfig = new QueryConfig(settings.filodbConfig.getConfig("query")) + val querySession = QuerySession(QueryContext(), queryConfig) val queryScheduler = Scheduler.fixedPool(s"$QuerySchedName", 3) - val res = exec.execute(downsampleTSStore, queryConfig)(queryScheduler) + val res = exec.execute(downsampleTSStore, querySession)(queryScheduler) .runAsync(queryScheduler).futureValue.asInstanceOf[QueryResult] queryScheduler.shutdown() @@ -702,14 +709,14 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w val exec = MultiSchemaPartitionsExec(QueryContext(sampleLimit = 1000), InProcessPlanDispatcher, batchDownsampler.rawDatasetRef, 0, queryFilters, AllChunkScan, colName = Option("sum")) - val queryConfig = new QueryConfig(settings.filodbConfig.getConfig("query")) + val querySession = QuerySession(QueryContext(), queryConfig) val queryScheduler = Scheduler.fixedPool(s"$QuerySchedName", 3) - val res = exec.execute(downsampleTSStore, queryConfig)(queryScheduler) + val res = exec.execute(downsampleTSStore, querySession)(queryScheduler) .runAsync(queryScheduler).futureValue.asInstanceOf[QueryResult] queryScheduler.shutdown() res.result.size shouldEqual 1 res.result.head.rows.map(r => (r.getLong(0), r.getDouble(1))).toList shouldEqual - List((1574372982000L, 88.0), (1574373042000L, 24.0)) + List((74372982000L, 88.0), (74373042000L, 24.0)) } it ("should fail when cardinality buster is not configured with any delete filters") { @@ -727,7 +734,56 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w .contains("No configuration setting found for key 'cardbuster'") shouldEqual true } - it ("should be able to bust cardinality in raw and downsample tables with spark job") { + it ("should be able to bust cardinality by time filter in downsample tables with spark job") { + val sparkConf = new SparkConf(loadDefaults = true) + sparkConf.setMaster("local[2]") + val deleteFilterConfig = ConfigFactory.parseString( + s""" + |filodb.cardbuster.delete-pk-filters = [ + | { + | _ns_ = "bulk_ns" + | _ws_ = "bulk_ws" + | } + |] + |filodb.cardbuster.delete-startTimeGTE = "${Instant.ofEpochMilli(0).toString}" + |filodb.cardbuster.delete-endTimeLTE = "${Instant.ofEpochMilli(600).toString}" + |""".stripMargin) + + val settings2 = new DownsamplerSettings(deleteFilterConfig.withFallback(conf)) + val dsIndexJobSettings2 = new DSIndexJobSettings(settings2) + val cardBuster = new CardinalityBuster(settings2, dsIndexJobSettings2) + cardBuster.run(sparkConf).close() + + sparkConf.set("spark.filodb.cardbuster.inDownsampleTables", "true") + cardBuster.run(sparkConf).close() + } + + it("should verify bulk part key records are absent after card busting by time filter in downsample tables") { + + def pkMetricName(pkr: PartKeyRecord): String = { + val strPairs = batchDownsampler.schemas.part.binSchema.toStringPairs(pkr.partKey, UnsafeUtils.arayOffset) + strPairs.find(p => p._1 == "_metric_").head._2 + } + + val readKeys = (0 until 4).flatMap { shard => + val partKeys = downsampleColStore.scanPartKeys(batchDownsampler.downsampleRefsByRes(FiniteDuration(5, "min")), + shard) + Await.result(partKeys.map(pkMetricName).toListL.runAsync, 1 minutes) + }.toSet + + // downsample set should not have a few bulk metrics + readKeys.size shouldEqual 9904 + + val readKeys2 = (0 until 4).flatMap { shard => + val partKeys = rawColStore.scanPartKeys(batchDownsampler.rawDatasetRef, shard) + Await.result(partKeys.map(pkMetricName).toListL.runAsync, 1 minutes) + }.toSet + + // raw set should remain same since inDownsampleTables=true in + readKeys2.size shouldEqual 10006 + } + + it ("should be able to bust cardinality in both raw and downsample tables with spark job") { val sparkConf = new SparkConf(loadDefaults = true) sparkConf.setMaster("local[2]") val deleteFilterConfig = ConfigFactory.parseString( """ @@ -741,8 +797,11 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w val settings2 = new DownsamplerSettings(deleteFilterConfig.withFallback(conf)) val dsIndexJobSettings2 = new DSIndexJobSettings(settings2) val cardBuster = new CardinalityBuster(settings2, dsIndexJobSettings2) + + // first run for downsample tables cardBuster.run(sparkConf).close() + // then run for raw tables sparkConf.set("spark.filodb.cardbuster.inDownsampleTables", "false") cardBuster.run(sparkConf).close() } diff --git a/standalone/src/main/scala/filodb.standalone/FiloServer.scala b/standalone/src/main/scala/filodb.standalone/FiloServer.scala index 707c8819e4..975853bc65 100644 --- a/standalone/src/main/scala/filodb.standalone/FiloServer.scala +++ b/standalone/src/main/scala/filodb.standalone/FiloServer.scala @@ -68,6 +68,7 @@ class FiloServer(watcher: Option[ActorRef]) extends FilodbClusterNode { filoHttpServer.start(coordinatorActor, singleton, bootstrapper.getAkkaHttpRoute()) // Launch the profiler after startup, if configured. SimpleProfiler.launch(systemConfig.getConfig("filodb.profiler")) + KamonShutdownHook.registerShutdownHook() } catch { // if there is an error in the initialization, we need to fail fast so that the process can be rescheduled case NonFatal(e) => diff --git a/version.sbt b/version.sbt index e82bdc0d40..f1cfa5fc78 100644 --- a/version.sbt +++ b/version.sbt @@ -1 +1 @@ -version in ThisBuild := "0.9.6" +version in ThisBuild := "0.9.7"