From 503e508a3b4f678c71aa59121ddf8c94263d8a0d Mon Sep 17 00:00:00 2001 From: Alex Theimer Date: Thu, 25 Jan 2024 22:05:29 -0800 Subject: [PATCH] recover/merge old commits --- .../queryplanner/LogicalPlanUtils.scala | 51 +- .../queryplanner/MultiPartitionPlanner.scala | 128 +- .../PartitionLocationPlanner.scala | 53 +- .../queryplanner/QueryUtils.scala | 36 - .../queryplanner/ShardKeyRegexPlanner.scala | 130 +- .../queryplanner/SingleClusterPlanner.scala | 94 +- .../queryplanner/PlannerHierarchySpec.scala | 2056 +++++++---------- .../ShardKeyRegexPlannerSpec.scala | 1057 ++++----- .../SingleClusterPlannerSpec.scala | 16 +- .../scala/filodb.core/query/QueryUtils.scala | 54 + .../filodb.core/query/QueryUtilsSpec.scala | 14 + .../filodb/prometheus/parse/ParserSpec.scala | 31 + 12 files changed, 1725 insertions(+), 1995 deletions(-) delete mode 100644 coordinator/src/main/scala/filodb.coordinator/queryplanner/QueryUtils.scala diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala index 267591f202..55a647ea24 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala @@ -1,12 +1,11 @@ package filodb.coordinator.queryplanner -import scala.collection.{mutable, Seq} +import scala.collection.{Seq, mutable} import scala.collection.mutable.ArrayBuffer - import com.typesafe.scalalogging.StrictLogging - import filodb.core.TargetSchemaProvider -import filodb.core.query.{ColumnFilter, Filter, QueryContext, RangeParams} +import filodb.core.query.{ColumnFilter, QueryContext, QueryUtils, RangeParams} +import filodb.core.query.Filter.{Equals, EqualsRegex} import filodb.prometheus.ast.SubqueryUtils import filodb.prometheus.ast.Vectors.PromMetricLabel import filodb.prometheus.ast.WindowConstants @@ -406,17 +405,6 @@ object LogicalPlanUtils extends StrictLogging { columnMap.values.toSeq } - /** - * Resolves non-Equals shard-key filters into sets of Equals filters. - * The argument shardKeyMatcher function is called only if a non-Equals filter exists. - */ - def resolveShardKeyFilters(shardKeyFilters: Seq[ColumnFilter], - shardKeyMatcher: Seq[ColumnFilter] => Seq[Seq[ColumnFilter]]) - : Seq[Seq[ColumnFilter]] = { - val hasNonEqualShardKeyFilter = shardKeyFilters.exists(!_.filter.isInstanceOf[Filter.Equals]) - if (hasNonEqualShardKeyFilter) shardKeyMatcher(shardKeyFilters) else Seq(shardKeyFilters) - } - /** * Returns a set of target-schema columns iff all of: * - all plan RawSeries share the same target-schema columns. @@ -424,7 +412,6 @@ object LogicalPlanUtils extends StrictLogging { */ private def sameRawSeriesTargetSchemaColumns(plan: LogicalPlan, targetSchemaProvider: TargetSchemaProvider, - shardKeyMatcher: Seq[ColumnFilter] => Seq[Seq[ColumnFilter]], getShardKeyFilters: RawSeries => Seq[Seq[ColumnFilter]]) : Option[Seq[String]] = { // compose a stream of Options for each RawSeries-- @@ -433,15 +420,22 @@ object LogicalPlanUtils extends StrictLogging { .filter(_.isInstanceOf[RawSeries]) .map(_.asInstanceOf[RawSeries]).flatMap{ rs => val interval = LogicalPlanUtils.getSpanningIntervalSelector(rs) - val shardKeyFilters = getShardKeyFilters(rs) - .flatMap(LogicalPlanUtils.resolveShardKeyFilters(_, shardKeyMatcher)) - shardKeyFilters.map{ shardKey => - // replace any possibly-regex shard-key filters with equals-only filters, then - // use all filters to determine whether-or-not an unchanging target-schema is defined. - val filters = LogicalPlanUtils.upsertFilters(rs.filters, shardKey) - LogicalPlanUtils.getTargetSchemaIfUnchanging(targetSchemaProvider, filters, interval) - } + val rawShardKeyFilters = getShardKeyFilters(rs) + val shardKeyFilters = rawShardKeyFilters.flatMap{ filters => + val resolvedFilters: Seq[Seq[ColumnFilter]] = filters.map { filter => + filter.filter match { + // Take care of pipe-joined values here -- create one Equals filter per value. + case EqualsRegex(values: String) if QueryUtils.containsPipeOnlyRegex(values) => + QueryUtils.splitAtUnescapedPipes(values).map(value => ColumnFilter(filter.column, Equals(value))) + case _ => Seq(filter) + }} + QueryUtils.combinations(resolvedFilters) } + shardKeyFilters.map{ shardKey => + val filters = LogicalPlanUtils.upsertFilters(rs.filters, shardKey) + LogicalPlanUtils.getTargetSchemaIfUnchanging(targetSchemaProvider, filters, interval) + } + } if (rsTschemaOpts.isEmpty) { return None } @@ -461,7 +455,6 @@ object LogicalPlanUtils extends StrictLogging { */ private def canPushdown(plan: CandidatePushdownPlan, targetSchemaProvider: TargetSchemaProvider, - shardKeyMatcher: Seq[ColumnFilter] => Seq[Seq[ColumnFilter]], nonMetricShardKeyCols: Seq[String], getShardKeyFilters: RawSeries => Seq[Seq[ColumnFilter]]): Boolean = { val hasPushdownableClause = plan match { @@ -488,7 +481,7 @@ object LogicalPlanUtils extends StrictLogging { // return true // } - val tschema = sameRawSeriesTargetSchemaColumns(plan, targetSchemaProvider, shardKeyMatcher, getShardKeyFilters) + val tschema = sameRawSeriesTargetSchemaColumns(plan, targetSchemaProvider, getShardKeyFilters) if (tschema.isEmpty) { return false } @@ -510,7 +503,6 @@ object LogicalPlanUtils extends StrictLogging { */ def getPushdownKeys[T](lp: LogicalPlan, targetSchemaProvider: TargetSchemaProvider, - shardKeyMatcher: Seq[ColumnFilter] => Seq[Seq[ColumnFilter]], nonMetricShardKeyCols: Seq[String], getRawPushdownKeys: RawSeries => Set[T], getShardKeyFilters: RawSeries => Seq[Seq[ColumnFilter]]): Option[Set[T]] = { @@ -542,13 +534,12 @@ object LogicalPlanUtils extends StrictLogging { lhsKeys.isDefined && rhsKeys.isDefined && // either the lhs/rhs keys are equal, or at least one of lhs/rhs includes only scalars. (lhsKeys.get.isEmpty || rhsKeys.get.isEmpty || lhsKeys == rhsKeys) && - canPushdown(bj, targetSchemaProvider, shardKeyMatcher, nonMetricShardKeyCols, getShardKeyFilters) + canPushdown(bj, targetSchemaProvider, nonMetricShardKeyCols, getShardKeyFilters) // union lhs/rhs keys, since one might be empty (if it's a scalar) if (canPushdownBj) Some(lhsKeys.get.union(rhsKeys.get)) else None case agg: Aggregate => val keys = helper(agg.vectors) - val canPushdownAgg = - canPushdown(agg, targetSchemaProvider, shardKeyMatcher, nonMetricShardKeyCols, getShardKeyFilters) + val canPushdownAgg = canPushdown(agg, targetSchemaProvider, nonMetricShardKeyCols, getShardKeyFilters) if (keys.isDefined && canPushdownAgg) keys else None case nl: NonLeafLogicalPlan => // return the set of all child keys iff all child plans can be pushdown-optimized diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala index 56c305ad6b..d7ee9b1dab 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala @@ -1,18 +1,16 @@ package filodb.coordinator.queryplanner import java.util.concurrent.ConcurrentHashMap - import scala.collection.concurrent.{Map => ConcurrentMap} import scala.collection.mutable.ListBuffer import scala.jdk.CollectionConverters._ - import com.typesafe.scalalogging.StrictLogging import io.grpc.ManagedChannel - import filodb.coordinator.queryplanner.LogicalPlanUtils._ import filodb.coordinator.queryplanner.PlannerUtil.rewritePlanWithRemoteRawExport import filodb.core.metadata.{Dataset, DatasetOptions, Schemas} -import filodb.core.query.{ColumnFilter, PromQlQueryParams, QueryConfig, QueryContext, RangeParams, RvRange} +import filodb.core.query.Filter.{Equals, EqualsRegex} +import filodb.core.query.{ColumnFilter, PromQlQueryParams, QueryConfig, QueryContext, QueryUtils, RangeParams, RvRange} import filodb.grpc.GrpcCommonUtils import filodb.query._ import filodb.query.LogicalPlan._ @@ -45,17 +43,15 @@ trait PartitionLocationProvider { * @param remoteExecHttpClient If the partition is not local, a remote call is made to the correct partition to * query and retrieve the data. */ -class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider, +class MultiPartitionPlanner(val partitionLocationProvider: PartitionLocationProvider, localPartitionPlanner: QueryPlanner, localPartitionName: String, val dataset: Dataset, val queryConfig: QueryConfig, - shardKeyMatcher: Seq[ColumnFilter] => Seq[Seq[ColumnFilter]] = - PartitionLocationPlanner.equalsOnlyShardKeyMatcher, remoteExecHttpClient: RemoteExecHttpClient = RemoteHttpClient.defaultClient, channels: ConcurrentMap[String, ManagedChannel] = new ConcurrentHashMap[String, ManagedChannel]().asScala) - extends PartitionLocationPlanner(dataset, partitionLocationProvider, shardKeyMatcher) with StrictLogging { + extends PartitionLocationPlanner(dataset, partitionLocationProvider) with StrictLogging { override val schemas: Schemas = Schemas(dataset.schema) override val dsOptions: DatasetOptions = schemas.part.options @@ -101,8 +97,7 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider case _ => val result = walkLogicalPlanTree(logicalPlan, qContext) if (result.plans.size > 1) { - val dispatcher = PlannerUtil.pickDispatcher(result.plans) - MultiPartitionDistConcatExec(qContext, dispatcher, result.plans) + MultiPartitionDistConcatExec(qContext, inProcessPlanDispatcher, result.plans) } else result.plans.head } } @@ -173,19 +168,18 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider val execPlan = if (partitionName.equals(localPartitionName)) { localPartitionPlanner.materialize(logicalPlan, qContext) } else { - val promQl = LogicalPlanParser.convertToQuery(logicalPlan) val remoteContext = logicalPlan match { case tls: TopLevelSubquery => val instantTime = qContext.origQueryParams.asInstanceOf[PromQlQueryParams].startSecs val stepSecs = tls.stepMs / 1000 - generateRemoteExecParamsWithStep(qContext, promQl, instantTime, stepSecs, instantTime) + generateRemoteExecParamsWithStep(qContext, instantTime, stepSecs, instantTime) case psp: PeriodicSeriesPlan => val startSecs = psp.startMs / 1000 val stepSecs = psp.stepMs / 1000 val endSecs = psp.endMs / 1000 - generateRemoteExecParamsWithStep(qContext, promQl, startSecs, stepSecs, endSecs) + generateRemoteExecParamsWithStep(qContext, startSecs, stepSecs, endSecs) case _ => - generateRemoteExecParams(qContext, promQl, startMs, endMs) + generateRemoteExecParams(qContext, startMs, endMs) } // Single partition but remote, send the entire plan remotely if (grpcEndpoint.isDefined && !(queryConfig.grpcPartitionsDenyList.contains("*") || @@ -217,19 +211,26 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider */ private def walkMultiPartitionPlan(logicalPlan: LogicalPlan, qContext: QueryContext): PlanResult = { logicalPlan match { + case lp: BinaryJoin => materializePlanHandleSplitLeaf(lp, qContext) case _: MetadataQueryPlan => throw new IllegalArgumentException( "MetadataQueryPlan unexpected here") + case lp: ApplyInstantFunction => materializePlanHandleSplitLeaf(lp, qContext) case lp: ApplyInstantFunctionRaw => super.materializeApplyInstantFunctionRaw(qContext, lp) + case lp: Aggregate => materializePlanHandleSplitLeaf(lp, qContext) + case lp: ScalarVectorBinaryOperation => materializePlanHandleSplitLeaf(lp, qContext) case lp: ApplyMiscellaneousFunction => super.materializeApplyMiscellaneousFunction(qContext, lp) case lp: ApplySortFunction => super.materializeApplySortFunction(qContext, lp) + case lp: ScalarVaryingDoublePlan => materializePlanHandleSplitLeaf(lp, qContext) case _: ScalarTimeBasedPlan => throw new IllegalArgumentException( "ScalarTimeBasedPlan unexpected here") case lp: VectorPlan => super.materializeVectorPlan(qContext, lp) case _: ScalarFixedDoublePlan => throw new IllegalArgumentException( "ScalarFixedDoublePlan unexpected here") + case lp: ApplyAbsentFunction => materializePlanHandleSplitLeaf(lp, qContext) case lp: ScalarBinaryOperation => super.materializeScalarBinaryOperation(qContext, lp) case lp: ApplyLimitFunction => super.materializeLimitFunction(qContext, lp) case lp: TsCardinalities => materializeTsCardinalities(lp, qContext) + case lp: SubqueryWithWindowing => materializePlanHandleSplitLeaf(lp, qContext) case lp: TopLevelSubquery => super.materializeTopLevelSubquery(qContext, lp) case _: BinaryJoin | _: ApplyInstantFunction | @@ -255,25 +256,41 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider } // scalastyle:on cyclomatic.complexity - private def getRoutingKeys(logicalPlan: LogicalPlan) = { - LogicalPlan.getNonMetricShardKeyFilters(logicalPlan, dataset.options.nonMetricShardColumns) - .flatMap(LogicalPlanUtils.resolveShardKeyFilters(_, shardKeyMatcher)) + // TODO(a_theimer) + private def getRoutingKeys(logicalPlan: LogicalPlan): Set[Map[String, String]] = { + val rawSeriesPlans = LogicalPlan.findLeafLogicalPlans(logicalPlan) + .filter(_.isInstanceOf[RawSeries]) + rawSeriesPlans.flatMap { rs => + val filterGroup = LogicalPlan.getNonMetricShardKeyFilters(rs, dataset.options.nonMetricShardColumns).head + val mapThing = filterGroup.map { colFilter => + colFilter.filter match { + case eq: Equals => + (colFilter.column, Seq(eq.value.toString)) + case re: EqualsRegex if QueryUtils.containsPipeOnlyRegex(re.value.toString) => + val allValues = QueryUtils.splitAtUnescapedPipes(re.value.toString) + (colFilter.column, allValues) + } + } + val keys = mapThing.map(_._1) + val vals = mapThing.map(_._2) + val valCombos = QueryUtils.combinations(vals) + valCombos.map(keys.zip(_).toMap) + }.toSet } - private def generateRemoteExecParams(queryContext: QueryContext, promQl: String, startMs: Long, endMs: Long) = { + private def generateRemoteExecParams(queryContext: QueryContext, startMs: Long, endMs: Long) = { val queryParams = queryContext.origQueryParams.asInstanceOf[PromQlQueryParams] queryContext.copy( - origQueryParams = queryParams.copy(promQl = promQl, startSecs = startMs / 1000, endSecs = endMs / 1000), + origQueryParams = queryParams.copy(startSecs = startMs / 1000, endSecs = endMs / 1000), plannerParams = queryContext.plannerParams.copy(processMultiPartition = false)) } private def generateRemoteExecParamsWithStep( - queryContext: QueryContext, promQl: String, startSecs: Long, stepSecs: Long, endSecs: Long + queryContext: QueryContext, startSecs: Long, stepSecs: Long, endSecs: Long ) = { val queryParams = queryContext.origQueryParams.asInstanceOf[PromQlQueryParams] queryContext.copy( - origQueryParams = - queryParams.copy(promQl = promQl, startSecs = startSecs, stepSecs = stepSecs, endSecs = endSecs), + origQueryParams = queryParams.copy(startSecs = startSecs, stepSecs = stepSecs, endSecs = endSecs), plannerParams = queryContext.plannerParams.copy(processMultiPartition = false) ) } @@ -299,14 +316,13 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider val queryTimeRange = TimeRange(periodicSeriesTimeWithOffset.startMs - lookBackMs, periodicSeriesTimeWithOffset.endMs) - val partitions = routingKeys - .map{ key => - // Convert Seq[ColumnFilter] to Map[String, String] - key.map(filter => (filter.column, filter.filter.valuesStrings.head.toString)).toMap - } - .flatMap(shardKey => partitionLocationProvider.getPartitions(shardKey, queryTimeRange)) - .distinct - .sortBy(_.timeRange.startMs) + val partitions = if (routingKeys.isEmpty) List.empty + else { + routingKeys.flatMap{ keys => + partitionLocationProvider.getPartitions(keys, queryTimeRange). + sortBy(_.timeRange.startMs) + }.toList + } if (partitions.isEmpty && routingKeys.nonEmpty) logger.warn(s"No partitions found for routing keys: $routingKeys") @@ -322,7 +338,32 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider val execPlan = if (partitions.isEmpty || routingKeys.isEmpty) localPartitionPlanner.materialize(logicalPlan, qContext) else { - val execPlans = partitions.map(materializeForPartition(logicalPlan, _, qContext)) + val stepMs = queryParams.stepSecs * 1000 + val isInstantQuery: Boolean = if (queryParams.startSecs == queryParams.endSecs) true else false + var prevPartitionStart = queryParams.startSecs * 1000 + val execPlans = partitions.zipWithIndex.map { case (p, i) => + // First partition should start from query start time, no need to calculate time according to step for instant + // queries + val startMs = + if (i == 0 || isInstantQuery) { + queryParams.startSecs * 1000 + } else { + // The logic below does not work for partitions split across time as we encounter a hole + // in the produced result. The size of the hole is lookBackMs + stepMs + val numStepsInPrevPartition = (p.timeRange.startMs - prevPartitionStart + lookBackMs) / stepMs + val lastPartitionInstant = prevPartitionStart + numStepsInPrevPartition * stepMs + val start = lastPartitionInstant + stepMs + // If query duration is less than or equal to lookback start will be greater than query end time + if (start > (queryParams.endSecs * 1000)) queryParams.endSecs * 1000 else start + } + prevPartitionStart = startMs + // we assume endMs should be equal partition endMs but if the query's end is smaller than partition endMs, + // why would we want to stretch the query?? + val endMs = if (isInstantQuery) queryParams.endSecs * 1000 else p.timeRange.endMs + offsetMs.min + logger.debug(s"partitionInfo=$p; updated startMs=$startMs, endMs=$endMs") + // TODO: playing it safe for now with the TimeRange override; the parameter can eventually be removed. + materializeForPartition(logicalPlan, p, qContext, timeRangeOverride = Some(TimeRange(startMs, endMs))) + } if (execPlans.size == 1) execPlans.head else { // TODO: Do we pass in QueryContext in LogicalPlan's helper rvRangeForPlan? @@ -357,8 +398,7 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider } else logicalPlan localPartitionPlanner.materialize(lpWithUpdatedTime, queryContext) } else { - val promQL = LogicalPlanParser.convertToQuery(logicalPlan) - val ctx = generateRemoteExecParams(queryContext, promQL, timeRange.startMs, timeRange.endMs) + val ctx = generateRemoteExecParams(queryContext, timeRange.startMs, timeRange.endMs) if (grpcEndpoint.isDefined && !(queryConfig.grpcPartitionsDenyList.contains("*") || queryConfig.grpcPartitionsDenyList.contains(partitionName.toLowerCase))) { @@ -380,7 +420,7 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider * (a) the returned ranges lie within the argument time-range, and * (b) lookbacks do not cross partition splits (where the "lookback" is defined only by the argument) * @param assignments must be sorted and time-disjoint - * @param queryRange the complete time-range. Does not include the offset. + * @param range the complete time-range. Does not include the offset. * @param lookbackMs the time to skip immediately after a partition split. * @param stepMsOpt occupied iff the returned ranges should describe periodic steps * (i.e. all range start times (except the first) should be snapped to a step) @@ -469,19 +509,10 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider private def materializePlanHandleSplitLeaf(logicalPlan: LogicalPlan, qContext: QueryContext): PlanResult = { val qParams = qContext.origQueryParams.asInstanceOf[PromQlQueryParams] - // (1) Find the set of shard-key filter groups for each plan. - // (2) Resolve each group into a set of groups (e.g. if any regex filters exist). - // (3) For each filter group, replace the filters of the source RawSeries plan. - // (4) Identify whether-or-not any of the RawSeries plans would source data from multiple partitions. - val hasMultiPartitionLeaves = - LogicalPlan.findLeafLogicalPlans(logicalPlan) - .filter(_.isInstanceOf[RawSeries]) - .flatMap { rs => - val rawFilters = LogicalPlan.getNonMetricShardKeyFilters(rs, dataset.options.nonMetricShardColumns) - val filters = rawFilters.flatMap(shardKeyMatcher(_)) - filters.map(rs.replaceFilters) - } - .exists(getPartitions(_, qParams).size > 1) + val hasMultiPartitionLeaves = LogicalPlan.findLeafLogicalPlans(logicalPlan) + .filter(_.isInstanceOf[RawSeries]) + .map(getPartitions(_, qParams)) + .exists(_.size > 1) if (hasMultiPartitionLeaves) { materializeSplitLeafPlan(logicalPlan, qContext) } else { logicalPlan match { @@ -683,7 +714,6 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider } else { execPlans } - } // stitch if necessary @@ -822,7 +852,7 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider private def createMetadataRemoteExec(qContext: QueryContext, partitionAssignment: PartitionAssignment, urlParams: Map[String, String]) = { val finalQueryContext = generateRemoteExecParams( - qContext, "", partitionAssignment.timeRange.startMs, partitionAssignment.timeRange.endMs) + qContext, partitionAssignment.timeRange.startMs, partitionAssignment.timeRange.endMs) val httpEndpoint = partitionAssignment.httpEndPoint + finalQueryContext.origQueryParams.asInstanceOf[PromQlQueryParams].remoteQueryPath.getOrElse("") MetadataRemoteExec(httpEndpoint, remoteHttpTimeoutMs, diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/PartitionLocationPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/PartitionLocationPlanner.scala index 01b84107db..a1ce935638 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/PartitionLocationPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/PartitionLocationPlanner.scala @@ -1,27 +1,12 @@ package filodb.coordinator.queryplanner import filodb.coordinator.queryplanner.LogicalPlanUtils.getLookBackMillis -import filodb.coordinator.queryplanner.PartitionLocationPlanner.equalsOnlyShardKeyMatcher import filodb.core.metadata.Dataset -import filodb.core.query.{ColumnFilter, PromQlQueryParams} -import filodb.core.query.Filter.Equals +import filodb.core.query.{PromQlQueryParams, QueryUtils} +import filodb.core.query.Filter.{Equals, EqualsRegex} import filodb.query.LogicalPlan import filodb.query.exec.{ExecPlan, PromQlRemoteExec} -object PartitionLocationPlanner { - /** - * A default matcher function. - * @throws IllegalArgumentException if any filter isn't an Equals. - */ - def equalsOnlyShardKeyMatcher(filters: Seq[ColumnFilter]): Seq[Seq[ColumnFilter]] = { - filters.foreach{ - case ColumnFilter(_, Equals(_)) => { /* do nothing */ } - case filter => throw new IllegalArgumentException("cannot match regex filters; filter: " + filter) - } - Seq(filters) - } -} - /** * Abstract class for planners that need getPartitions functionality. * @@ -32,14 +17,10 @@ object PartitionLocationPlanner { * and materialize accordingly, then the SKRP will handle any higher-level concatenation/aggregation/joins * for each of these groups. */ -abstract class PartitionLocationPlanner( - dataset: Dataset, - partitionLocationProvider: PartitionLocationProvider, - shardKeyMatcher: Seq[ColumnFilter] => Seq[Seq[ColumnFilter]] = equalsOnlyShardKeyMatcher) +abstract class PartitionLocationPlanner(dataset: Dataset, + partitionLocationProvider: PartitionLocationProvider) extends QueryPlanner with DefaultPlanner { - private val nonMetricColumnSet = dataset.options.nonMetricShardColumns.toSet - // scalastyle:off method.length /** * Gets the partition Assignment for the given plan @@ -50,23 +31,31 @@ abstract class PartitionLocationPlanner( //1. Get a Seq of all Leaf node filters val leafFilters = LogicalPlan.getColumnFilterGroup(logicalPlan) + val nonMetricColumnSet = dataset.options.nonMetricShardColumns.toSet //2. Filter from each leaf node filters to keep only nonShardKeyColumns and convert them to key value map val routingKeyMap: Seq[Map[String, String]] = leafFilters .filter(_.nonEmpty) .map(_.filter(col => nonMetricColumnSet.contains(col.column))) - .flatMap{ filters => - val hasNonEqualShardKeyFilter = filters.exists(!_.filter.isInstanceOf[Equals]) - if (hasNonEqualShardKeyFilter) shardKeyMatcher(filters.toSeq) else Seq(filters.toSeq) - } .map{ filters => filters.map { filter => - val value = filter.filter match { - case Equals(value) => value.toString + val values = filter.filter match { + case Equals(value) => Seq(value.toString) + // Split '|'-joined values if pipes are the only regex chars used. + case EqualsRegex(value: String) if QueryUtils.containsPipeOnlyRegex(value) => + QueryUtils.splitAtUnescapedPipes(value) case _ => throw new IllegalArgumentException( - s"""shard keys must be filtered by equality. filter=${filter}""") + s"""shard keys must be filtered by equality or "|"-only regex. filter=${filter}""") } - (filter.column, value) - }.toMap + (filter.column, values) + } + } + .flatMap{ keyValuesPairs => + // Get all possible value combos, then create a key->value map for each combo. + // Ordering the pairs first since the output of combinations() is also ordered. + val orderedPairs = keyValuesPairs.toSeq + val keys: Seq[String] = orderedPairs.map(_._1) + val values: Seq[Seq[String]] = orderedPairs.map(_._2) + QueryUtils.combinations(values).map(keys.zip(_).toMap) } // 3. Determine the query time range diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/QueryUtils.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/QueryUtils.scala deleted file mode 100644 index 4d986ca6ba..0000000000 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/QueryUtils.scala +++ /dev/null @@ -1,36 +0,0 @@ -package filodb.coordinator.queryplanner - -import scala.collection.mutable - -/** - * Storage for miscellaneous utility functions. - */ -object QueryUtils { - // TODO(amt): remove? - /** - * Returns all possible sets of elements where exactly one element is - * chosen from each of the argument sequences. - * - * @param choices : all sequences should have at least one element. - * @return ordered sequences; each sequence is ordered such that the element - * at index i is chosen from the ith argument sequence. - */ - def combinations[T](choices: Seq[Seq[T]]): Seq[Seq[T]] = { - val running = new mutable.ArraySeq[T](choices.size) - val result = new mutable.ArrayBuffer[Seq[T]] - - def helper(iChoice: Int): Unit = { - if (iChoice == choices.size) { - result.append(Nil ++ running) - return - } - for (choice <- choices(iChoice)) { - running(iChoice) = choice - helper(iChoice + 1) - } - } - - helper(0) - result - } -} diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlanner.scala index 8c118c3610..14040f97fa 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlanner.scala @@ -1,8 +1,10 @@ package filodb.coordinator.queryplanner +import scala.collection.{mutable, Seq} + import filodb.core.{StaticTargetSchemaProvider, TargetSchemaProvider} import filodb.core.metadata.{Dataset, DatasetOptions, Schemas} -import filodb.core.query.{ColumnFilter, PromQlQueryParams, QueryConfig, QueryContext, RangeParams} +import filodb.core.query.{ColumnFilter, Filter, PromQlQueryParams, QueryConfig, QueryContext, RangeParams} import filodb.query._ import filodb.query.LogicalPlan._ import filodb.query.exec._ @@ -32,13 +34,19 @@ class ShardKeyRegexPlanner(val dataset: Dataset, partitionLocationProvider: PartitionLocationProvider, config: QueryConfig, _targetSchemaProvider: TargetSchemaProvider = StaticTargetSchemaProvider()) - extends PartitionLocationPlanner(dataset, partitionLocationProvider, shardKeyMatcher) { + extends PartitionLocationPlanner(dataset, partitionLocationProvider) { override def queryConfig: QueryConfig = config override val schemas: Schemas = Schemas(dataset.schema) override val dsOptions: DatasetOptions = schemas.part.options private val datasetMetricColumn = dataset.options.metricColumn + // TODO(a_theimer) + private val nonMetricShardKeyColToIndex = dataset.options.shardKeyColumns + .filterNot(_ == dataset.options.metricColumn) + .zipWithIndex + .toMap + private def targetSchemaProvider(qContext: QueryContext): TargetSchemaProvider = { qContext.plannerParams.targetSchemaProviderOverride.getOrElse(_targetSchemaProvider) } @@ -58,7 +66,6 @@ class ShardKeyRegexPlanner(val dataset: Dataset, LogicalPlanUtils.getPushdownKeys( plan, targetSchemaProvider(qContext), - shardKeyMatcher, dataset.options.nonMetricShardColumns, getRawShardKeys, rs => getShardKeys(rs)) @@ -179,7 +186,120 @@ class ShardKeyRegexPlanner(val dataset: Dataset, generateExec(logicalPlan, shardKeyMatches, qContext) } - // This will be deprecated to reduce query fanout. + /** + * Updates the time params and query of the the argument PromQlQueryParams according to the argument LogicalPlan. + */ + private def updateQueryParams(logicalPlan: LogicalPlan, + queryParams: PromQlQueryParams): PromQlQueryParams = { + logicalPlan match { + case tls: TopLevelSubquery => { + val instantTime = queryParams.startSecs + queryParams.copy( + promQl = LogicalPlanParser.convertToQuery(logicalPlan), + startSecs = instantTime, + endSecs = instantTime + ) + } + case psp: PeriodicSeriesPlan => { + queryParams.copy( + promQl = LogicalPlanParser.convertToQuery(logicalPlan), + startSecs = psp.startMs / 1000, + endSecs = psp.endMs / 1000, + stepSecs = psp.stepMs / 1000 + ) + } + case _ => queryParams.copy(promQl = LogicalPlanParser.convertToQuery(logicalPlan)) + } + } + + // scalastyle:off method.length + /** + * Group shard keys by partition and generate an ExecPlan for each. + * Plans will match shard-keys by pipe-separated regex filters. For example, Suppose the following keys are provided: + * keys = {{a=1, b=1}, {a=1, b=2}, {a=2, b=3}} + * These will be grouped according to the partitions they occupy: + * part1 -> {{a=1, b=1}, {a=1, b=2}} + * part2 -> {{a=2, b=3}} + * Then a plan will be generated for each partition: + * plan1 -> {a=1, b=~"1|2"} + * plan2 -> {a=2, b=3} + * Additional plans are materialized per partition when multiple regex filters would otherwise be required. + * This prevents scenarios such as: + * keys = {{a=1, b=2}, {a=3, b=4}, {a=5, b=6}} + * These will be grouped according to the partitions they occupy: + * part1 -> {{a=1, b=2}, {a=3, b=4}} + * part2 -> {{a=5, b=6}} + * Then a plan will be generated for each partition: + * plan1 -> {a=~"1|3", b=~"2|4"} + * plan2 -> {a=5, b=6} + * This might erroneously read {a=1, b=4}, which was not included in the original key set. + */ + private def generateExecForEachPartition(logicalPlan: LogicalPlan, + keys: Seq[Seq[ColumnFilter]], + qContext: QueryContext): Seq[ExecPlan] = { + val queryParams = qContext.origQueryParams.asInstanceOf[PromQlQueryParams] + + // maps individual partitions to the set of shard-keys they contain. + val partitionsToKeys = new mutable.HashMap[String, mutable.Buffer[Seq[ColumnFilter]]]() + keys.foreach { key => + val newLogicalPlan = logicalPlan.replaceFilters(key) + // Querycontext should just have the part of query which has regex + // For example for exp(sum(test{_ws_ = "demo", _ns_ =~ "App.*"})), sub queries should be + // sum(test{_ws_ = "demo", _ns_ = "App-1"}), sum(test{_ws_ = "demo", _ns_ = "App-2"}) etc + val newQueryParams = updateQueryParams(newLogicalPlan, queryParams) + getPartitions(newLogicalPlan, newQueryParams) + .map(_.partitionName) + .distinct + .foreach(part => partitionsToKeys.getOrElseUpdate(part, new mutable.ArrayBuffer).append(key)) + } + + // Sort each key into the same order as nonMetricShardKeys, then group keys with the same prefix. + // A plan will be created for each group; this prevents the scenario mentioned in the javadoc. + // NOTE: this solution is not optimal in all cases, but it does guarantee groupings are valid. + val partitionToKeyGroups = partitionsToKeys.map{ case (partition, keys) => + val prefixGroups = keys + .map(key => key.sortBy(filter => nonMetricShardKeyColToIndex.getOrElse(filter.column, 0))) + .groupBy(_.dropRight(1)) + .values + (partition, prefixGroups) + } + + // Skip the aggregate presentation if there are more than one plans to materialize. + val skipAggregatePresentValue = partitionToKeyGroups.size > 1 || + partitionToKeyGroups.values.headOption.map(_.size).getOrElse(0) > 1 + + // Create one plan per key group. + partitionToKeyGroups.flatMap{ case (partition, keyGroups) => + // NOTE: partition is intentionally unused; the inner planner will again determine which partitions own the data. + keyGroups.map{ keys => + // Create a map of key->values, then create a ColumnFilter for each key. + val keyToValues = new mutable.HashMap[String, mutable.Set[String]]() + keys.flatten.foreach { filter => + // Find the key's list of values in the map (or create it), then add the filter's values. + val values = keyToValues.getOrElseUpdate(filter.column, new mutable.HashSet[String]()) + filter.filter.valuesStrings.map(_.toString).foreach(values.add) + } + val newFilters = keyToValues.map { case (key, values) => + val filter = if (values.size == 1) { + Filter.Equals(values.head) + } else { + // Concatenate values with "|" for multi-valued keys. + Filter.EqualsRegex(values.toSeq.sorted.mkString("|")) + } + ColumnFilter(key, filter) + }.toSeq + // Update the LogicalPlan with the new partition-specific filters, then materialize. + val newLogicalPlan = logicalPlan.replaceFilters(newFilters) + val newQueryParams = updateQueryParams(newLogicalPlan, queryParams) + val newQueryContext = qContext.copy(origQueryParams = newQueryParams, plannerParams = qContext.plannerParams. + copy(skipAggregatePresent = skipAggregatePresentValue)) + queryPlanner.materialize(newLogicalPlan, newQueryContext) + } + }.toSeq + } + // scalastyle:on method.length + + // FIXME: This will eventually be replaced with generateExecForEachPartition. @Deprecated private def generateExecForEachKey(logicalPlan: LogicalPlan, keys: Seq[Seq[ColumnFilter]], @@ -220,7 +340,7 @@ class ShardKeyRegexPlanner(val dataset: Dataset, keys: Seq[Seq[ColumnFilter]], qContext: QueryContext): Seq[ExecPlan] = if (qContext.plannerParams.reduceShardKeyRegexFanout) { - Seq(queryPlanner.materialize(logicalPlan, qContext)) + generateExecForEachPartition(logicalPlan, keys, qContext) } else { generateExecForEachKey(logicalPlan, keys, qContext) } diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala index 33110019d7..017e7d60f2 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala @@ -22,7 +22,6 @@ import filodb.query.LogicalPlan._ import filodb.query.exec.{LocalPartitionDistConcatExec, _} import filodb.query.exec.InternalRangeFunction.Last - // scalastyle:off file.size.limit object SingleClusterPlanner { @@ -62,8 +61,6 @@ class SingleClusterPlanner(val dataset: Dataset, clusterName: String, spreadProvider: SpreadProvider = StaticSpreadProvider(), _targetSchemaProvider: TargetSchemaProvider = StaticTargetSchemaProvider(), - shardKeyMatcher: Seq[ColumnFilter] => Seq[Seq[ColumnFilter]] = - PartitionLocationPlanner.equalsOnlyShardKeyMatcher, timeSplitEnabled: Boolean = false, minTimeRangeForSplitMs: => Long = 1.day.toMillis, splitSizeMs: => Long = 1.day.toMillis) @@ -99,7 +96,8 @@ class SingleClusterPlanner(val dataset: Dataset, } /** - * If TargetSchema exists and all of the target-schema label filters are provided in the query, then return true. + * If TargetSchema exists and all of the target-schema label filters (equals or pipe-only EqualsRegex) are + * provided in the query, then return true. * * @param filters Query Column Filters * @param targetSchema TargetSchema @@ -110,12 +108,10 @@ class SingleClusterPlanner(val dataset: Dataset, if (targetSchema.isEmpty || targetSchema.get.schema.isEmpty) { return false } - // Make sure each target-schema column has a filter. + // Make sure each target-schema column is filtered by equality. targetSchema.get.schema - .forall(tschemaCol => filters.exists(cf => { - cf.column == tschemaCol && - (shardColumns.contains(tschemaCol) || cf.filter.isInstanceOf[Equals]) - })) + .forall( tschemaCol => filters.exists( cf => + cf.column == tschemaCol && cf.filter.isInstanceOf[Equals])) } /** @@ -257,40 +253,52 @@ class SingleClusterPlanner(val dataset: Dataset, } // scalastyle:off method.length - def shardsFromFilters(rawFilters: Seq[ColumnFilter], + def shardsFromFilters(filters: Seq[ColumnFilter], qContext: QueryContext, startMs: Long, endMs: Long, useTargetSchemaForShards: Seq[ColumnFilter] => Boolean = _ => false): Seq[Int] = { + require(shardColumns.nonEmpty || qContext.plannerParams.shardOverrides.nonEmpty, s"Dataset $dsRef does not have shard columns defined, and shard overrides were not mentioned") qContext.plannerParams.shardOverrides.getOrElse { - val hasNonEqualsShardKeyFilter = rawFilters.exists { filter => - shardColumns.contains(filter.column) && !filter.filter.isInstanceOf[Equals] + val shardColToValues: Seq[(String, Seq[String])] = shardColumns.map { shardCol => + // To compute the shard hash, filters must match all shard columns either by equality or EqualsRegex, + // where any match by EqualsRegex can use at most the '|' regex character. + val values = filters.find(f => f.column == shardCol) match { + case Some(ColumnFilter(_, Filter.Equals(filtVal: String))) => + Seq(filtVal) + case Some(ColumnFilter(_, Filter.EqualsRegex(filtVal: String))) + if QueryUtils.containsPipeOnlyRegex(filtVal) => QueryUtils.splitAtUnescapedPipes(filtVal) + case Some(ColumnFilter(_, filter)) => + throw new BadQueryException(s"Found filter for shard column $shardCol but " + + s"$filter cannot be used for shard key routing") + case _ => + throw new BadQueryException(s"Could not find filter for shard key column " + + s"$shardCol, shard key hashing disabled") + } + val trimmedValues = values.map(value => RecordBuilder.trimShardColumn(dsOptions, shardCol, value)) + (shardCol, trimmedValues) } - val filterGroups = if (hasNonEqualsShardKeyFilter) { - shardKeyMatcher(rawFilters).map(LogicalPlanUtils.upsertFilters(rawFilters, _)) - } else { - Seq(rawFilters) + + // Get all (ordered) combinations of values, then create (key,value) pairs for each. + val shardKeyValuePairs: Seq[Seq[(String, String)]] = { + val keys = shardColToValues.map(_._1) + val valueGroups = shardColToValues.map(_._2) + QueryUtils.combinations(valueGroups).map(keys.zip(_)) } - filterGroups.flatMap{ filters => - val shardValues = shardColumns.map { shardCol => - // To compute the shard hash, filters must match all shard columns by equality. - val value = filters.find(f => f.column == shardCol) match { - case Some(ColumnFilter(_, Filter.Equals(filtVal: String))) => filtVal - case Some(ColumnFilter(_, filter)) => - throw new BadQueryException(s"Found filter for shard column $shardCol but " + - s"$filter cannot be used for shard key routing") - case _ => - throw new BadQueryException(s"Could not find filter for shard key column " + - s"$shardCol, shard key hashing disabled") + + // For each set of pairs, create a set of Equals filters and compute the shards for each. + shardKeyValuePairs.flatMap{ kvPairs => + val kvMap = kvPairs.toMap + val updFilters = filters.map{ filt => + kvMap.get(filt.column) + .map(value => ColumnFilter(filt.column, Filter.Equals(value))) + .getOrElse(filt) } - RecordBuilder.trimShardColumn(dsOptions, shardCol, value) - } - val shardPairs = shardColumns.zip(shardValues) - shardsFromValues(shardPairs, filters, qContext, startMs, endMs, useTargetSchemaForShards) + shardsFromValues(kvPairs, updFilters, qContext, startMs, endMs, useTargetSchemaForShards) }.distinct } } @@ -419,10 +427,9 @@ class SingleClusterPlanner(val dataset: Dataset, LogicalPlanUtils.getPushdownKeys( plan, targetSchemaProvider(qContext), - shardKeyMatcher, dataset.options.nonMetricShardColumns, getRawPushdownShards, - rs => LogicalPlan.getRawSeriesFilters(rs).flatMap(shardKeyMatcher(_))) + rs => LogicalPlan.getRawSeriesFilters(rs)) } /** @@ -790,6 +797,8 @@ class SingleClusterPlanner(val dataset: Dataset, // If a target-schema is defined and is not changing during the query window and all the target-schema labels are // provided in query filters (Equals), then find the target shard to route to using ingestionShard helper method. val useTschema = !tsChangeExists && allTSLabelsPresent + // NOTE: this should probably be "tsChangesExists && useTschema", + // but some tests conditionally check for stitching. if (tsChangeExists) { // Record that some tschema changes exist. This will later invoke result stitching. tschemaChangesOverTime(0) = true @@ -837,14 +846,23 @@ class SingleClusterPlanner(val dataset: Dataset, PlanResult(metaExec) } - // allow metadataQueries to get list of shards from shardKeyFilters only if - // filters are given for all shard-key columns + // allow metadataQueries to get list of shards from shardKeyFilters only if all shardCols have Equals filter + // or EqualsRegex filter with only the pipe special character. private def canGetShardsFromFilters(renamedFilters: Seq[ColumnFilter], qContext: QueryContext): Boolean = { - qContext.plannerParams.shardOverrides.isEmpty && - shardColumns.nonEmpty && + if (qContext.plannerParams.shardOverrides.isEmpty && shardColumns.nonEmpty) { shardColumns.toSet.subsetOf(renamedFilters.map(_.column).toSet) && - shardColumns.forall { shardCol => renamedFilters.exists(f => f.column == shardCol) } + shardColumns.forall { shardCol => + // So to compute the shard hash we need shardCol == value filter (exact equals) for each shardColumn + renamedFilters.find(f => f.column == shardCol) match { + case Some(ColumnFilter(_, Filter.Equals(_: String))) => true + case Some(ColumnFilter(_, Filter.EqualsRegex(value: String))) => + // Make sure no regex chars except the pipe, which can be used to concatenate values. + QueryUtils.containsPipeOnlyRegex(value) + case _ => false + } + } + } else false } private def materializeLabelNames(qContext: QueryContext, diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala index 1a15715c87..bebb55ebfc 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala @@ -7,8 +7,8 @@ import com.typesafe.config.ConfigFactory import org.scalatest.funspec.AnyFunSpec import org.scalatest.matchers.should.Matchers import filodb.coordinator.ShardMapper -import filodb.coordinator.client.QueryCommands.{FunctionalTargetSchemaProvider, StaticSpreadProvider} -import filodb.core.{MetricsTestData, SpreadChange, StaticTargetSchemaProvider, TargetSchemaChange} +import filodb.coordinator.client.QueryCommands.{FunctionalTargetSchemaProvider} +import filodb.core.{MetricsTestData, StaticTargetSchemaProvider, TargetSchemaChange} import filodb.core.metadata.{Dataset, Schemas} import filodb.core.query._ import filodb.core.query.Filter.{Equals, EqualsRegex} @@ -18,10 +18,6 @@ import filodb.prometheus.parse.Parser.Antlr import filodb.query.{BadQueryException, IntervalSelector, LabelCardinality, PlanValidationSpec, RawSeries} import filodb.query.exec._ - -import scala.collection.mutable -import scala.collection.mutable.ArrayBuffer - // scalastyle:off line.size.limit // scalastyle:off number.of.methods class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationSpec { @@ -53,50 +49,28 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS case class Planners(spp: SinglePartitionPlanner, lt: LongTimeRangePlanner, raw: SingleClusterPlanner, ds: SingleClusterPlanner, rr: SingleClusterPlanner) - def getPlanners(nShards: Int, dataset: Dataset, shardKeyMatcher: Seq[ColumnFilter] => Seq[Seq[ColumnFilter]]): Planners = { + def getPlanners(nShards: Int, dataset: Dataset): Planners = { val mapper = new ShardMapper(nShards) for {i <- 0 until nShards} mapper.registerNode(Seq(i), node) def mapperRef = mapper val rawPlanner = new SingleClusterPlanner(dataset, schemas, mapperRef, - earliestRetainedTimestampFn = now - rawRetention, queryConfig, "raw", shardKeyMatcher = shardKeyMatcher) + earliestRetainedTimestampFn = now - rawRetention, queryConfig, "raw") val downsamplePlanner = new SingleClusterPlanner(dataset, schemas, mapperRef, - earliestRetainedTimestampFn = now - downsampleRetention, queryConfig, "downsample", shardKeyMatcher = shardKeyMatcher) + earliestRetainedTimestampFn = now - downsampleRetention, queryConfig, "downsample") val longTermPlanner = new LongTimeRangePlanner(rawPlanner, downsamplePlanner, earliestRawTimestampFn = now - rawRetention, latestDownsampleTimestampFn = now - timeToDownsample, inProcessDispatcher, queryConfig, dataset) val recordingRulesPlanner = new SingleClusterPlanner(dataset, schemas, mapperRef, earliestRetainedTimestampFn = now - rrRetention, - queryConfig, "recordingRules", shardKeyMatcher = shardKeyMatcher) + queryConfig, "recordingRules") val planners = Map("longTerm" -> longTermPlanner, "recordingRules" -> recordingRulesPlanner) val singlePartitionPlanner = new SinglePartitionPlanner(planners, plannerSelector, dataset, queryConfig) Planners(singlePartitionPlanner, longTermPlanner, rawPlanner, downsamplePlanner, recordingRulesPlanner) } - private val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { - // we may have mixed of a regex filter and a non-regex filter. - if (shardColumnFilters.exists(f => f.column == "_ns_" && f.filter.isInstanceOf[EqualsRegex])) { - // to ensure that tests dont call something else that is not configured - require(shardColumnFilters.exists(f => f.column == "_ns_" && f.filter.isInstanceOf[EqualsRegex] - && ( - f.filter.asInstanceOf[EqualsRegex].pattern.toString == ".*Ns" - || f.filter.asInstanceOf[EqualsRegex].pattern.toString == "localNs.*"))) - val nsCol = shardColumnFilters.find(_.column == "_ns_").get - if (nsCol.filter.asInstanceOf[EqualsRegex].pattern.toString == "localNs.*") { - Seq( - Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("localNs"))), - Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("localNs1"))) - ) - } else { - Seq( - Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("localNs"))), - Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("remoteNs"))) - ) - } - } else if (shardColumnFilters.exists(f => f.column == "_ns_" && f.filter.isInstanceOf[Equals])) { - Seq(shardColumnFilters) - } else { - Nil - } // i.e. filters for a scalar + val (singlePartitionPlanner, longTermPlanner, rawPlanner) = { + val planners = getPlanners(nShards = 2, dataset) + (planners.spp, planners.lt, planners.raw) } private val oneRemotePartitionLocationProvider = new PartitionLocationProvider { @@ -129,13 +103,45 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS case "remoteNs1" => List(PartitionAssignment("remotePartition1", "remote1Partition-url1", TimeRange(timeRange.startMs, timeRange.endMs))) - case key => throw new IllegalArgumentException("unexpected routing key: " + key) + case _ => throw new IllegalArgumentException("nope") } } override def getMetadataPartitions(nonMetricShardKeyFilters: Seq[ColumnFilter], timeRange: TimeRange): List[PartitionAssignment] = ??? } + val oneRemoteMultiPartitionPlanner = new MultiPartitionPlanner(oneRemotePartitionLocationProvider, singlePartitionPlanner, + "localPartition", dataset, queryConfig) + val twoRemoteMultiPartitionPlanner = new MultiPartitionPlanner(twoRemotePartitionLocationProvider, singlePartitionPlanner, + "localPartition", dataset, queryConfig) + + private val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { + // we may have mixed of a regex filter and a non-regex filter. + if (shardColumnFilters.exists(f => f.column == "_ns_" && f.filter.isInstanceOf[EqualsRegex])) { + // to ensure that tests dont call something else that is not configured + require(shardColumnFilters.exists(f => f.column == "_ns_" && f.filter.isInstanceOf[EqualsRegex] + && ( + f.filter.asInstanceOf[EqualsRegex].pattern.toString == ".*Ns" + || f.filter.asInstanceOf[EqualsRegex].pattern.toString == "localNs.*"))) + val nsCol = shardColumnFilters.find(_.column == "_ns_").get + if (nsCol.filter.asInstanceOf[EqualsRegex].pattern.toString == "localNs.*") { + Seq( + Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("localNs"))), + Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("localNs1"))) + ) + } else { + Seq( + Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("localNs"))), + Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("remoteNs"))) + ) + } + } else if (shardColumnFilters.exists(f => f.column == "_ns_" && f.filter.isInstanceOf[Equals])) { + Seq(shardColumnFilters) + } else { + Nil + } // i.e. filters for a scalar + } + private val oneRemoteShardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { if (shardColumnFilters.exists(f => f.column == "_ns_" && f.filter.isInstanceOf[EqualsRegex])) { // to ensure that tests dont call something else that is not configured @@ -167,24 +173,9 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS } // i.e. filters for a scalar } - - val (singlePartitionPlanner, longTermPlanner, rawPlanner) = { - val planners = getPlanners(nShards = 2, dataset, shardKeyMatcherFn) - (planners.spp, planners.lt, planners.raw) - } - val oneRemotePlanners = getPlanners(nShards = 2, dataset, oneRemoteShardKeyMatcherFn) - val twoRemotePlanners = getPlanners(nShards = 2, dataset, twoRemoteShardKeyMatcherFn) - - val defaultMultiPartitionPlanner = new MultiPartitionPlanner(oneRemotePartitionLocationProvider, singlePartitionPlanner, - "localPartition", dataset, queryConfig, shardKeyMatcher = shardKeyMatcherFn) - val oneRemoteMultiPartitionPlanner = new MultiPartitionPlanner(oneRemotePartitionLocationProvider, oneRemotePlanners.spp, - "localPartition", dataset, queryConfig, shardKeyMatcher = oneRemoteShardKeyMatcherFn) - val twoRemoteMultiPartitionPlanner = new MultiPartitionPlanner(twoRemotePartitionLocationProvider, twoRemotePlanners.spp, - "localPartition", dataset, queryConfig, shardKeyMatcher = twoRemoteShardKeyMatcherFn) - private val targetSchemaProvider = StaticTargetSchemaProvider() - val rootPlanner = new ShardKeyRegexPlanner(dataset, defaultMultiPartitionPlanner, shardKeyMatcherFn, oneRemotePartitionLocationProvider, queryConfig) + val rootPlanner = new ShardKeyRegexPlanner(dataset, oneRemoteMultiPartitionPlanner, shardKeyMatcherFn, oneRemotePartitionLocationProvider, queryConfig) val oneRemoteRootPlanner = new ShardKeyRegexPlanner(dataset, oneRemoteMultiPartitionPlanner, oneRemoteShardKeyMatcherFn, oneRemotePartitionLocationProvider, queryConfig) val twoRemoteRootPlanner = new ShardKeyRegexPlanner(dataset, twoRemoteMultiPartitionPlanner, twoRemoteShardKeyMatcherFn, twoRemotePartitionLocationProvider, queryConfig) @@ -421,31 +412,43 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS val execPlan = rootPlanner.materialize(lp, QueryContext(origQueryParams = queryParams)) val expected = """T~AggregatePresenter(aggrOp=Count, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634777330)) - |-E~LocalPartitionReduceAggregateExec(aggrOp=Count, aggrParams=List()) on InProcessPlanDispatcher + |-E~LocalPartitionReduceAggregateExec(aggrOp=Count, aggrParams=List()) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) |--T~AggregateMapReduce(aggrOp=Count, aggrParams=List(), without=List(), by=List()) |---E~SetOperatorExec(binaryOp=LUnless, on=Some(List(instance)), ignoring=List()) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) |----E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192))) |-----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],raw) |------T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],raw) |------T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) + |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],raw) + |-----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],downsample) |------T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) + |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],downsample) |------T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |----E~StitchRvsExec() on InProcessPlanDispatcher - |-----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |------T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(localNs.*)), ColumnFilter(instance,EqualsRegex(.*)), ColumnFilter(_metric_,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |------T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(localNs.*)), ColumnFilter(instance,EqualsRegex(.*)), ColumnFilter(_metric_,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |------T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(localNs.*)), ColumnFilter(instance,EqualsRegex(.*)), ColumnFilter(_metric_,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |------T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(localNs.*)), ColumnFilter(instance,EqualsRegex(.*)), ColumnFilter(_metric_,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample)""".stripMargin + |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],downsample) + |----E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) + |-----E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192))) + |------E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],raw) + |-------T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) + |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(instance,EqualsRegex(.*)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs1)), ColumnFilter(_metric_,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],raw) + |-------T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) + |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(instance,EqualsRegex(.*)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs1)), ColumnFilter(_metric_,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],raw) + |------E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],downsample) + |-------T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) + |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(instance,EqualsRegex(.*)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs1)), ColumnFilter(_metric_,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],downsample) + |-------T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) + |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(instance,EqualsRegex(.*)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs1)), ColumnFilter(_metric_,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],downsample) + |-----E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192))) + |------E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],raw) + |-------T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) + |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(instance,EqualsRegex(.*)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],raw) + |-------T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) + |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(instance,EqualsRegex(.*)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],raw) + |------E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],downsample) + |-------T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) + |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(instance,EqualsRegex(.*)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],downsample) + |-------T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) + |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(instance,EqualsRegex(.*)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],downsample)""".stripMargin validatePlan(execPlan, expected) } @@ -459,23 +462,23 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS plannerParams = PlannerParams(processMultiPartition = true))) val expected = """T~AggregatePresenter(aggrOp=Count, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634777330)) - |-E~LocalPartitionReduceAggregateExec(aggrOp=Count, aggrParams=List()) on InProcessPlanDispatcher + |-E~LocalPartitionReduceAggregateExec(aggrOp=Count, aggrParams=List()) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) |--T~AggregateMapReduce(aggrOp=Count, aggrParams=List(), without=List(), by=List()) |---E~SetOperatorExec(binaryOp=LUnless, on=Some(List(instance)), ignoring=List()) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) |----E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192))) |-----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1681100802],raw) |------T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1681100802],raw) |------T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) + |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1681100802],raw) + |-----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1681100802],downsample) |------T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) + |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1681100802],downsample) |------T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |----E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |-----E~PromQlRemoteExec(PromQlQueryParams(bar{_ws_="demo",_ns_=~"remoteNs.*",instance=~".*"},100,1,1000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remotePartition-url0, requestTimeoutMs=10000) on InProcessPlanDispatcher - |-----E~PromQlRemoteExec(PromQlQueryParams(bar{_ws_="demo",_ns_=~"remoteNs.*",instance=~".*"},100,1,1000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1Partition-url1, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin + |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1681100802],downsample) + |----E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) + |-----E~PromQlRemoteExec(PromQlQueryParams(bar{instance=~".*",_ws_="demo",_ns_="remoteNs0"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remotePartition-url0, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) + |-----E~PromQlRemoteExec(PromQlQueryParams(bar{instance=~".*",_ws_="demo",_ns_="remoteNs1"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remote1Partition-url1, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin validatePlan(execPlan, expected) } @@ -506,7 +509,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS plannerParams = PlannerParams(processMultiPartition = true))) val expected = - """E~PromQlRemoteExec(PromQlQueryParams(sum(foo:1m{_ws_="demo",_ns_="RemoteNs",instance="Inst-1"}),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin + """E~PromQlRemoteExec(PromQlQueryParams(sum(foo:1m{_ws_ = "demo", _ns_ = "RemoteNs", instance = "Inst-1" }),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(filodb.core.query.QueryConfig@39de9bda)""".stripMargin validatePlan(execPlan, expected) } @@ -539,7 +542,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |-----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) |------T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |---E~PromQlRemoteExec(PromQlQueryParams(sum(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs"}),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin + |---E~PromQlRemoteExec(PromQlQueryParams(sum(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs"}),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin validatePlan(execPlan, expected) } @@ -553,7 +556,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS plannerParams = PlannerParams(processMultiPartition = true))) val expected = - """E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_ = "demo", _ns_ = "RemoteNs", instance = "Inst-1" }) or vector(0),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin + """E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_ = "demo", _ns_ = "RemoteNs", instance = "Inst-1" }) or vector(0),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin validatePlan(execPlan, expected) } @@ -589,7 +592,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |----FA1~StaticFuncArgs(0.0,RangeParams(1633913330,300,1634172830)) |----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |-E~PromQlRemoteExec(PromQlQueryParams((bar{_ws_="demo",_ns_="remoteNs0",instance="Inst-1"} or vector(0.0)),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin + |-E~PromQlRemoteExec(PromQlQueryParams((bar{_ws_="demo",_ns_="remoteNs0",instance="Inst-1"} or vector(0.0)),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin validatePlan(execPlan, expected) } @@ -597,24 +600,29 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS it("should not pushdown root scalar operation into a RemoteExec's promql when query spans multiple partitions") { val queryExpectedPairs = Seq( + // Instant selector without namespace regex. ("""foo{_ws_ = "demo", _ns_ = "RemoteNs", instance = "Inst-1" } > 5""", - """E~PromQlRemoteExec(PromQlQueryParams((foo{_ws_="demo",_ns_="RemoteNs",instance="Inst-1"} > 5.0),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), + // Should include the scalar op in the promql, since the query does not span multiple partitions. + """E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_ = "demo", _ns_ = "RemoteNs", instance = "Inst-1" } > 5,1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))"""), + // Instant selector with namesapce regex. + // NOTE: this can be optimized by pushing down the scalar operation. ("""foo{_ws_ = "demo", _ns_ =~ ".*Ns", instance = "Inst-1" } + 5""", - """T~ScalarOperationMapper(operator=ADD, scalarOnLhs=false) - |-FA1~StaticFuncArgs(5.0,RangeParams(1633913330,300,1634777330)) - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |--E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192))) - |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],raw) - |----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],raw) - |----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],raw) - |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],downsample) - |----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],downsample) - |----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],downsample) - |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~".*Ns",instance="Inst-1"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), + """T~ScalarOperationMapper(operator=ADD, scalarOnLhs=false) + |-FA1~StaticFuncArgs(5.0,RangeParams(1633913330,300,1634777330)) + |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |--E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,false,false,true)) + |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1122536221],raw) + |----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1122536221],raw) + |----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1122536221],raw) + |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1122536221],downsample) + |----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1122536221],downsample) + |----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1122536221],downsample) + |--E~PromQlRemoteExec(PromQlQueryParams(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + // Join with namespace regex in both operands. ("""5 * (foo{_ws_ = "demo", _ns_ =~ ".*Ns", instance = "Inst-1" } + foo{_ws_ = "demo", _ns_ =~ ".*Ns", instance = "Inst-1" })""", """T~ScalarOperationMapper(operator=MUL, scalarOnLhs=true) |-FA1~StaticFuncArgs(5.0,RangeParams(1633913330,300,1634777330)) @@ -647,78 +655,82 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |---E~PromQlRemoteExec(PromQlQueryParams(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), // Aggregate with namespace regex selector. ("""5 / sum(foo{_ws_ = "demo", _ns_ =~ ".*Ns", instance = "Inst-1" })""", - """T~ScalarOperationMapper(operator=DIV, scalarOnLhs=true) - |-FA1~StaticFuncArgs(5.0,RangeParams(1633913330,300,1634777330)) - |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634777330)) - |--E~MultiPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |---E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192))) - |----E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],raw) - |-----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |------T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],raw) - |-----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |------T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],raw) - |----E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],downsample) - |-----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |------T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],downsample) - |-----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |------T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],downsample) - |---E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~".*Ns",instance="Inst-1"}),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), + """T~ScalarOperationMapper(operator=DIV, scalarOnLhs=true) + |-FA1~StaticFuncArgs(5.0,RangeParams(1633913330,300,1634777330)) + |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634777330)) + |--E~MultiPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |---E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,false,false,true)) + |----E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1557189041],raw) + |-----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) + |------T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) + |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1557189041],raw) + |-----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) + |------T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) + |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1557189041],raw) + |----E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1557189041],downsample) + |-----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) + |------T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) + |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1557189041],downsample) + |-----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) + |------T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) + |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1557189041],downsample) + |---E~PromQlRemoteExec(PromQlQueryParams(sum(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs"}),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + // Function with namespace regex selector. + // NOTE: this can be optimized by pushing down the scalar operation. ("""sgn(foo{_ws_ = "demo", _ns_ =~ ".*Ns", instance = "Inst-1" }) == 5""", - """T~ScalarOperationMapper(operator=EQL, scalarOnLhs=false) - |-FA1~StaticFuncArgs(5.0,RangeParams(1633913330,300,1634777330)) - |-T~InstantVectorFunctionMapper(function=Sgn) - |--E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |---E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192))) - |----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],raw) - |-----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],raw) - |-----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],raw) - |----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],downsample) - |-----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],downsample) - |-----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],downsample) - |---E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~".*Ns",instance="Inst-1"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), + """T~ScalarOperationMapper(operator=EQL, scalarOnLhs=false) + |-FA1~StaticFuncArgs(5.0,RangeParams(1633913330,300,1634777330)) + |-T~InstantVectorFunctionMapper(function=Sgn) + |--E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |---E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,false,false,true)) + |----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-931342189],raw) + |-----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-931342189],raw) + |-----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-931342189],raw) + |----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-931342189],downsample) + |-----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-931342189],downsample) + |-----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-931342189],downsample) + |---E~PromQlRemoteExec(PromQlQueryParams(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + // scalar-wrapped vector ("""foo{_ws_ = "demo", _ns_ =~ ".*Ns", instance = "Inst-1" } - scalar(sum(foo{_ws_ = "demo", _ns_ =~ ".*Ns", instance = "Inst-1" }))""", - """T~ScalarOperationMapper(operator=SUB, scalarOnLhs=false) - |-FA1~ - |-T~ScalarFunctionMapper(function=Scalar, funcParams=List()) - |--T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634777330)) - |---E~MultiPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |----E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192))) - |-----E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],raw) - |------T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |-------T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],raw) - |------T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |-------T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],raw) - |-----E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],downsample) - |------T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |-------T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],downsample) - |------T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |-------T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],downsample) - |----E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~".*Ns",instance="Inst-1"}),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |--E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192))) - |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],raw) - |----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],raw) - |----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],raw) - |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],downsample) - |----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],downsample) - |----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],downsample) - |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~".*Ns",instance="Inst-1"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), + """T~ScalarOperationMapper(operator=SUB, scalarOnLhs=false) + |-FA1~ + |-T~ScalarFunctionMapper(function=Scalar, funcParams=List()) + |--T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634777330)) + |---E~MultiPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |----E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,false,false,true)) + |-----E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-855048932],raw) + |------T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) + |-------T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) + |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-855048932],raw) + |------T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) + |-------T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) + |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-855048932],raw) + |-----E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-855048932],downsample) + |------T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) + |-------T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) + |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-855048932],downsample) + |------T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) + |-------T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) + |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-855048932],downsample) + |----E~PromQlRemoteExec(PromQlQueryParams(sum(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs"}),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |--E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,false,false,true)) + |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-855048932],raw) + |----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-855048932],raw) + |----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-855048932],raw) + |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-855048932],downsample) + |----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-855048932],downsample) + |----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-855048932],downsample) + |--E~PromQlRemoteExec(PromQlQueryParams(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + // vector-wrapped scalar ("""foo{_ws_ = "demo", _ns_ =~ ".*Ns", instance = "Inst-1" } > vector(5)""", """E~BinaryJoinExec(binaryOp=GTR, on=None, ignoring=List()) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) @@ -739,230 +751,198 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS // join multiple adjacent scalars // NOTE: this can be optimized by pushing down the scalar operation. ("""5 * 5 * foo{_ws_ = "demo", _ns_ =~ ".*Ns", instance = "Inst-1" }""", - """T~ScalarOperationMapper(operator=MUL, scalarOnLhs=true) - |-FA1~ - |-E~ScalarBinaryOperationExec(params = RangeParams(1633913330,300,1634777330), operator = MUL, lhs = Left(5.0), rhs = Left(5.0)) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |--E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192))) - |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],raw) - |----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],raw) - |----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],raw) - |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],downsample) - |----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],downsample) - |----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],downsample) - |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~".*Ns",instance="Inst-1"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), + """T~ScalarOperationMapper(operator=MUL, scalarOnLhs=true) + |-FA1~ + |-E~ScalarBinaryOperationExec(params = RangeParams(1633913330,300,1634777330), operator = MUL, lhs = Left(5.0), rhs = Left(5.0)) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |--E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,false,false,true)) + |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#609935711],raw) + |----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#609935711],raw) + |----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#609935711],raw) + |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#609935711],downsample) + |----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#609935711],downsample) + |----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#609935711],downsample) + |--E~PromQlRemoteExec(PromQlQueryParams(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + // join multiple non-adjacent scalars + // NOTE: this can be optimized by pushing down the scalar operation. ("""5 * foo{_ws_ = "demo", _ns_ =~ ".*Ns", instance = "Inst-1" } * 5""", - """T~ScalarOperationMapper(operator=MUL, scalarOnLhs=false) - |-FA1~StaticFuncArgs(5.0,RangeParams(1633913330,300,1634777330)) - |-T~ScalarOperationMapper(operator=MUL, scalarOnLhs=true) - |--FA1~StaticFuncArgs(5.0,RangeParams(1633913330,300,1634777330)) - |--E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |---E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192))) - |----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],raw) - |-----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],raw) - |-----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],raw) - |----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],downsample) - |-----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],downsample) - |-----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],downsample) - |---E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~".*Ns",instance="Inst-1"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), + """T~ScalarOperationMapper(operator=MUL, scalarOnLhs=false) + |-FA1~StaticFuncArgs(5.0,RangeParams(1633913330,300,1634777330)) + |-T~ScalarOperationMapper(operator=MUL, scalarOnLhs=true) + |--FA1~StaticFuncArgs(5.0,RangeParams(1633913330,300,1634777330)) + |--E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |---E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,false,false,true)) + |----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-483218976],raw) + |-----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-483218976],raw) + |-----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-483218976],raw) + |----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-483218976],downsample) + |-----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-483218976],downsample) + |-----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-483218976],downsample) + |---E~PromQlRemoteExec(PromQlQueryParams(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + // sum(rate) ("""sum(rate(foo{_ws_ = "demo", _ns_ =~ ".*Ns", instance = "Inst-1" }[5m])) > 5""", - """T~ScalarOperationMapper(operator=GTR, scalarOnLhs=false) - |-FA1~StaticFuncArgs(5.0,RangeParams(1633913330,300,1634777330)) - |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634777330)) - |--E~MultiPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |---E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192))) - |----E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],raw) - |-----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |------T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=Some(300000), functionId=Some(Rate), rawSource=true, offsetMs=None) - |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],raw) - |-----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |------T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=Some(300000), functionId=Some(Rate), rawSource=true, offsetMs=None) - |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],raw) - |----E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],downsample) - |-----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |------T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=Some(300000), functionId=Some(Rate), rawSource=true, offsetMs=None) - |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],downsample) - |-----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |------T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=Some(300000), functionId=Some(Rate), rawSource=true, offsetMs=None) - |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],downsample) - |---E~PromQlRemoteExec(PromQlQueryParams(sum(rate(foo{_ws_="demo",_ns_=~".*Ns",instance="Inst-1"}[300s])),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), + """T~ScalarOperationMapper(operator=GTR, scalarOnLhs=false) + |-FA1~StaticFuncArgs(5.0,RangeParams(1633913330,300,1634777330)) + |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634777330)) + |--E~MultiPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |---E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,false,false,true)) + |----E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-725608106],raw) + |-----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) + |------T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=Some(300000), functionId=Some(Rate), rawSource=true, offsetMs=None) + |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-725608106],raw) + |-----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) + |------T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=Some(300000), functionId=Some(Rate), rawSource=true, offsetMs=None) + |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-725608106],raw) + |----E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-725608106],downsample) + |-----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) + |------T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=Some(300000), functionId=Some(Rate), rawSource=true, offsetMs=None) + |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-725608106],downsample) + |-----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) + |------T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=Some(300000), functionId=Some(Rate), rawSource=true, offsetMs=None) + |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-725608106],downsample) + |---E~PromQlRemoteExec(PromQlQueryParams(sum(rate(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs"}[300s])),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + // subquery ("""rate(foo{_ws_ = "demo", _ns_ =~ ".*Ns", instance = "Inst-1" }[1h:5m]) - 5""", - """T~ScalarOperationMapper(operator=SUB, scalarOnLhs=false) - |-FA1~StaticFuncArgs(5.0,RangeParams(1633913330,300,1634777330)) - |-T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634777330000, window=Some(3600000), functionId=Some(Rate), rawSource=false, offsetMs=None) - |--E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |---E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192))) - |----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],raw) - |-----T~PeriodicSamplesMapper(start=1634172900000, step=300000, end=1634777100000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172600000,1634777100000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],raw) - |-----T~PeriodicSamplesMapper(start=1634172900000, step=300000, end=1634777100000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172600000,1634777100000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],raw) - |----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],downsample) - |-----T~PeriodicSamplesMapper(start=1633909800000, step=300000, end=1634172600000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633909500000,1634172600000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],downsample) - |-----T~PeriodicSamplesMapper(start=1633909800000, step=300000, end=1634172600000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633909500000,1634172600000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],downsample) - |---E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~".*Ns",instance="Inst-1"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), + """T~ScalarOperationMapper(operator=SUB, scalarOnLhs=false) + |-FA1~StaticFuncArgs(5.0,RangeParams(1633913330,300,1634777330)) + |-T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634777330000, window=Some(3600000), functionId=Some(Rate), rawSource=false, offsetMs=None) + |--E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |---E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,false,false,true)) + |----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1279558207],raw) + |-----T~PeriodicSamplesMapper(start=1634172900000, step=300000, end=1634777100000, window=None, functionId=None, rawSource=true, offsetMs=None) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172600000,1634777100000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1279558207],raw) + |-----T~PeriodicSamplesMapper(start=1634172900000, step=300000, end=1634777100000, window=None, functionId=None, rawSource=true, offsetMs=None) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172600000,1634777100000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1279558207],raw) + |----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1279558207],downsample) + |-----T~PeriodicSamplesMapper(start=1633909800000, step=300000, end=1634172600000, window=None, functionId=None, rawSource=true, offsetMs=None) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633909500000,1634172600000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1279558207],downsample) + |-----T~PeriodicSamplesMapper(start=1633909800000, step=300000, end=1634172600000, window=None, functionId=None, rawSource=true, offsetMs=None) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633909500000,1634172600000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1279558207],downsample) + |---E~PromQlRemoteExec(PromQlQueryParams(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs"},1633909800,300,1634777100,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + // X_over_time ("""5 / count_over_time(foo{_ws_ = "demo", _ns_ =~ ".*Ns", instance = "Inst-1" }[20m])""", - """T~ScalarOperationMapper(operator=DIV, scalarOnLhs=true) - |-FA1~StaticFuncArgs(5.0,RangeParams(1633913330,300,1634777330)) - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |--E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192))) - |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],raw) - |----T~PeriodicSamplesMapper(start=1634174030000, step=300000, end=1634777330000, window=Some(1200000), functionId=Some(CountOverTime), rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],raw) - |----T~PeriodicSamplesMapper(start=1634174030000, step=300000, end=1634777330000, window=Some(1200000), functionId=Some(CountOverTime), rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],raw) - |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],downsample) - |----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634173730000, window=Some(1200000), functionId=Some(CountOverTime), rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633912130000,1634173730000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],downsample) - |----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634173730000, window=Some(1200000), functionId=Some(CountOverTime), rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633912130000,1634173730000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],downsample) - |--E~PromQlRemoteExec(PromQlQueryParams(count_over_time(foo{_ws_="demo",_ns_=~".*Ns",instance="Inst-1"}[1200s]),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), + """T~ScalarOperationMapper(operator=DIV, scalarOnLhs=true) + |-FA1~StaticFuncArgs(5.0,RangeParams(1633913330,300,1634777330)) + |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |--E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,false,false,true)) + |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1005674756],raw) + |----T~PeriodicSamplesMapper(start=1634174030000, step=300000, end=1634777330000, window=Some(1200000), functionId=Some(CountOverTime), rawSource=true, offsetMs=None) + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1005674756],raw) + |----T~PeriodicSamplesMapper(start=1634174030000, step=300000, end=1634777330000, window=Some(1200000), functionId=Some(CountOverTime), rawSource=true, offsetMs=None) + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1005674756],raw) + |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1005674756],downsample) + |----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634173730000, window=Some(1200000), functionId=Some(CountOverTime), rawSource=true, offsetMs=None) + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633912130000,1634173730000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1005674756],downsample) + |----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634173730000, window=Some(1200000), functionId=Some(CountOverTime), rawSource=true, offsetMs=None) + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633912130000,1634173730000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1005674756],downsample) + |--E~PromQlRemoteExec(PromQlQueryParams(count_over_time(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs"}[1200s]),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + // topk -- shard key regex not supported + // ("""5 <= topk(2, foo{_ws_ = "demo", _ns_ =~ ".*Ns", instance = "Inst-1" })""", + // """""".stripMargin), + // histogram_quantile ("""histogram_quantile(0.9, foo{_ws_ = "demo", _ns_ =~ ".*Ns", instance = "Inst-1" }) - 5""", - """T~ScalarOperationMapper(operator=SUB, scalarOnLhs=false) - |-FA1~StaticFuncArgs(5.0,RangeParams(1633913330,300,1634777330)) - |-T~InstantVectorFunctionMapper(function=HistogramQuantile) - |--FA1~StaticFuncArgs(0.9,RangeParams(1633913330,300,1634777330)) - |--E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |---E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192))) - |----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],raw) - |-----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],raw) - |-----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],raw) - |----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],downsample) - |-----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],downsample) - |-----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#598303643],downsample) - |---E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~".*Ns",instance="Inst-1"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin) + """T~ScalarOperationMapper(operator=SUB, scalarOnLhs=false) + |-FA1~StaticFuncArgs(5.0,RangeParams(1633913330,300,1634777330)) + |-T~InstantVectorFunctionMapper(function=HistogramQuantile) + |--FA1~StaticFuncArgs(0.9,RangeParams(1633913330,300,1634777330)) + |--E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |---E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,false,false,true)) + |----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1532463646],raw) + |-----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1532463646],raw) + |-----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1532463646],raw) + |----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1532463646],downsample) + |-----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1532463646],downsample) + |-----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1532463646],downsample) + |---E~PromQlRemoteExec(PromQlQueryParams(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), ) - val plans = new mutable.ArrayBuffer[ExecPlan] for ((query, rootPlan) <- queryExpectedPairs) { val queryParams = PromQlQueryParams(query, startSeconds, step, endSeconds) val lp = Parser.queryRangeToLogicalPlan(query, TimeStepParams(startSeconds, step, endSeconds), Antlr) val execPlan = rootPlanner.materialize(lp, QueryContext(origQueryParams = queryParams, plannerParams = PlannerParams(processMultiPartition = true))) validatePlan(execPlan, rootPlan) - plans.append(execPlan) - } - printTests(queryExpectedPairs.map(_._1).zip(plans)) - } - - def printTests(queryPlanPairs: Seq[(String, ExecPlan)]): Unit = { - val doPrint = true - if (!doPrint) { - return - } - val indentCount = 4 - val firstIndent = " ".repeat(indentCount) - val tailIndents = " ".repeat(indentCount + 2) - val quotes = "\"".repeat(3) - val f = queryPlanPairs.map { case (query, expected) => - val planStr = expected.printTree().replaceAll("\n", s"\n$tailIndents|") - s"""($quotes$query$quotes,\n$firstIndent$quotes$planStr$quotes.stripMargin)""" - } - println(f.mkString(",\n")) - } - - def printTests2(queryPlanPairs: Seq[(String, Set[String], ExecPlan)]): Unit = { - val doPrint = true - if (!doPrint) { - return - } - val indentCount = 4 - val firstIndent = " ".repeat(indentCount) - val tailIndents = " ".repeat(indentCount + 14) - val quotes = "\"".repeat(3) -// Test(query = s"""sum(foo{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bar"}) by ($TSCHEMA_LABEL)""", -// tschemaEnabled = Set("local1"), -// expected = """E~Stitc - val f = queryPlanPairs.map { case (query, set, expected) => - val planStr = expected.printTree().replaceAll("\n", s"\n$tailIndents|") - s"""Test(query = $quotes$query$quotes,\n${firstIndent}tschemaEnabled = Set(${set.map(f => s""""$f"""").mkString(", ")}),\n${firstIndent}expected = $quotes$planStr$quotes.stripMargin)""" } - println(f.mkString(",\n")) - } - - def printTests3(queryPlanPairs: Seq[(String, Long, Long, ExecPlan)]): Unit = { - val doPrint = true - if (!doPrint) { - return - } - val indentCount = 4 - val firstIndent = " ".repeat(indentCount) - val tailIndents = " ".repeat(indentCount + 14) - val quotes = "\"".repeat(3) - // Test(query = s"""sum(foo{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bar"}) by ($TSCHEMA_LABEL)""", - // tschemaEnabled = Set("local1"), - // expected = """E~Stitc - val f = queryPlanPairs.map { case (query, lookbackSec, offsetSec, expected) => - val planStr = expected.printTree().replaceAll("\n", s"\n$tailIndents|") - s"""Test(query = $quotes$query$quotes,\n${firstIndent}lookbackSec = $lookbackSec,\n${firstIndent}offsetSec = $offsetSec,\n${firstIndent}expected = $quotes$planStr$quotes.stripMargin)""" - } - println(f.mkString(",\n")) } it("should pushdown root scalar operations into a RemoteExec's promql when query spans single remote partition") { val queryExpectedPairs = Seq( + // Instant selector without namespace regex. ("""foo{_ws_ = "demo", _ns_ = "RemoteNs", instance = "Inst-1" } > 5""", - """E~PromQlRemoteExec(PromQlQueryParams((foo{_ws_="demo",_ns_="RemoteNs",instance="Inst-1"} > 5.0),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), + """E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_ = "demo", _ns_ = "RemoteNs", instance = "Inst-1" } > 5,1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))"""), + // Instant selector with namespace regex. ("""foo{_ws_ = "demo", _ns_ =~ ".*remoteNs", instance = "Inst-1" } + 5""", - """E~PromQlRemoteExec(PromQlQueryParams((foo{_ws_="demo",_ns_=~".*remoteNs",instance="Inst-1"} + 5.0),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), + // Should include the scalar in the promql, since it will only hit the remote partition. + """E~PromQlRemoteExec(PromQlQueryParams((foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs"} + 5.0),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + // Join with namespace regex in both operands. ("""5 * (foo{_ws_ = "demo", _ns_ =~ ".*remoteNs", instance = "Inst-1" } + foo{_ws_ = "demo", _ns_ =~ ".*remoteNs", instance = "Inst-1" })""", - """E~PromQlRemoteExec(PromQlQueryParams((5.0 * (foo{_ws_="demo",_ns_=~".*remoteNs",instance="Inst-1"} + foo{_ws_="demo",_ns_=~".*remoteNs",instance="Inst-1"})),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), + """E~PromQlRemoteExec(PromQlQueryParams((5.0 * (foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs"} + foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs"})),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))"""), + // Aggregate with namespace regex selector. ("""5 / sum(foo{_ws_ = "demo", _ns_ =~ ".*remoteNs", instance = "Inst-1" })""", - """E~PromQlRemoteExec(PromQlQueryParams((5.0 / sum(foo{_ws_="demo",_ns_=~".*remoteNs",instance="Inst-1"})),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), + """E~PromQlRemoteExec(PromQlQueryParams((5.0 / sum(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs"})),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))"""), + // Function with namespace regex selector. ("""sgn(foo{_ws_ = "demo", _ns_ =~ ".*remoteNs", instance = "Inst-1" }) == 5""", - """E~PromQlRemoteExec(PromQlQueryParams((sgn(foo{_ws_="demo",_ns_=~".*remoteNs",instance="Inst-1"}) == 5.0),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), + """E~PromQlRemoteExec(PromQlQueryParams((sgn(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs"}) == 5.0),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))"""), + // scalar-wrapped vector ("""foo{_ws_ = "demo", _ns_ =~ ".*remoteNs", instance = "Inst-1" } - scalar(sum(foo{_ws_ = "demo", _ns_ =~ ".*remoteNs", instance = "Inst-1" }))""", - """E~PromQlRemoteExec(PromQlQueryParams((foo{_ws_="demo",_ns_=~".*remoteNs",instance="Inst-1"} - sum(foo{_ws_="demo",_ns_=~".*remoteNs",instance="Inst-1"})),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), + """E~PromQlRemoteExec(PromQlQueryParams((foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs"} - sum(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs"})),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + // vector-wrapped scalar ("""foo{_ws_ = "demo", _ns_ =~ ".*remoteNs", instance = "Inst-1" } > vector(5)""", - """E~PromQlRemoteExec(PromQlQueryParams((foo{_ws_="demo",_ns_=~".*remoteNs",instance="Inst-1"} > vector(5.0)),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), + """E~PromQlRemoteExec(PromQlQueryParams((foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs"} > vector(5.0)),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + // join multiple adjacent scalars ("""5 * 5 * foo{_ws_ = "demo", _ns_ =~ ".*remoteNs", instance = "Inst-1" }""", - """E~PromQlRemoteExec(PromQlQueryParams(((5.0 * 5.0) * foo{_ws_="demo",_ns_=~".*remoteNs",instance="Inst-1"}),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), + """E~PromQlRemoteExec(PromQlQueryParams(((5.0 * 5.0) * foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs"}),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + // join multiple non-adjacent scalars ("""5 * foo{_ws_ = "demo", _ns_ =~ ".*remoteNs", instance = "Inst-1" } * 5""", - """E~PromQlRemoteExec(PromQlQueryParams(((5.0 * foo{_ws_="demo",_ns_=~".*remoteNs",instance="Inst-1"}) * 5.0),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), + """E~PromQlRemoteExec(PromQlQueryParams(((5.0 * foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs"}) * 5.0),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + // sum(rate) ("""sum(rate(foo{_ws_ = "demo", _ns_ =~ ".*remoteNs", instance = "Inst-1" }[5m])) > 5""", - """E~PromQlRemoteExec(PromQlQueryParams((sum(rate(foo{_ws_="demo",_ns_=~".*remoteNs",instance="Inst-1"}[300s])) > 5.0),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), + """E~PromQlRemoteExec(PromQlQueryParams((sum(rate(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs"}[300s])) > 5.0),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + // subquery ("""rate(foo{_ws_ = "demo", _ns_ =~ ".*remoteNs", instance = "Inst-1" }[1h:5m]) - 5""", - """E~PromQlRemoteExec(PromQlQueryParams((rate(foo{_ws_="demo",_ns_=~".*remoteNs",instance="Inst-1"}[3600s:300s]) - 5.0),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), + """E~PromQlRemoteExec(PromQlQueryParams((rate(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs"}[3600s:300s]) - 5.0),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + // X_over_time ("""5 / count_over_time(foo{_ws_ = "demo", _ns_ =~ ".*remoteNs", instance = "Inst-1" }[20m])""", - """E~PromQlRemoteExec(PromQlQueryParams((5.0 / count_over_time(foo{_ws_="demo",_ns_=~".*remoteNs",instance="Inst-1"}[1200s])),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), + """E~PromQlRemoteExec(PromQlQueryParams((5.0 / count_over_time(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs"}[1200s])),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + // topk ("""5 <= topk(2, foo{_ws_ = "demo", _ns_ =~ ".*remoteNs", instance = "Inst-1" })""", - """E~PromQlRemoteExec(PromQlQueryParams((5.0 <= topk(2.0,foo{_ws_="demo",_ns_=~".*remoteNs",instance="Inst-1"})),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), + """E~PromQlRemoteExec(PromQlQueryParams((5.0 <= topk(2.0,foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs"})),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + // histogram_quantile ("""histogram_quantile(0.9, foo{_ws_ = "demo", _ns_ =~ ".*remoteNs", instance = "Inst-1" }) - 5""", - """E~PromQlRemoteExec(PromQlQueryParams((histogram_quantile(0.9,foo{_ws_="demo",_ns_=~".*remoteNs",instance="Inst-1"}) - 5.0),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin) + """E~PromQlRemoteExec(PromQlQueryParams((histogram_quantile(0.9,foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs"}) - 5.0),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), ) - val plans = new ArrayBuffer[ExecPlan] for ((query, otherPlan) <- queryExpectedPairs) { val queryParams = PromQlQueryParams(query, startSeconds, step, endSeconds) val lp = Parser.queryRangeToLogicalPlan(query, TimeStepParams(startSeconds, step, endSeconds), Antlr) val execPlan = oneRemoteRootPlanner.materialize(lp, QueryContext(origQueryParams = queryParams, plannerParams = PlannerParams(processMultiPartition = true))) validatePlan(execPlan, otherPlan) - plans.append(execPlan) } - printTests(queryExpectedPairs.map(_._1).zip(plans)) } it("should not pushdown root scalar operations into a RemoteExec's promql when query spans multiple remote partition") { val queryExpectedPairs = Seq( + // Instant selector with namespace regex. + // NOTE: this can be optimized by pushing down the scalar operation. ("""foo{_ws_ = "demo", _ns_ =~ "remoteNs.*", instance = "Inst-1" } + 5""", - """T~ScalarOperationMapper(operator=ADD, scalarOnLhs=false) - |-FA1~StaticFuncArgs(5.0,RangeParams(1633913330,300,1634777330)) - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"remoteNs.*",instance="Inst-1"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remotePartition-url0, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"remoteNs.*",instance="Inst-1"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1Partition-url1, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), + // Should include the scalar in the promql, since it will only hit the remote partition. + """T~ScalarOperationMapper(operator=ADD, scalarOnLhs=false) + |-FA1~StaticFuncArgs(5.0,RangeParams(1633913330,300,1634777330)) + |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs0"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remotePartition-url0, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs1"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remote1Partition-url1, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + // Join with namespace regex in both operands. ("""5 * (foo{_ws_ = "demo", _ns_ =~ "remoteNs.*", instance = "Inst-1" } + foo{_ws_ = "demo", _ns_ =~ "remoteNs.*", instance = "Inst-1" })""", """T~ScalarOperationMapper(operator=MUL, scalarOnLhs=true) |-FA1~StaticFuncArgs(5.0,RangeParams(1633913330,300,1634777330)) @@ -975,30 +955,34 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |---E~PromQlRemoteExec(PromQlQueryParams(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs1"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remote1Partition-url1, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), // Aggregate with namespace regex selector. ("""5 / sum(foo{_ws_ = "demo", _ns_ =~ "remoteNs.*", instance = "Inst-1" })""", - """T~ScalarOperationMapper(operator=DIV, scalarOnLhs=true) - |-FA1~StaticFuncArgs(5.0,RangeParams(1633913330,300,1634777330)) - |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634777330)) - |--E~MultiPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |---E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"remoteNs.*",instance="Inst-1"}),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false,true), queryEndpoint=remotePartition-url0, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |---E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"remoteNs.*",instance="Inst-1"}),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false,true), queryEndpoint=remote1Partition-url1, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), + """T~ScalarOperationMapper(operator=DIV, scalarOnLhs=true) + |-FA1~StaticFuncArgs(5.0,RangeParams(1633913330,300,1634777330)) + |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634777330)) + |--E~MultiPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |---E~PromQlRemoteExec(PromQlQueryParams(sum(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs0"}),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remotePartition-url0, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |---E~PromQlRemoteExec(PromQlQueryParams(sum(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs1"}),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remote1Partition-url1, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + // Function with namespace regex selector. + // NOTE: this can be optimized by pushing down the scalar operation. ("""sgn(foo{_ws_ = "demo", _ns_ =~ "remoteNs.*", instance = "Inst-1" }) == 5""", - """T~ScalarOperationMapper(operator=EQL, scalarOnLhs=false) - |-FA1~StaticFuncArgs(5.0,RangeParams(1633913330,300,1634777330)) - |-T~InstantVectorFunctionMapper(function=Sgn) - |--E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |---E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"remoteNs.*",instance="Inst-1"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remotePartition-url0, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |---E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"remoteNs.*",instance="Inst-1"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1Partition-url1, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), + """T~ScalarOperationMapper(operator=EQL, scalarOnLhs=false) + |-FA1~StaticFuncArgs(5.0,RangeParams(1633913330,300,1634777330)) + |-T~InstantVectorFunctionMapper(function=Sgn) + |--E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |---E~PromQlRemoteExec(PromQlQueryParams(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs0"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remotePartition-url0, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |---E~PromQlRemoteExec(PromQlQueryParams(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs1"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remote1Partition-url1, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + // scalar-wrapped vector ("""foo{_ws_ = "demo", _ns_ =~ "remoteNs.*", instance = "Inst-1" } - scalar(sum(foo{_ws_ = "demo", _ns_ =~ "remoteNs.*", instance = "Inst-1" }))""", - """T~ScalarOperationMapper(operator=SUB, scalarOnLhs=false) - |-FA1~ - |-T~ScalarFunctionMapper(function=Scalar, funcParams=List()) - |--T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634777330)) - |---E~MultiPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |----E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"remoteNs.*",instance="Inst-1"}),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false,true), queryEndpoint=remotePartition-url0, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |----E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"remoteNs.*",instance="Inst-1"}),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false,true), queryEndpoint=remote1Partition-url1, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"remoteNs.*",instance="Inst-1"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remotePartition-url0, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"remoteNs.*",instance="Inst-1"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1Partition-url1, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), + """T~ScalarOperationMapper(operator=SUB, scalarOnLhs=false) + |-FA1~ + |-T~ScalarFunctionMapper(function=Scalar, funcParams=List()) + |--T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634777330)) + |---E~MultiPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |----E~PromQlRemoteExec(PromQlQueryParams(sum(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs0"}),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remotePartition-url0, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |----E~PromQlRemoteExec(PromQlQueryParams(sum(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs1"}),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remote1Partition-url1, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs0"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remotePartition-url0, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs1"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remote1Partition-url1, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + // vector-wrapped scalar (sanity check-- not expected to treat this as a scalar operation) ("""foo{_ws_ = "demo", _ns_ =~ "remoteNs.*", instance = "Inst-1" } > vector(5)""", """E~BinaryJoinExec(binaryOp=GTR, on=None, ignoring=List()) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) @@ -1009,60 +993,65 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS // join multiple adjacent scalars // NOTE: this can be optimized by pushing down the scalar operation. ("""5 * 5 * foo{_ws_ = "demo", _ns_ =~ "remoteNs.*", instance = "Inst-1" }""", - """T~ScalarOperationMapper(operator=MUL, scalarOnLhs=true) - |-FA1~ - |-E~ScalarBinaryOperationExec(params = RangeParams(1633913330,300,1634777330), operator = MUL, lhs = Left(5.0), rhs = Left(5.0)) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"remoteNs.*",instance="Inst-1"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remotePartition-url0, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"remoteNs.*",instance="Inst-1"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1Partition-url1, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), + """T~ScalarOperationMapper(operator=MUL, scalarOnLhs=true) + |-FA1~ + |-E~ScalarBinaryOperationExec(params = RangeParams(1633913330,300,1634777330), operator = MUL, lhs = Left(5.0), rhs = Left(5.0)) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs0"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remotePartition-url0, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs1"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remote1Partition-url1, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + // join multiple non-adjacent scalars + // NOTE: this can be optimized by pushing down the scalar operation. ("""5 * foo{_ws_ = "demo", _ns_ =~ "remoteNs.*", instance = "Inst-1" } * 5""", - """T~ScalarOperationMapper(operator=MUL, scalarOnLhs=false) - |-FA1~StaticFuncArgs(5.0,RangeParams(1633913330,300,1634777330)) - |-T~ScalarOperationMapper(operator=MUL, scalarOnLhs=true) - |--FA1~StaticFuncArgs(5.0,RangeParams(1633913330,300,1634777330)) - |--E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |---E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"remoteNs.*",instance="Inst-1"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remotePartition-url0, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |---E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"remoteNs.*",instance="Inst-1"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1Partition-url1, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), + """T~ScalarOperationMapper(operator=MUL, scalarOnLhs=false) + |-FA1~StaticFuncArgs(5.0,RangeParams(1633913330,300,1634777330)) + |-T~ScalarOperationMapper(operator=MUL, scalarOnLhs=true) + |--FA1~StaticFuncArgs(5.0,RangeParams(1633913330,300,1634777330)) + |--E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |---E~PromQlRemoteExec(PromQlQueryParams(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs0"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remotePartition-url0, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |---E~PromQlRemoteExec(PromQlQueryParams(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs1"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remote1Partition-url1, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + // sum(rate) ("""sum(rate(foo{_ws_ = "demo", _ns_ =~ "remoteNs.*", instance = "Inst-1" }[5m])) > 5""", - """T~ScalarOperationMapper(operator=GTR, scalarOnLhs=false) - |-FA1~StaticFuncArgs(5.0,RangeParams(1633913330,300,1634777330)) - |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634777330)) - |--E~MultiPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |---E~PromQlRemoteExec(PromQlQueryParams(sum(rate(foo{_ws_="demo",_ns_=~"remoteNs.*",instance="Inst-1"}[300s])),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false,true), queryEndpoint=remotePartition-url0, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |---E~PromQlRemoteExec(PromQlQueryParams(sum(rate(foo{_ws_="demo",_ns_=~"remoteNs.*",instance="Inst-1"}[300s])),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false,true), queryEndpoint=remote1Partition-url1, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), + """T~ScalarOperationMapper(operator=GTR, scalarOnLhs=false) + |-FA1~StaticFuncArgs(5.0,RangeParams(1633913330,300,1634777330)) + |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634777330)) + |--E~MultiPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |---E~PromQlRemoteExec(PromQlQueryParams(sum(rate(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs0"}[300s])),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remotePartition-url0, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |---E~PromQlRemoteExec(PromQlQueryParams(sum(rate(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs1"}[300s])),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remote1Partition-url1, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + // subquery ("""rate(foo{_ws_ = "demo", _ns_ =~ "remoteNs.*", instance = "Inst-1" }[1h:5m]) - 5""", - """T~ScalarOperationMapper(operator=SUB, scalarOnLhs=false) - |-FA1~StaticFuncArgs(5.0,RangeParams(1633913330,300,1634777330)) - |-T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634777330000, window=Some(3600000), functionId=Some(Rate), rawSource=false, offsetMs=None) - |--E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |---E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"remoteNs.*",instance="Inst-1"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remotePartition-url0, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |---E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"remoteNs.*",instance="Inst-1"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1Partition-url1, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), + """T~ScalarOperationMapper(operator=SUB, scalarOnLhs=false) + |-FA1~StaticFuncArgs(5.0,RangeParams(1633913330,300,1634777330)) + |-T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634777330000, window=Some(3600000), functionId=Some(Rate), rawSource=false, offsetMs=None) + |--E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |---E~PromQlRemoteExec(PromQlQueryParams(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs0"},1633909800,300,1634777100,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remotePartition-url0, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |---E~PromQlRemoteExec(PromQlQueryParams(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs1"},1633909800,300,1634777100,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remote1Partition-url1, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + // X_over_time ("""5 / count_over_time(foo{_ws_ = "demo", _ns_ =~ "remoteNs.*", instance = "Inst-1" }[20m])""", - """T~ScalarOperationMapper(operator=DIV, scalarOnLhs=true) - |-FA1~StaticFuncArgs(5.0,RangeParams(1633913330,300,1634777330)) - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |--E~PromQlRemoteExec(PromQlQueryParams(count_over_time(foo{_ws_="demo",_ns_=~"remoteNs.*",instance="Inst-1"}[1200s]),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remotePartition-url0, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |--E~PromQlRemoteExec(PromQlQueryParams(count_over_time(foo{_ws_="demo",_ns_=~"remoteNs.*",instance="Inst-1"}[1200s]),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1Partition-url1, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), + """T~ScalarOperationMapper(operator=DIV, scalarOnLhs=true) + |-FA1~StaticFuncArgs(5.0,RangeParams(1633913330,300,1634777330)) + |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(count_over_time(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs0"}[1200s]),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remotePartition-url0, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(count_over_time(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs1"}[1200s]),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remote1Partition-url1, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + // topk -- unsupported for shard key regex + // ("""5 <= topk(2, foo{_ws_ = "demo", _ns_ =~ "remoteNs.*", instance = "Inst-1" })""", + // """""".stripMargin), + // histogram_quantile ("""histogram_quantile(0.9, foo{_ws_ = "demo", _ns_ =~ "remoteNs.*", instance = "Inst-1" }) - 5""", - """T~ScalarOperationMapper(operator=SUB, scalarOnLhs=false) - |-FA1~StaticFuncArgs(5.0,RangeParams(1633913330,300,1634777330)) - |-T~InstantVectorFunctionMapper(function=HistogramQuantile) - |--FA1~StaticFuncArgs(0.9,RangeParams(1633913330,300,1634777330)) - |--E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |---E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"remoteNs.*",instance="Inst-1"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remotePartition-url0, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |---E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"remoteNs.*",instance="Inst-1"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1Partition-url1, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin) + """T~ScalarOperationMapper(operator=SUB, scalarOnLhs=false) + |-FA1~StaticFuncArgs(5.0,RangeParams(1633913330,300,1634777330)) + |-T~InstantVectorFunctionMapper(function=HistogramQuantile) + |--FA1~StaticFuncArgs(0.9,RangeParams(1633913330,300,1634777330)) + |--E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |---E~PromQlRemoteExec(PromQlQueryParams(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs0"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remotePartition-url0, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |---E~PromQlRemoteExec(PromQlQueryParams(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs1"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remote1Partition-url1, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), ) - - val plans = new ArrayBuffer[ExecPlan] for ((query, otherPlan) <- queryExpectedPairs) { val queryParams = PromQlQueryParams(query, startSeconds, step, endSeconds) val lp = Parser.queryRangeToLogicalPlan(query, TimeStepParams(startSeconds, step, endSeconds), Antlr) val execPlan = twoRemoteRootPlanner.materialize(lp, QueryContext(origQueryParams = queryParams, plannerParams = PlannerParams(processMultiPartition = true))) - validatePlan(execPlan, otherPlan, sort = true) - plans.append(execPlan) + validatePlan(execPlan, otherPlan) } - printTests(queryExpectedPairs.map(_._1).zip(plans)) } it("should generate plan for recording rule query spanning multiple partitions") { @@ -1139,23 +1128,24 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS val expected = """T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634777330)) - |-E~MultiPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on InProcessPlanDispatcher - |--E~StitchRvsExec() on InProcessPlanDispatcher - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |-E~MultiPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on InProcessPlanDispatcher(filodb.core.query.QueryConfig@5974b7e8) + |--E~StitchRvsExec() on InProcessPlanDispatcher(filodb.core.query.EmptyQueryConfig$@2839e3c8) + |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1211350849],raw) |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) |-----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1211350849],raw) |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) |-----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1211350849],raw) + |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1211350849],downsample) |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) |-----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1211350849],downsample) |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) |-----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~".*Ns",instance="Inst-1"}),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1211350849],downsample) + |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs"}),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(filodb.core.query.QueryConfig@5974b7e8)""".stripMargin + validatePlan(execPlan, expected) } @@ -1176,18 +1166,18 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) |-----T~PeriodicSamplesMapper(start=1634172900000, step=300000, end=1634775000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172600000,1634775000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172600000,1634775000000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) |-----T~PeriodicSamplesMapper(start=1634172900000, step=300000, end=1634775000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172600000,1634775000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172600000,1634775000000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) |-----T~PeriodicSamplesMapper(start=1633911000000, step=300000, end=1634172600000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633910700000,1634172600000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633910700000,1634172600000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) |-----T~PeriodicSamplesMapper(start=1633911000000, step=300000, end=1634172600000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633910700000,1634172600000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~".*Ns",instance="Inst-1"}),1633911000,300,1634775000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633910700000,1634172600000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) + |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs"}),1633911000,300,1634775000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin validatePlan(execPlan, expected) } @@ -1207,18 +1197,18 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) |-----T~PeriodicSamplesMapper(start=1634172900000, step=300000, end=1634775000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172600000,1634775000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172600000,1634775000000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) |-----T~PeriodicSamplesMapper(start=1634172900000, step=300000, end=1634775000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172600000,1634775000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172600000,1634775000000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) |-----T~PeriodicSamplesMapper(start=1633911000000, step=300000, end=1634172600000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633910700000,1634172600000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633910700000,1634172600000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) |-----T~PeriodicSamplesMapper(start=1633911000000, step=300000, end=1634172600000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633910700000,1634172600000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~".*Ns",instance="Inst-1"}),1634775000,0,1634775000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633910700000,1634172600000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) + |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs"}),1633911000,300,1634775000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin validatePlan(execPlan, expected) } @@ -1236,15 +1226,15 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |-E~StitchRvsExec() on InProcessPlanDispatcher |--E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) |---T~PeriodicSamplesMapper(start=1634172900000, step=300000, end=1634775000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172600000,1634775000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172600000,1634775000000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) |---T~PeriodicSamplesMapper(start=1634172900000, step=300000, end=1634775000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172600000,1634775000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172600000,1634775000000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) |--E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) |---T~PeriodicSamplesMapper(start=1633911000000, step=300000, end=1634172600000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633910700000,1634172600000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) + |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633910700000,1634172600000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) |---T~PeriodicSamplesMapper(start=1633911000000, step=300000, end=1634172600000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633910700000,1634172600000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |-E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~".*Ns",instance="Inst-1"},1634775000,0,1634775000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin + |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633910700000,1634172600000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) + |-E~PromQlRemoteExec(PromQlQueryParams(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs"},1633911000,300,1634775000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin validatePlan(execPlan, expected) } @@ -1266,10 +1256,10 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher |--E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) |---T~PeriodicSamplesMapper(start=1634343000000, step=300000, end=1634775000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634342700000,1634775000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634342700000,1634775000000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) |---T~PeriodicSamplesMapper(start=1634343000000, step=300000, end=1634775000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634342700000,1634775000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~".*Ns",instance="Inst-1"},1634775000,0,1634775000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin + |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634342700000,1634775000000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |--E~PromQlRemoteExec(PromQlQueryParams(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs"},1634343000,300,1634775000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin validatePlan(execPlan, expected) } @@ -1285,10 +1275,10 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS """E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher |-E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) |--T~PeriodicSamplesMapper(start=1634775000000, step=0, end=1634775000000, window=Some(432000000), functionId=Some(SumOverTime), rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634343000000,1634775000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634343000000,1634775000000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) |--T~PeriodicSamplesMapper(start=1634775000000, step=0, end=1634775000000, window=Some(432000000), functionId=Some(SumOverTime), rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634343000000,1634775000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-E~PromQlRemoteExec(PromQlQueryParams(sum_over_time(foo{_ws_="demo",_ns_=~".*Ns",instance="Inst-1"}[432000s]),1634775000,0,1634775000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin + |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634343000000,1634775000000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |-E~PromQlRemoteExec(PromQlQueryParams(sum_over_time(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs"}[432000s]),1634775000,0,1634775000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin validatePlan(execPlan, expected) } @@ -1304,10 +1294,10 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS """E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher |-E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) |--T~PeriodicSamplesMapper(start=1634771400000, step=60000, end=1634775000000, window=Some(600000), functionId=Some(Rate), rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634770800000,1634775000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634770800000,1634775000000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) |--T~PeriodicSamplesMapper(start=1634771400000, step=60000, end=1634775000000, window=Some(600000), functionId=Some(Rate), rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634770800000,1634775000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-E~PromQlRemoteExec(PromQlQueryParams(rate(foo{_ws_="demo",_ns_=~".*Ns",instance="Inst-1"}[600s]),1634775000,0,1634775000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin + |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634770800000,1634775000000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |-E~PromQlRemoteExec(PromQlQueryParams(rate(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs"}[600s]),1634771400,60,1634775000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin validatePlan(execPlan, expected) } @@ -1325,11 +1315,11 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) |----T~PeriodicSamplesMapper(start=1634771400000, step=60000, end=1634775000000, window=Some(600000), functionId=Some(Rate), rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634770800000,1634775000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634770800000,1634775000000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) |----T~PeriodicSamplesMapper(start=1634771400000, step=60000, end=1634775000000, window=Some(600000), functionId=Some(Rate), rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634770800000,1634775000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--E~PromQlRemoteExec(PromQlQueryParams(sum(rate(foo{_ws_="demo",_ns_=~".*Ns",instance="Inst-1"}[600s])),1634775000,0,1634775000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634770800000,1634775000000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |--E~PromQlRemoteExec(PromQlQueryParams(sum(rate(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs"}[600s])),1634771400,60,1634775000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin validatePlan(execPlan, expected) } @@ -1348,11 +1338,11 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) |-----T~PeriodicSamplesMapper(start=1634771400000, step=60000, end=1634775000000, window=Some(600000), functionId=Some(Rate), rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634770800000,1634775000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634770800000,1634775000000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) |-----T~PeriodicSamplesMapper(start=1634771400000, step=60000, end=1634775000000, window=Some(600000), functionId=Some(Rate), rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634770800000,1634775000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |---E~PromQlRemoteExec(PromQlQueryParams(sum(rate(foo{_ws_="demo",_ns_=~".*Ns",instance="Inst-1"}[600s])),1634775000,0,1634775000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634770800000,1634775000000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |---E~PromQlRemoteExec(PromQlQueryParams(sum(rate(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs"}[600s])),1634771400,60,1634775000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin validatePlan(execPlan, expected) } @@ -1365,7 +1355,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS plannerParams = PlannerParams(processMultiPartition = true))) val expected = - """E~PromQlRemoteExec(PromQlQueryParams(max_over_time(sum(rate(foo{_ws_="demo",_ns_=~".*remoteNs",instance="Inst-1"}[600s]))[3600s:60s]),1634775000,0,1634775000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher""" + """E~PromQlRemoteExec(PromQlQueryParams(max_over_time(sum(rate(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs"}[600s]))[3600s:60s]),1634775000,0,1634775000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher""" validatePlan(execPlan, expected) } @@ -1384,10 +1374,10 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher |--E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) |---T~PeriodicSamplesMapper(start=1634771400000, step=60000, end=1634775000000, window=Some(600000), functionId=Some(Rate), rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634770800000,1634775000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634770800000,1634775000000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) |---T~PeriodicSamplesMapper(start=1634771400000, step=60000, end=1634775000000, window=Some(600000), functionId=Some(Rate), rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634770800000,1634775000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--E~PromQlRemoteExec(PromQlQueryParams(rate(foo{_ws_="demo",_ns_=~".*Ns",instance="Inst-1"}[600s]),1634775000,0,1634775000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin + |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634770800000,1634775000000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |--E~PromQlRemoteExec(PromQlQueryParams(rate(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs"}[600s]),1634771400,60,1634775000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin validatePlan(execPlan, expected) } @@ -1407,10 +1397,10 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |--E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) |----T~PeriodicSamplesMapper(start=1634771100000, step=60000, end=1634775000000, window=Some(60000), functionId=Some(Rate), rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634771040000,1634775000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634771040000,1634775000000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) |----T~PeriodicSamplesMapper(start=1634771100000, step=60000, end=1634775000000, window=Some(60000), functionId=Some(Rate), rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634771040000,1634775000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |---E~PromQlRemoteExec(PromQlQueryParams(rate(foo{_ws_="demo",_ns_=~".*Ns",instance="Inst-1"}[60s]),1634775000,0,1634775000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634771040000,1634775000000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |---E~PromQlRemoteExec(PromQlQueryParams(rate(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs"}[60s]),1634771100,60,1634775000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin validatePlan(execPlan, expected) } @@ -1443,41 +1433,41 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS val expected = """E~BinaryJoinExec(binaryOp=MUL, on=None, ignoring=List()) on InProcessPlanDispatcher(filodb.core.query.QueryConfig@5eabff6b) |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634777330)) - |--E~MultiPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on InProcessPlanDispatcher - |---E~StitchRvsExec() on InProcessPlanDispatcher - |----E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |--E~MultiPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on InProcessPlanDispatcher(filodb.core.query.QueryConfig@5eabff6b) + |---E~StitchRvsExec() on InProcessPlanDispatcher(filodb.core.query.EmptyQueryConfig$@5b000fe6) + |----E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#542289610],raw) |-----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) |------T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#542289610],raw) |-----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) |------T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |----E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) + |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#542289610],raw) + |----E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#542289610],downsample) |-----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) |------T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) + |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#542289610],downsample) |-----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) |------T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |---E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~".*Ns",instance="Inst-1"}),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher + |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#542289610],downsample) + |---E~PromQlRemoteExec(PromQlQueryParams(sum(foo{instance="Inst-1",_ws_="demo",_ns_="remoteNs"}),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(filodb.core.query.QueryConfig@5eabff6b) |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634777330)) - |--E~MultiPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on InProcessPlanDispatcher - |---E~StitchRvsExec() on InProcessPlanDispatcher - |----E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |--E~MultiPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on InProcessPlanDispatcher(filodb.core.query.QueryConfig@5eabff6b) + |---E~StitchRvsExec() on InProcessPlanDispatcher(filodb.core.query.EmptyQueryConfig$@5b000fe6) + |----E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#542289610],raw) |-----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) |------T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#542289610],raw) |-----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) |------T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |----E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) + |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#542289610],raw) + |----E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#542289610],downsample) |-----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) |------T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) + |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#542289610],downsample) |-----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) |------T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |---E~PromQlRemoteExec(PromQlQueryParams(sum(bar{_ws_="demo",_ns_=~".*Ns",instance="Inst-1"}),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin + |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#542289610],downsample) + |---E~PromQlRemoteExec(PromQlQueryParams(sum(bar{instance="Inst-1",_ws_="demo",_ns_="remoteNs"}),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(filodb.core.query.QueryConfig@5eabff6b)""".stripMargin validatePlan(execPlan, expected) } @@ -1728,7 +1718,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS lp, QueryContext(origQueryParams = queryParams.copy(promQl = LogicalPlanParser.convertToQuery(lp)), plannerParams = PlannerParams(processMultiPartition = true))) - val expectedPlan = "E~PromQlRemoteExec(PromQlQueryParams(bottomk(1.0,((min((foo{_ws_=\"demo\",_ns_=\"remoteNs\"} * on(userID) group_left(userName) bar{_ws_=\"demo\",_ns_=\"remoteNs\"})) by (userName,time) - time()) > 0.0)),1634777330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(filodb.core.query.QueryConfig@570ba13)" + val expectedPlan = "E~PromQlRemoteExec(PromQlQueryParams(bottomk(1.0,((min((foo{_ws_=\"demo\",_ns_=\"remoteNs\"} * on(userID) group_left(userName) bar{_ws_=\"demo\",_ns_=\"remoteNs\"})) by (userName,time) - time()) > 0.0)),1634777330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(filodb.core.query.QueryConfig@570ba13)" validatePlan(execPlan, expectedPlan) } @@ -2259,230 +2249,158 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS } val tests = Seq( - Test(query = """sum(test{job="app"} offset 10m)""", - lookbackSec = 300, + // aggregate + Test("""sum(test{job="app"} offset 10m)""", offsetSec = 600, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(sum(test{job="app"} offset 600s),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(sum(test{job="app"} offset 600s),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(sum(test{job="app"} offset 600s),5901,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(sum(test{job="app"} offset 600s),5901,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """count(rate(test{job="app"}[20m] offset 10m))""", + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(sum(test{job="app"} offset 10m),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(sum(test{job="app"} offset 10m),5901,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + Test("""count(rate(test{job="app"}[20m] offset 10m))""", lookbackSec = 1200, offsetSec = 600, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(count(rate(test{job="app"}[1200s] offset 600s)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(count(rate(test{job="app"}[1200s] offset 600s)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(count(rate(test{job="app"}[1200s] offset 600s)),6801,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(count(rate(test{job="app"}[1200s] offset 600s)),6801,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """group(rate(test{job="app"}[20m:30s] offset 10m))""", - lookbackSec = 1500, + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(count(rate(test{job="app"}[20m] offset 10m)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(count(rate(test{job="app"}[20m] offset 10m)),6801,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + Test("""group(rate(test{job="app"}[20m:30s] offset 10m))""", + lookbackSec = 1200 + staleLookbackSec, offsetSec = 600, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(group(rate(test{job="app"}[1200s:30s] offset 600s)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(group(rate(test{job="app"}[1200s:30s] offset 600s)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(group(rate(test{job="app"}[1200s:30s] offset 600s)),7101,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(group(rate(test{job="app"}[1200s:30s] offset 600s)),7101,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """sum(rate(test{job="app"}[5m]) + rate(test{job="app"}[20m]))""", + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(group(rate(test{job="app"}[20m:30s] offset 10m)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(group(rate(test{job="app"}[20m:30s] offset 10m)),7101,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + Test("""sum(rate(test{job="app"}[5m]) + rate(test{job="app"}[20m]))""", lookbackSec = 1200, - offsetSec = 0, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(sum((rate(test{job="app"}[300s]) + rate(test{job="app"}[1200s]))),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(sum((rate(test{job="app"}[300s]) + rate(test{job="app"}[1200s]))),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(sum((rate(test{job="app"}[300s]) + rate(test{job="app"}[1200s]))),6201,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(sum((rate(test{job="app"}[300s]) + rate(test{job="app"}[1200s]))),6201,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """sgn(test{job="app"} offset 10m)""", - lookbackSec = 300, + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(sum(rate(test{job="app"}[5m]) + rate(test{job="app"}[20m])),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(sum(rate(test{job="app"}[5m]) + rate(test{job="app"}[20m])),6201,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + // instant + Test("""sgn(test{job="app"} offset 10m)""", offsetSec = 600, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(sgn(test{job="app"} offset 600s),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(sgn(test{job="app"} offset 600s),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(sgn(test{job="app"} offset 600s),5901,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(sgn(test{job="app"} offset 600s),5901,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """ln(rate(test{job="app"}[20m] offset 10m))""", + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(sgn(test{job="app"} offset 10m),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(sgn(test{job="app"} offset 10m),5901,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + Test("""ln(rate(test{job="app"}[20m] offset 10m))""", lookbackSec = 1200, offsetSec = 600, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(ln(rate(test{job="app"}[1200s] offset 600s)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(ln(rate(test{job="app"}[1200s] offset 600s)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(ln(rate(test{job="app"}[1200s] offset 600s)),6801,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(ln(rate(test{job="app"}[1200s] offset 600s)),6801,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """exp(rate(test{job="app"}[20m:30s] offset 10m))""", - lookbackSec = 1500, + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,00000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(ln(rate(test{job="app"}[20m] offset 10m)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(ln(rate(test{job="app"}[20m] offset 10m)),6801,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + Test("""exp(rate(test{job="app"}[20m:30s] offset 10m))""", + lookbackSec = 1200 + staleLookbackSec, offsetSec = 600, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(exp(rate(test{job="app"}[1200s:30s] offset 600s)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(exp(rate(test{job="app"}[1200s:30s] offset 600s)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(exp(rate(test{job="app"}[1200s:30s] offset 600s)),7101,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(exp(rate(test{job="app"}[1200s:30s] offset 600s)),7101,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """floor(rate(test{job="app"}[5m]) + rate(test{job="app"}[20m]))""", + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(exp(rate(test{job="app"}[20m:30s] offset 10m)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(exp(rate(test{job="app"}[20m:30s] offset 10m)),7101,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + Test("""floor(rate(test{job="app"}[5m]) + rate(test{job="app"}[20m]))""", lookbackSec = 1200, - offsetSec = 0, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(floor((rate(test{job="app"}[300s]) + rate(test{job="app"}[1200s]))),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(floor((rate(test{job="app"}[300s]) + rate(test{job="app"}[1200s]))),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(floor((rate(test{job="app"}[300s]) + rate(test{job="app"}[1200s]))),6201,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(floor((rate(test{job="app"}[300s]) + rate(test{job="app"}[1200s]))),6201,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """test{job="app"} + test{job="app"}""", - lookbackSec = 300, - offsetSec = 0, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((test{job="app"} + test{job="app"}),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((test{job="app"} + test{job="app"}),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((test{job="app"} + test{job="app"}),5301,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((test{job="app"} + test{job="app"}),5301,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """test{job="app"} + (test{job="app"} + test{job="app"})""", - lookbackSec = 300, - offsetSec = 0, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((test{job="app"} + (test{job="app"} + test{job="app"})),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((test{job="app"} + (test{job="app"} + test{job="app"})),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((test{job="app"} + (test{job="app"} + test{job="app"})),5301,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((test{job="app"} + (test{job="app"} + test{job="app"})),5301,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """count(test{job="app"}) + sum(test{job="app"})""", - lookbackSec = 300, - offsetSec = 0, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((count(test{job="app"}) + sum(test{job="app"})),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((count(test{job="app"}) + sum(test{job="app"})),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((count(test{job="app"}) + sum(test{job="app"})),5301,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((count(test{job="app"}) + sum(test{job="app"})),5301,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """count_over_time(foo{job="app"}[15m]) unless rate(bar{job="app"}[5m])""", + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(floor(rate(test{job="app"}[5m]) + rate(test{job="app"}[20m])),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(floor(rate(test{job="app"}[5m]) + rate(test{job="app"}[20m])),6201,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + // binary join + Test("""test{job="app"} + test{job="app"}""", + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(test{job="app"} + test{job="app"},0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(test{job="app"} + test{job="app"},5301,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + Test("""test{job="app"} + (test{job="app"} + test{job="app"})""", + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(test{job="app"} + (test{job="app"} + test{job="app"}),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(test{job="app"} + (test{job="app"} + test{job="app"}),5301,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + Test("""count(test{job="app"}) + sum(test{job="app"})""", + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(count(test{job="app"}) + sum(test{job="app"}),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(count(test{job="app"}) + sum(test{job="app"}),5301,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + Test("""count_over_time(foo{job="app"}[15m]) unless rate(bar{job="app"}[5m])""", lookbackSec = 900, - offsetSec = 0, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((count_over_time(foo{job="app"}[900s]) unless rate(bar{job="app"}[300s])),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((count_over_time(foo{job="app"}[900s]) unless rate(bar{job="app"}[300s])),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((count_over_time(foo{job="app"}[900s]) unless rate(bar{job="app"}[300s])),5901,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((count_over_time(foo{job="app"}[900s]) unless rate(bar{job="app"}[300s])),5901,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """count_over_time(foo{job="app1"}[5m]) unless rate(bar{job="app1"}[15m:30s])""", + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(count_over_time(foo{job="app"}[15m]) unless rate(bar{job="app"}[5m]),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(count_over_time(foo{job="app"}[15m]) unless rate(bar{job="app"}[5m]),5901,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + Test("""count_over_time(foo{job="app1"}[5m]) unless rate(bar{job="app1"}[15m:30s])""", + lookbackSec = 900 + staleLookbackSec, + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(count_over_time(foo{job="app1"}[5m]) unless rate(bar{job="app1"}[15m:30s]),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(count_over_time(foo{job="app1"}[5m]) unless rate(bar{job="app1"}[15m:30s]),6201,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + Test("""rate(foo{job="app1"}[5m]) + (rate(bar{job="app1"}[20m]) + count_over_time(baz{job="app1"}[5m]))""", lookbackSec = 1200, - offsetSec = 0, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((count_over_time(foo{job="app1"}[300s]) unless rate(bar{job="app1"}[900s:30s])),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((count_over_time(foo{job="app1"}[300s]) unless rate(bar{job="app1"}[900s:30s])),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((count_over_time(foo{job="app1"}[300s]) unless rate(bar{job="app1"}[900s:30s])),6201,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((count_over_time(foo{job="app1"}[300s]) unless rate(bar{job="app1"}[900s:30s])),6201,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """rate(foo{job="app1"}[5m]) + (rate(bar{job="app1"}[20m]) + count_over_time(baz{job="app1"}[5m]))""", - lookbackSec = 1200, - offsetSec = 0, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((rate(foo{job="app1"}[300s]) + (rate(bar{job="app1"}[1200s]) + count_over_time(baz{job="app1"}[300s]))),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((rate(foo{job="app1"}[300s]) + (rate(bar{job="app1"}[1200s]) + count_over_time(baz{job="app1"}[300s]))),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((rate(foo{job="app1"}[300s]) + (rate(bar{job="app1"}[1200s]) + count_over_time(baz{job="app1"}[300s]))),6201,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((rate(foo{job="app1"}[300s]) + (rate(bar{job="app1"}[1200s]) + count_over_time(baz{job="app1"}[300s]))),6201,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """rate(foo{job="app1"}[5m:30s]) + (rate(bar{job="app1"}[20m:30s]) + count_over_time(baz{job="app1"}[5m:30s]))""", - lookbackSec = 1500, - offsetSec = 0, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((rate(foo{job="app1"}[300s:30s]) + (rate(bar{job="app1"}[1200s:30s]) + count_over_time(baz{job="app1"}[300s:30s]))),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((rate(foo{job="app1"}[300s:30s]) + (rate(bar{job="app1"}[1200s:30s]) + count_over_time(baz{job="app1"}[300s:30s]))),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((rate(foo{job="app1"}[300s:30s]) + (rate(bar{job="app1"}[1200s:30s]) + count_over_time(baz{job="app1"}[300s:30s]))),6501,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((rate(foo{job="app1"}[300s:30s]) + (rate(bar{job="app1"}[1200s:30s]) + count_over_time(baz{job="app1"}[300s:30s]))),6501,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """test{job="app"} + 123""", - lookbackSec = 300, - offsetSec = 0, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((test{job="app"} + 123.0),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((test{job="app"} + 123.0),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((test{job="app"} + 123.0),5301,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((test{job="app"} + 123.0),5301,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """123 + sgn(test{job="app"} offset 10m)""", - lookbackSec = 300, + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(rate(foo{job="app1"}[5m]) + (rate(bar{job="app1"}[20m]) + count_over_time(baz{job="app1"}[5m])),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(rate(foo{job="app1"}[5m]) + (rate(bar{job="app1"}[20m]) + count_over_time(baz{job="app1"}[5m])),6201,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + Test("""rate(foo{job="app1"}[5m:30s]) + (rate(bar{job="app1"}[20m:30s]) + count_over_time(baz{job="app1"}[5m:30s]))""", + lookbackSec = 1200 + staleLookbackSec, + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(rate(foo{job="app1"}[5m:30s]) + (rate(bar{job="app1"}[20m:30s]) + count_over_time(baz{job="app1"}[5m:30s])),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(rate(foo{job="app1"}[5m:30s]) + (rate(bar{job="app1"}[20m:30s]) + count_over_time(baz{job="app1"}[5m:30s])),6501,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + // scalar vector join + Test("""test{job="app"} + 123""", + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(test{job="app"} + 123,0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(test{job="app"} + 123,5301,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + Test("""123 + sgn(test{job="app"} offset 10m)""", offsetSec = 600, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((123.0 + sgn(test{job="app"} offset 600s)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((123.0 + sgn(test{job="app"} offset 600s)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((123.0 + sgn(test{job="app"} offset 600s)),5901,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((123.0 + sgn(test{job="app"} offset 600s)),5901,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """123 + sum(rate(test{job="app"}[20m] offset 10m))""", + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(123 + sgn(test{job="app"} offset 10m),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(123 + sgn(test{job="app"} offset 10m),5901,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + Test("""123 + sum(rate(test{job="app"}[20m] offset 10m))""", lookbackSec = 1200, offsetSec = 600, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((123.0 + sum(rate(test{job="app"}[1200s] offset 600s))),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((123.0 + sum(rate(test{job="app"}[1200s] offset 600s))),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((123.0 + sum(rate(test{job="app"}[1200s] offset 600s))),6801,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((123.0 + sum(rate(test{job="app"}[1200s] offset 600s))),6801,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """123 + group(rate(test{job="app"}[20m:30s] offset 10m))""", - lookbackSec = 1500, + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(123 + sum(rate(test{job="app"}[20m] offset 10m)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(123 + sum(rate(test{job="app"}[20m] offset 10m)),6801,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + Test("""123 + group(rate(test{job="app"}[20m:30s] offset 10m))""", + lookbackSec = 1200 + staleLookbackSec, offsetSec = 600, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((123.0 + group(rate(test{job="app"}[1200s:30s] offset 600s))),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((123.0 + group(rate(test{job="app"}[1200s:30s] offset 600s))),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((123.0 + group(rate(test{job="app"}[1200s:30s] offset 600s))),7101,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((123.0 + group(rate(test{job="app"}[1200s:30s] offset 600s))),7101,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """123 + sum(count_over_time(test{job="app"}[5m]) + rate(test{job="app"}[20m]))""", + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(123 + group(rate(test{job="app"}[20m:30s] offset 10m)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(123 + group(rate(test{job="app"}[20m:30s] offset 10m)),7101,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + Test("""123 + sum(count_over_time(test{job="app"}[5m]) + rate(test{job="app"}[20m]))""", lookbackSec = 1200, - offsetSec = 0, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((123.0 + sum((count_over_time(test{job="app"}[300s]) + rate(test{job="app"}[1200s])))),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((123.0 + sum((count_over_time(test{job="app"}[300s]) + rate(test{job="app"}[1200s])))),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((123.0 + sum((count_over_time(test{job="app"}[300s]) + rate(test{job="app"}[1200s])))),6201,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((123.0 + sum((count_over_time(test{job="app"}[300s]) + rate(test{job="app"}[1200s])))),6201,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """absent(test{job="app"} offset 10m)""", - lookbackSec = 300, + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(123 + sum(count_over_time(test{job="app"}[5m]) + rate(test{job="app"}[20m])),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(123 + sum(count_over_time(test{job="app"}[5m]) + rate(test{job="app"}[20m])),6201,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + // absent + Test("""absent(test{job="app"} offset 10m)""", offsetSec = 600, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(absent(test{job="app"} offset 600s),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(absent(test{job="app"} offset 600s),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(absent(test{job="app"} offset 600s),5901,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(absent(test{job="app"} offset 600s),5901,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """absent(rate(test{job="app"}[20m] offset 10m))""", + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(absent(test{job="app"} offset 10m),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(absent(test{job="app"} offset 10m),5901,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + Test("""absent(rate(test{job="app"}[20m] offset 10m))""", lookbackSec = 1200, offsetSec = 600, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(absent(rate(test{job="app"}[1200s] offset 600s)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(absent(rate(test{job="app"}[1200s] offset 600s)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(absent(rate(test{job="app"}[1200s] offset 600s)),6801,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(absent(rate(test{job="app"}[1200s] offset 600s)),6801,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """absent(count_over_time(test{job="app"}[20m:30s] offset 10m))""", - lookbackSec = 1500, + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(absent(rate(test{job="app"}[20m] offset 10m)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(absent(rate(test{job="app"}[20m] offset 10m)),6801,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + Test("""absent(count_over_time(test{job="app"}[20m:30s] offset 10m))""", + lookbackSec = 1200 + staleLookbackSec, offsetSec = 600, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(absent(count_over_time(test{job="app"}[1200s:30s] offset 600s)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(absent(count_over_time(test{job="app"}[1200s:30s] offset 600s)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(absent(count_over_time(test{job="app"}[1200s:30s] offset 600s)),7101,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(absent(count_over_time(test{job="app"}[1200s:30s] offset 600s)),7101,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """absent(sum_over_time(test{job="app"}[5m]) + rate(test{job="app"}[20m]))""", + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(absent(count_over_time(test{job="app"}[20m:30s] offset 10m)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(absent(count_over_time(test{job="app"}[20m:30s] offset 10m)),7101,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + Test("""absent(sum_over_time(test{job="app"}[5m]) + rate(test{job="app"}[20m]))""", lookbackSec = 1200, - offsetSec = 0, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(absent((sum_over_time(test{job="app"}[300s]) + rate(test{job="app"}[1200s]))),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(absent((sum_over_time(test{job="app"}[300s]) + rate(test{job="app"}[1200s]))),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(absent((sum_over_time(test{job="app"}[300s]) + rate(test{job="app"}[1200s]))),6201,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(absent((sum_over_time(test{job="app"}[300s]) + rate(test{job="app"}[1200s]))),6201,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """scalar(test{job="app"} offset 10m)""", - lookbackSec = 300, + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(absent(sum_over_time(test{job="app"}[5m]) + rate(test{job="app"}[20m])),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(absent(sum_over_time(test{job="app"}[5m]) + rate(test{job="app"}[20m])),6201,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + // scalar + Test("""scalar(test{job="app"} offset 10m)""", offsetSec = 600, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(scalar(test{job="app"} offset 600s),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(scalar(test{job="app"} offset 600s),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(scalar(test{job="app"} offset 600s),5901,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(scalar(test{job="app"} offset 600s),5901,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """scalar(rate(test{job="app"}[20m] offset 10m))""", + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(scalar(test{job="app"} offset 10m),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(scalar(test{job="app"} offset 10m),5901,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + Test("""scalar(rate(test{job="app"}[20m] offset 10m))""", lookbackSec = 1200, offsetSec = 600, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(scalar(rate(test{job="app"}[1200s] offset 600s)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(scalar(rate(test{job="app"}[1200s] offset 600s)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(scalar(rate(test{job="app"}[1200s] offset 600s)),6801,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(scalar(rate(test{job="app"}[1200s] offset 600s)),6801,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """scalar(count_over_time(test{job="app"}[20m:30s] offset 10m))""", - lookbackSec = 1500, + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(scalar(rate(test{job="app"}[20m] offset 10m)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(scalar(rate(test{job="app"}[20m] offset 10m)),6801,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + Test("""scalar(count_over_time(test{job="app"}[20m:30s] offset 10m))""", + lookbackSec = 1200 + staleLookbackSec, offsetSec = 600, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(scalar(count_over_time(test{job="app"}[1200s:30s] offset 600s)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(scalar(count_over_time(test{job="app"}[1200s:30s] offset 600s)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(scalar(count_over_time(test{job="app"}[1200s:30s] offset 600s)),7101,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(scalar(count_over_time(test{job="app"}[1200s:30s] offset 600s)),7101,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """scalar(sum_over_time(test{job="app"}[5m]) + rate(test{job="app"}[20m]))""", + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(scalar(count_over_time(test{job="app"}[20m:30s] offset 10m)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(scalar(count_over_time(test{job="app"}[20m:30s] offset 10m)),7101,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + Test("""scalar(sum_over_time(test{job="app"}[5m]) + rate(test{job="app"}[20m]))""", lookbackSec = 1200, - offsetSec = 0, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(scalar((sum_over_time(test{job="app"}[300s]) + rate(test{job="app"}[1200s]))),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(scalar((sum_over_time(test{job="app"}[300s]) + rate(test{job="app"}[1200s]))),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(scalar((sum_over_time(test{job="app"}[300s]) + rate(test{job="app"}[1200s]))),6201,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(scalar((sum_over_time(test{job="app"}[300s]) + rate(test{job="app"}[1200s]))),6201,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin) + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(scalar(sum_over_time(test{job="app"}[5m]) + rate(test{job="app"}[20m])),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(scalar(sum_over_time(test{job="app"}[5m]) + rate(test{job="app"}[20m])),6201,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), ) val partitionLocationProvider = new PartitionLocationProvider { override def getPartitions(routingKey: Map[String, String], @@ -2501,7 +2419,6 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS partitionLocationProvider, singlePartitionPlanner, "local", MetricsTestData.timeseriesDataset, queryConfig ) - val tests2 = new mutable.ArrayBuffer[(String, Long, Long, ExecPlan)] for (test <- tests) { val lp = Parser.queryRangeToLogicalPlan(test.query, TimeStepParams(startSec, stepSec, endSec)) val promQlQueryParams = PromQlQueryParams(test.query, startSec, stepSec, endSec) @@ -2514,9 +2431,25 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS // -E~PromQlRemoteExec(PromQlQueryParams(sgn(test{job="app"}) + 123,123,45,3306,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) // -E~PromQlRemoteExec(PromQlQueryParams(sgn(test{job="app"}) + 123,3633,45,6789,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) val root = execPlan.asInstanceOf[StitchRvsExec] + // Make sure one PromQlRemoteExec for each partition. + root.children.size shouldEqual 2 + // Extract the endpoint/TimeStepParams and make sure they are as-expected. + val expectedQueryParams = { + val timeStepParams = test.getExpectedRangesSec().map { case (startSecExp, endSecExp) => + TimeStepParams(startSecExp, stepSec, endSecExp) + } + expectedUrls.zip(timeStepParams) + }.toSet + root.children.map{ child => + val remote = child.asInstanceOf[PromQlRemoteExec] + val params = remote.promQlQueryParams + // Each plan should dispatch the same query. + params.promQl shouldEqual test.query + (remote.queryEndpoint, TimeStepParams(params.startSecs, params.stepSecs, params.endSecs)) + }.toSet shouldEqual expectedQueryParams + // sanity check validatePlan(root, test.expected) } - printTests3(tests2) } @@ -2539,230 +2472,162 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS } val tests = Seq( - Test(query = """sum(test{job="app"} offset 10m)""", - lookbackSec = 300, + // aggregate + Test("""sum(test{job="app"} offset 10m)""", offsetSec = 600, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(sum(test{job="app"} offset 600s),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams(sum(test{job="app"} offset 600s),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(sum(test{job="app"} offset 600s),5901,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams(sum(test{job="app"} offset 600s),5901,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """count(rate(test{job="app"}[20m] offset 10m))""", + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQlRemoteExec(PromQlQueryParams(sum(test{job="app"} offset 10m),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQLGrpcRemoteExec(PromQlQueryParams(sum(test{job="app"} offset 10m),5901,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set()))""".stripMargin), + Test("""count(rate(test{job="app"}[20m] offset 10m))""", lookbackSec = 1200, offsetSec = 600, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(count(rate(test{job="app"}[1200s] offset 600s)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams(count(rate(test{job="app"}[1200s] offset 600s)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(count(rate(test{job="app"}[1200s] offset 600s)),6801,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams(count(rate(test{job="app"}[1200s] offset 600s)),6801,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """group(rate(test{job="app"}[20m:30s] offset 10m))""", - lookbackSec = 1500, + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQlRemoteExec(PromQlQueryParams(count(rate(test{job="app"}[20m] offset 10m)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQLGrpcRemoteExec(PromQlQueryParams(count(rate(test{job="app"}[20m] offset 10m)),6801,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set()))""".stripMargin), + Test("""group(rate(test{job="app"}[20m:30s] offset 10m))""", + lookbackSec = 1200 + staleLookbackSec, offsetSec = 600, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(group(rate(test{job="app"}[1200s:30s] offset 600s)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams(group(rate(test{job="app"}[1200s:30s] offset 600s)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(group(rate(test{job="app"}[1200s:30s] offset 600s)),7101,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams(group(rate(test{job="app"}[1200s:30s] offset 600s)),7101,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """sum(rate(test{job="app"}[5m]) + rate(test{job="app"}[20m]))""", + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQlRemoteExec(PromQlQueryParams(group(rate(test{job="app"}[20m:30s] offset 10m)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQLGrpcRemoteExec(PromQlQueryParams(group(rate(test{job="app"}[20m:30s] offset 10m)),7101,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set()))""".stripMargin), + Test("""sum(rate(test{job="app"}[5m]) + rate(test{job="app"}[20m]))""", lookbackSec = 1200, - offsetSec = 0, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(sum((rate(test{job="app"}[300s]) + rate(test{job="app"}[1200s]))),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams(sum((rate(test{job="app"}[300s]) + rate(test{job="app"}[1200s]))),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(sum((rate(test{job="app"}[300s]) + rate(test{job="app"}[1200s]))),6201,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams(sum((rate(test{job="app"}[300s]) + rate(test{job="app"}[1200s]))),6201,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """sgn(test{job="app"} offset 10m)""", - lookbackSec = 300, + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQlRemoteExec(PromQlQueryParams(sum(rate(test{job="app"}[5m]) + rate(test{job="app"}[20m])),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQLGrpcRemoteExec(PromQlQueryParams(sum(rate(test{job="app"}[5m]) + rate(test{job="app"}[20m])),6201,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set()))""".stripMargin), + // instant + Test("""sgn(test{job="app"} offset 10m)""", offsetSec = 600, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(sgn(test{job="app"} offset 600s),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams(sgn(test{job="app"} offset 600s),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(sgn(test{job="app"} offset 600s),5901,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams(sgn(test{job="app"} offset 600s),5901,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """ln(rate(test{job="app"}[20m] offset 10m))""", + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQlRemoteExec(PromQlQueryParams(sgn(test{job="app"} offset 10m),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQLGrpcRemoteExec(PromQlQueryParams(sgn(test{job="app"} offset 10m),5901,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set()))""".stripMargin), + Test("""ln(rate(test{job="app"}[20m] offset 10m))""", lookbackSec = 1200, offsetSec = 600, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(ln(rate(test{job="app"}[1200s] offset 600s)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams(ln(rate(test{job="app"}[1200s] offset 600s)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(ln(rate(test{job="app"}[1200s] offset 600s)),6801,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams(ln(rate(test{job="app"}[1200s] offset 600s)),6801,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """exp(rate(test{job="app"}[20m:30s] offset 10m))""", - lookbackSec = 1500, + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQlRemoteExec(PromQlQueryParams(ln(rate(test{job="app"}[20m] offset 10m)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQLGrpcRemoteExec(PromQlQueryParams(ln(rate(test{job="app"}[20m] offset 10m)),6801,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set()))""".stripMargin), + Test("""exp(rate(test{job="app"}[20m:30s] offset 10m))""", + lookbackSec = 1200 + staleLookbackSec, offsetSec = 600, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(exp(rate(test{job="app"}[1200s:30s] offset 600s)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams(exp(rate(test{job="app"}[1200s:30s] offset 600s)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(exp(rate(test{job="app"}[1200s:30s] offset 600s)),7101,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams(exp(rate(test{job="app"}[1200s:30s] offset 600s)),7101,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """floor(rate(test{job="app"}[5m]) + rate(test{job="app"}[20m]))""", + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQlRemoteExec(PromQlQueryParams(exp(rate(test{job="app"}[20m:30s] offset 10m)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQLGrpcRemoteExec(PromQlQueryParams(exp(rate(test{job="app"}[20m:30s] offset 10m)),7101,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set()))""".stripMargin), + Test("""floor(rate(test{job="app"}[5m]) + rate(test{job="app"}[20m]))""", lookbackSec = 1200, - offsetSec = 0, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(floor((rate(test{job="app"}[300s]) + rate(test{job="app"}[1200s]))),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams(floor((rate(test{job="app"}[300s]) + rate(test{job="app"}[1200s]))),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(floor((rate(test{job="app"}[300s]) + rate(test{job="app"}[1200s]))),6201,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams(floor((rate(test{job="app"}[300s]) + rate(test{job="app"}[1200s]))),6201,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """test{job="app"} + test{job="app"}""", - lookbackSec = 300, - offsetSec = 0, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((test{job="app"} + test{job="app"}),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams((test{job="app"} + test{job="app"}),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((test{job="app"} + test{job="app"}),5301,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams((test{job="app"} + test{job="app"}),5301,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """test{job="app"} + (test{job="app"} + test{job="app"})""", - lookbackSec = 300, - offsetSec = 0, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((test{job="app"} + (test{job="app"} + test{job="app"})),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams((test{job="app"} + (test{job="app"} + test{job="app"})),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((test{job="app"} + (test{job="app"} + test{job="app"})),5301,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams((test{job="app"} + (test{job="app"} + test{job="app"})),5301,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """count(test{job="app"}) + sum(test{job="app"})""", - lookbackSec = 300, - offsetSec = 0, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((count(test{job="app"}) + sum(test{job="app"})),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams((count(test{job="app"}) + sum(test{job="app"})),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((count(test{job="app"}) + sum(test{job="app"})),5301,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams((count(test{job="app"}) + sum(test{job="app"})),5301,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """count_over_time(foo{job="app"}[15m]) unless rate(bar{job="app"}[5m])""", + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQlRemoteExec(PromQlQueryParams(floor(rate(test{job="app"}[5m]) + rate(test{job="app"}[20m])),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQLGrpcRemoteExec(PromQlQueryParams(floor(rate(test{job="app"}[5m]) + rate(test{job="app"}[20m])),6201,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set()))""".stripMargin), + // binary join + Test("""test{job="app"} + test{job="app"}""", + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQlRemoteExec(PromQlQueryParams(test{job="app"} + test{job="app"},0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQLGrpcRemoteExec(PromQlQueryParams(test{job="app"} + test{job="app"},5301,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set()))""".stripMargin), + Test("""test{job="app"} + (test{job="app"} + test{job="app"})""", + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQlRemoteExec(PromQlQueryParams(test{job="app"} + (test{job="app"} + test{job="app"}),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQLGrpcRemoteExec(PromQlQueryParams(test{job="app"} + (test{job="app"} + test{job="app"}),5301,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set()))""".stripMargin), + Test("""count(test{job="app"}) + sum(test{job="app"})""", + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQlRemoteExec(PromQlQueryParams(count(test{job="app"}) + sum(test{job="app"}),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQLGrpcRemoteExec(PromQlQueryParams(count(test{job="app"}) + sum(test{job="app"}),5301,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set()))""".stripMargin), + Test("""count_over_time(foo{job="app"}[15m]) unless rate(bar{job="app"}[5m])""", lookbackSec = 900, - offsetSec = 0, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((count_over_time(foo{job="app"}[900s]) unless rate(bar{job="app"}[300s])),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams((count_over_time(foo{job="app"}[900s]) unless rate(bar{job="app"}[300s])),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((count_over_time(foo{job="app"}[900s]) unless rate(bar{job="app"}[300s])),5901,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams((count_over_time(foo{job="app"}[900s]) unless rate(bar{job="app"}[300s])),5901,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """count_over_time(foo{job="app1"}[5m]) unless rate(bar{job="app1"}[15m:30s])""", + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQlRemoteExec(PromQlQueryParams(count_over_time(foo{job="app"}[15m]) unless rate(bar{job="app"}[5m]),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQLGrpcRemoteExec(PromQlQueryParams(count_over_time(foo{job="app"}[15m]) unless rate(bar{job="app"}[5m]),5901,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set()))""".stripMargin), + Test("""count_over_time(foo{job="app1"}[5m]) unless rate(bar{job="app1"}[15m:30s])""", + lookbackSec = 900 + staleLookbackSec, + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQlRemoteExec(PromQlQueryParams(count_over_time(foo{job="app1"}[5m]) unless rate(bar{job="app1"}[15m:30s]),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQLGrpcRemoteExec(PromQlQueryParams(count_over_time(foo{job="app1"}[5m]) unless rate(bar{job="app1"}[15m:30s]),6201,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set()))""".stripMargin), + Test("""rate(foo{job="app1"}[5m]) + (rate(bar{job="app1"}[20m]) + count_over_time(baz{job="app1"}[5m]))""", lookbackSec = 1200, - offsetSec = 0, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((count_over_time(foo{job="app1"}[300s]) unless rate(bar{job="app1"}[900s:30s])),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams((count_over_time(foo{job="app1"}[300s]) unless rate(bar{job="app1"}[900s:30s])),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((count_over_time(foo{job="app1"}[300s]) unless rate(bar{job="app1"}[900s:30s])),6201,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams((count_over_time(foo{job="app1"}[300s]) unless rate(bar{job="app1"}[900s:30s])),6201,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """rate(foo{job="app1"}[5m]) + (rate(bar{job="app1"}[20m]) + count_over_time(baz{job="app1"}[5m]))""", - lookbackSec = 1200, - offsetSec = 0, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((rate(foo{job="app1"}[300s]) + (rate(bar{job="app1"}[1200s]) + count_over_time(baz{job="app1"}[300s]))),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams((rate(foo{job="app1"}[300s]) + (rate(bar{job="app1"}[1200s]) + count_over_time(baz{job="app1"}[300s]))),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((rate(foo{job="app1"}[300s]) + (rate(bar{job="app1"}[1200s]) + count_over_time(baz{job="app1"}[300s]))),6201,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams((rate(foo{job="app1"}[300s]) + (rate(bar{job="app1"}[1200s]) + count_over_time(baz{job="app1"}[300s]))),6201,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """rate(foo{job="app1"}[5m:30s]) + (rate(bar{job="app1"}[20m:30s]) + count_over_time(baz{job="app1"}[5m:30s]))""", - lookbackSec = 1500, - offsetSec = 0, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((rate(foo{job="app1"}[300s:30s]) + (rate(bar{job="app1"}[1200s:30s]) + count_over_time(baz{job="app1"}[300s:30s]))),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams((rate(foo{job="app1"}[300s:30s]) + (rate(bar{job="app1"}[1200s:30s]) + count_over_time(baz{job="app1"}[300s:30s]))),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((rate(foo{job="app1"}[300s:30s]) + (rate(bar{job="app1"}[1200s:30s]) + count_over_time(baz{job="app1"}[300s:30s]))),6501,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams((rate(foo{job="app1"}[300s:30s]) + (rate(bar{job="app1"}[1200s:30s]) + count_over_time(baz{job="app1"}[300s:30s]))),6501,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """test{job="app"} + 123""", - lookbackSec = 300, - offsetSec = 0, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((test{job="app"} + 123.0),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams((test{job="app"} + 123.0),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((test{job="app"} + 123.0),5301,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams((test{job="app"} + 123.0),5301,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """123 + sgn(test{job="app"} offset 10m)""", - lookbackSec = 300, + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQlRemoteExec(PromQlQueryParams(rate(foo{job="app1"}[5m]) + (rate(bar{job="app1"}[20m]) + count_over_time(baz{job="app1"}[5m])),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQLGrpcRemoteExec(PromQlQueryParams(rate(foo{job="app1"}[5m]) + (rate(bar{job="app1"}[20m]) + count_over_time(baz{job="app1"}[5m])),6201,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set()))""".stripMargin), + + Test("""rate(foo{job="app1"}[5m:30s]) + (rate(bar{job="app1"}[20m:30s]) + count_over_time(baz{job="app1"}[5m:30s]))""", + lookbackSec = 1200 + staleLookbackSec, + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQlRemoteExec(PromQlQueryParams(rate(foo{job="app1"}[5m:30s]) + (rate(bar{job="app1"}[20m:30s]) + count_over_time(baz{job="app1"}[5m:30s])),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQLGrpcRemoteExec(PromQlQueryParams(rate(foo{job="app1"}[5m:30s]) + (rate(bar{job="app1"}[20m:30s]) + count_over_time(baz{job="app1"}[5m:30s])),6501,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set()))""".stripMargin), + + // scalar vector join + Test("""test{job="app"} + 123""", + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQlRemoteExec(PromQlQueryParams(test{job="app"} + 123,0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQLGrpcRemoteExec(PromQlQueryParams(test{job="app"} + 123,5301,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set()))""".stripMargin), + Test("""123 + sgn(test{job="app"} offset 10m)""", offsetSec = 600, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((123.0 + sgn(test{job="app"} offset 600s)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams((123.0 + sgn(test{job="app"} offset 600s)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((123.0 + sgn(test{job="app"} offset 600s)),5901,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams((123.0 + sgn(test{job="app"} offset 600s)),5901,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """123 + sum(rate(test{job="app"}[20m] offset 10m))""", + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQlRemoteExec(PromQlQueryParams(123 + sgn(test{job="app"} offset 10m),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQLGrpcRemoteExec(PromQlQueryParams(123 + sgn(test{job="app"} offset 10m),5901,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set()))""".stripMargin), + Test("""123 + sum(rate(test{job="app"}[20m] offset 10m))""", lookbackSec = 1200, offsetSec = 600, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((123.0 + sum(rate(test{job="app"}[1200s] offset 600s))),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams((123.0 + sum(rate(test{job="app"}[1200s] offset 600s))),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((123.0 + sum(rate(test{job="app"}[1200s] offset 600s))),6801,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams((123.0 + sum(rate(test{job="app"}[1200s] offset 600s))),6801,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """123 + group(rate(test{job="app"}[20m:30s] offset 10m))""", - lookbackSec = 1500, + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQlRemoteExec(PromQlQueryParams(123 + sum(rate(test{job="app"}[20m] offset 10m)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQLGrpcRemoteExec(PromQlQueryParams(123 + sum(rate(test{job="app"}[20m] offset 10m)),6801,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set()))""".stripMargin), + Test("""123 + group(rate(test{job="app"}[20m:30s] offset 10m))""", + lookbackSec = 1200 + staleLookbackSec, offsetSec = 600, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((123.0 + group(rate(test{job="app"}[1200s:30s] offset 600s))),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams((123.0 + group(rate(test{job="app"}[1200s:30s] offset 600s))),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((123.0 + group(rate(test{job="app"}[1200s:30s] offset 600s))),7101,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams((123.0 + group(rate(test{job="app"}[1200s:30s] offset 600s))),7101,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """123 + sum(count_over_time(test{job="app"}[5m]) + rate(test{job="app"}[20m]))""", + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQlRemoteExec(PromQlQueryParams(123 + group(rate(test{job="app"}[20m:30s] offset 10m)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQLGrpcRemoteExec(PromQlQueryParams(123 + group(rate(test{job="app"}[20m:30s] offset 10m)),7101,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set()))""".stripMargin), + + Test("""123 + sum(count_over_time(test{job="app"}[5m]) + rate(test{job="app"}[20m]))""", lookbackSec = 1200, - offsetSec = 0, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((123.0 + sum((count_over_time(test{job="app"}[300s]) + rate(test{job="app"}[1200s])))),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams((123.0 + sum((count_over_time(test{job="app"}[300s]) + rate(test{job="app"}[1200s])))),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams((123.0 + sum((count_over_time(test{job="app"}[300s]) + rate(test{job="app"}[1200s])))),6201,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams((123.0 + sum((count_over_time(test{job="app"}[300s]) + rate(test{job="app"}[1200s])))),6201,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """absent(test{job="app"} offset 10m)""", - lookbackSec = 300, + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQlRemoteExec(PromQlQueryParams(123 + sum(count_over_time(test{job="app"}[5m]) + rate(test{job="app"}[20m])),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQLGrpcRemoteExec(PromQlQueryParams(123 + sum(count_over_time(test{job="app"}[5m]) + rate(test{job="app"}[20m])),6201,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set()))""".stripMargin), + // absent + Test("""absent(test{job="app"} offset 10m)""", offsetSec = 600, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(absent(test{job="app"} offset 600s),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams(absent(test{job="app"} offset 600s),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(absent(test{job="app"} offset 600s),5901,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams(absent(test{job="app"} offset 600s),5901,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """absent(rate(test{job="app"}[20m] offset 10m))""", + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQlRemoteExec(PromQlQueryParams(absent(test{job="app"} offset 10m),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQLGrpcRemoteExec(PromQlQueryParams(absent(test{job="app"} offset 10m),5901,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set()))""".stripMargin), + Test("""absent(rate(test{job="app"}[20m] offset 10m))""", lookbackSec = 1200, offsetSec = 600, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(absent(rate(test{job="app"}[1200s] offset 600s)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams(absent(rate(test{job="app"}[1200s] offset 600s)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(absent(rate(test{job="app"}[1200s] offset 600s)),6801,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams(absent(rate(test{job="app"}[1200s] offset 600s)),6801,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """absent(count_over_time(test{job="app"}[20m:30s] offset 10m))""", - lookbackSec = 1500, + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQlRemoteExec(PromQlQueryParams(absent(rate(test{job="app"}[20m] offset 10m)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQLGrpcRemoteExec(PromQlQueryParams(absent(rate(test{job="app"}[20m] offset 10m)),6801,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set()))""".stripMargin), + Test("""absent(count_over_time(test{job="app"}[20m:30s] offset 10m))""", + lookbackSec = 1200 + staleLookbackSec, offsetSec = 600, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(absent(count_over_time(test{job="app"}[1200s:30s] offset 600s)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams(absent(count_over_time(test{job="app"}[1200s:30s] offset 600s)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(absent(count_over_time(test{job="app"}[1200s:30s] offset 600s)),7101,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams(absent(count_over_time(test{job="app"}[1200s:30s] offset 600s)),7101,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """absent(sum_over_time(test{job="app"}[5m]) + rate(test{job="app"}[20m]))""", + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQlRemoteExec(PromQlQueryParams(absent(count_over_time(test{job="app"}[20m:30s] offset 10m)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQLGrpcRemoteExec(PromQlQueryParams(absent(count_over_time(test{job="app"}[20m:30s] offset 10m)),7101,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set()))""".stripMargin), + Test("""absent(sum_over_time(test{job="app"}[5m]) + rate(test{job="app"}[20m]))""", lookbackSec = 1200, - offsetSec = 0, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(absent((sum_over_time(test{job="app"}[300s]) + rate(test{job="app"}[1200s]))),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams(absent((sum_over_time(test{job="app"}[300s]) + rate(test{job="app"}[1200s]))),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(absent((sum_over_time(test{job="app"}[300s]) + rate(test{job="app"}[1200s]))),6201,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams(absent((sum_over_time(test{job="app"}[300s]) + rate(test{job="app"}[1200s]))),6201,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """scalar(test{job="app"} offset 10m)""", - lookbackSec = 300, + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQlRemoteExec(PromQlQueryParams(absent(sum_over_time(test{job="app"}[5m]) + rate(test{job="app"}[20m])),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQLGrpcRemoteExec(PromQlQueryParams(absent(sum_over_time(test{job="app"}[5m]) + rate(test{job="app"}[20m])),6201,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set()))""".stripMargin), + // scalar + Test("""scalar(test{job="app"} offset 10m)""", offsetSec = 600, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(scalar(test{job="app"} offset 600s),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams(scalar(test{job="app"} offset 600s),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(scalar(test{job="app"} offset 600s),5901,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams(scalar(test{job="app"} offset 600s),5901,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """scalar(rate(test{job="app"}[20m] offset 10m))""", + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQlRemoteExec(PromQlQueryParams(scalar(test{job="app"} offset 10m),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQLGrpcRemoteExec(PromQlQueryParams(scalar(test{job="app"} offset 10m),5901,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set()))""".stripMargin), + Test("""scalar(rate(test{job="app"}[20m] offset 10m))""", lookbackSec = 1200, offsetSec = 600, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(scalar(rate(test{job="app"}[1200s] offset 600s)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams(scalar(rate(test{job="app"}[1200s] offset 600s)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(scalar(rate(test{job="app"}[1200s] offset 600s)),6801,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams(scalar(rate(test{job="app"}[1200s] offset 600s)),6801,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """scalar(count_over_time(test{job="app"}[20m:30s] offset 10m))""", - lookbackSec = 1500, + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQlRemoteExec(PromQlQueryParams(scalar(rate(test{job="app"}[20m] offset 10m)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQLGrpcRemoteExec(PromQlQueryParams(scalar(rate(test{job="app"}[20m] offset 10m)),6801,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set()))""".stripMargin), + + Test("""scalar(count_over_time(test{job="app"}[20m:30s] offset 10m))""", + lookbackSec = 1200 + staleLookbackSec, offsetSec = 600, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(scalar(count_over_time(test{job="app"}[1200s:30s] offset 600s)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams(scalar(count_over_time(test{job="app"}[1200s:30s] offset 600s)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(scalar(count_over_time(test{job="app"}[1200s:30s] offset 600s)),7101,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams(scalar(count_over_time(test{job="app"}[1200s:30s] offset 600s)),7101,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - Test(query = """scalar(sum_over_time(test{job="app"}[5m]) + rate(test{job="app"}[20m]))""", + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQlRemoteExec(PromQlQueryParams(scalar(count_over_time(test{job="app"}[20m:30s] offset 10m)),0,3,5600,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQLGrpcRemoteExec(PromQlQueryParams(scalar(count_over_time(test{job="app"}[20m:30s] offset 10m)),7101,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set()))""".stripMargin), + Test("""scalar(sum_over_time(test{job="app"}[5m]) + rate(test{job="app"}[20m]))""", lookbackSec = 1200, - offsetSec = 0, - expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(scalar((sum_over_time(test{job="app"}[300s]) + rate(test{job="app"}[1200s]))),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams(scalar((sum_over_time(test{job="app"}[300s]) + rate(test{job="app"}[1200s]))),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(scalar((sum_over_time(test{job="app"}[300s]) + rate(test{job="app"}[1200s]))),6201,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams(scalar((sum_over_time(test{job="app"}[300s]) + rate(test{job="app"}[1200s]))),6201,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin) + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQlRemoteExec(PromQlQueryParams(scalar(sum_over_time(test{job="app"}[5m]) + rate(test{job="app"}[20m])),0,3,5000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |-E~PromQLGrpcRemoteExec(PromQlQueryParams(scalar(sum_over_time(test{job="app"}[5m]) + rate(test{job="app"}[20m])),6201,3,9999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=grpc-remote1-url.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set()))""".stripMargin), ) val partitionLocationProvider = new PartitionLocationProvider { override def getPartitions(routingKey: Map[String, String], @@ -2780,7 +2645,6 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS partitionLocationProvider, singlePartitionPlanner, "local", MetricsTestData.timeseriesDataset, queryConfig ) - val tests2 = new mutable.ArrayBuffer[(String, Long, Long, ExecPlan)] for (test <- tests) { val lp = Parser.queryRangeToLogicalPlan(test.query, TimeStepParams(startSec, stepSec, endSec)) val promQlQueryParams = PromQlQueryParams(test.query, startSec, stepSec, endSec) @@ -2790,31 +2654,29 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS ) val root = execPlan.asInstanceOf[StitchRvsExec] -// // Make sure one PromQlRemoteExec for each partition. -// root.children.size shouldEqual 2 -// // Extract the endpoint/TimeStepParams and make sure they are as-expected. -// val expectedQueryParams = { -// val timeStepParams = test.getExpectedRangesSec().map { case (startSecExp, endSecExp) => -// TimeStepParams(startSecExp, stepSec, endSecExp) -// } -// expectedUrls.zip(timeStepParams) -// }.toSet -// root.children.map{ child => -// val (params, url) = child match { -// case remote: PromQlRemoteExec => (remote.promQlQueryParams, remote.queryEndpoint) -// case grpc: PromQLGrpcRemoteExec => (grpc.promQlQueryParams, grpc.queryEndpoint) -// case _ => fail("unsupported execplan") -// } -// // Each plan should dispatch the same query. -// params.promQl shouldEqual test.query -// (url, TimeStepParams(params.startSecs, params.stepSecs, params.endSecs)) -// }.toSet shouldEqual expectedQueryParams + // Make sure one PromQlRemoteExec for each partition. + root.children.size shouldEqual 2 + // Extract the endpoint/TimeStepParams and make sure they are as-expected. + val expectedQueryParams = { + val timeStepParams = test.getExpectedRangesSec().map { case (startSecExp, endSecExp) => + TimeStepParams(startSecExp, stepSec, endSecExp) + } + expectedUrls.zip(timeStepParams) + }.toSet + root.children.map{ child => + val (params, url) = child match { + case remote: PromQlRemoteExec => (remote.promQlQueryParams, remote.queryEndpoint) + case grpc: PromQLGrpcRemoteExec => (grpc.promQlQueryParams, grpc.queryEndpoint) + case _ => fail("unsupported execplan") + } + // Each plan should dispatch the same query. + params.promQl shouldEqual test.query + (url, TimeStepParams(params.startSecs, params.stepSecs, params.endSecs)) + }.toSet shouldEqual expectedQueryParams // sanity check println("\n\n" + root.printTree()) validatePlan(root, test.expected) - tests2.append((test.query, test.lookbackSec, test.offsetSec, execPlan)) } - printTests3(tests2) } it ("should fail to materialize unsupported split-partition queries with binary joins") { @@ -2916,7 +2778,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS origQueryParams = PromQlQueryParams(query2, startSeconds, step, endSeconds), plannerParams = PlannerParams(processMultiPartition = true))) - val expectedPlan2 = """E~PromQlRemoteExec(PromQlQueryParams(topk(2.0,test{_ws_="demo",_ns_="RemoteNs",instance="Inst-1"}),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher""" + val expectedPlan2 = """E~PromQlRemoteExec(PromQlQueryParams(topk(2, test{_ws_ = "demo", _ns_ = "RemoteNs", instance = "Inst-1"}),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""" validatePlan(execPlan2, expectedPlan2) // Case 3: top k with regex, the resolved regex should be in local and remote partitions and use PromQLRemoteExec, should fail @@ -2943,7 +2805,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS override def getMetadataPartitions(nonMetricShardKeyFilters: Seq[ColumnFilter], timeRange: TimeRange): List[PartitionAssignment] = ??? } val grpcRemoteMultiPartitionPlanner = new MultiPartitionPlanner(gRpcRemotePartitionLocationProvider, singlePartitionPlanner, - "localPartition", dataset, queryConfig, shardKeyMatcher = shardKeyMatcherFn) + "localPartition", dataset, queryConfig) val gRpcRemoteRootPlanner = new ShardKeyRegexPlanner(dataset, grpcRemoteMultiPartitionPlanner, shardKeyMatcherFn, gRpcRemotePartitionLocationProvider, queryConfig) val query4 = """topk(2, test{_ws_ = "demo", _ns_ =~ ".*Ns", instance = "Inst-1"})""" @@ -2955,23 +2817,23 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS val expectedPlan4 = """T~AggregatePresenter(aggrOp=TopK, aggrParams=List(2.0), rangeParams=RangeParams(1633913330,300,1634777330)) - |-E~MultiPartitionReduceAggregateExec(aggrOp=TopK, aggrParams=List(2.0)) on InProcessPlanDispatcher - |--E~PromQLGrpcRemoteExec(PromQlQueryParams(topk(2.0,test{_ws_="demo",_ns_=~".*Ns",instance="Inst-1"}),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false,true), queryEndpoint=remotePartition-grpcUrl.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher - |--E~StitchRvsExec() on InProcessPlanDispatcher - |---E~LocalPartitionReduceAggregateExec(aggrOp=TopK, aggrParams=List(2.0)) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |----T~AggregateMapReduce(aggrOp=TopK, aggrParams=List(2.0), without=List(), by=List()) - |-----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |----T~AggregateMapReduce(aggrOp=TopK, aggrParams=List(2.0), without=List(), by=List()) - |-----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |---E~LocalPartitionReduceAggregateExec(aggrOp=TopK, aggrParams=List(2.0)) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |----T~AggregateMapReduce(aggrOp=TopK, aggrParams=List(2.0), without=List(), by=List()) - |-----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |----T~AggregateMapReduce(aggrOp=TopK, aggrParams=List(2.0), without=List(), by=List()) - |-----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(.*Ns)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample)""".stripMargin + |-E~MultiPartitionReduceAggregateExec(aggrOp=TopK, aggrParams=List(2.0)) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |--E~PromQLGrpcRemoteExec(PromQlQueryParams(topk(2.0,test{instance="Inst-1",_ws_="demo",_ns_="remoteNs"}),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remotePartition-grpcUrl.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |--E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,false,false,true)) + |---E~LocalPartitionReduceAggregateExec(aggrOp=TopK, aggrParams=List(2.0)) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1737727940],raw) + |----T~AggregateMapReduce(aggrOp=TopK, aggrParams=List(2.0), without=List(), by=List()) + |-----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1737727940],raw) + |----T~AggregateMapReduce(aggrOp=TopK, aggrParams=List(2.0), without=List(), by=List()) + |-----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1737727940],raw) + |---E~LocalPartitionReduceAggregateExec(aggrOp=TopK, aggrParams=List(2.0)) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1737727940],downsample) + |----T~AggregateMapReduce(aggrOp=TopK, aggrParams=List(2.0), without=List(), by=List()) + |-----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1737727940],downsample) + |----T~AggregateMapReduce(aggrOp=TopK, aggrParams=List(2.0), without=List(), by=List()) + |-----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1737727940],downsample)""".stripMargin validatePlan(execPlan4, expectedPlan4) @@ -2985,23 +2847,38 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS plannerParams = PlannerParams(processMultiPartition = true))) val expectedPlan5 = - """E~StitchRvsExec() on InProcessPlanDispatcher - |-T~AggregatePresenter(aggrOp=TopK, aggrParams=List(2.0), rangeParams=RangeParams(1634173130,300,1634777330)) - |--E~LocalPartitionReduceAggregateExec(aggrOp=TopK, aggrParams=List(2.0)) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |---T~AggregateMapReduce(aggrOp=TopK, aggrParams=List(2.0), without=List(), by=List()) - |----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(localNs.*)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |---T~AggregateMapReduce(aggrOp=TopK, aggrParams=List(2.0), without=List(), by=List()) - |----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(localNs.*)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-T~AggregatePresenter(aggrOp=TopK, aggrParams=List(2.0), rangeParams=RangeParams(1633913330,300,1634172830)) - |--E~LocalPartitionReduceAggregateExec(aggrOp=TopK, aggrParams=List(2.0)) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |---T~AggregateMapReduce(aggrOp=TopK, aggrParams=List(2.0), without=List(), by=List()) - |----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(localNs.*)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |---T~AggregateMapReduce(aggrOp=TopK, aggrParams=List(2.0), without=List(), by=List()) - |----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(localNs.*)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample)""".stripMargin + """T~AggregatePresenter(aggrOp=TopK, aggrParams=List(2.0), rangeParams=RangeParams(1633913330,300,1634777330)) + |-E~MultiPartitionReduceAggregateExec(aggrOp=TopK, aggrParams=List(2.0)) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |--E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,false,false,true)) + |---E~LocalPartitionReduceAggregateExec(aggrOp=TopK, aggrParams=List(2.0)) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1469117252],raw) + |----T~AggregateMapReduce(aggrOp=TopK, aggrParams=List(2.0), without=List(), by=List()) + |-----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1469117252],raw) + |----T~AggregateMapReduce(aggrOp=TopK, aggrParams=List(2.0), without=List(), by=List()) + |-----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1469117252],raw) + |---E~LocalPartitionReduceAggregateExec(aggrOp=TopK, aggrParams=List(2.0)) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1469117252],downsample) + |----T~AggregateMapReduce(aggrOp=TopK, aggrParams=List(2.0), without=List(), by=List()) + |-----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1469117252],downsample) + |----T~AggregateMapReduce(aggrOp=TopK, aggrParams=List(2.0), without=List(), by=List()) + |-----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1469117252],downsample) + |--E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,false,false,true)) + |---E~LocalPartitionReduceAggregateExec(aggrOp=TopK, aggrParams=List(2.0)) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1469117252],raw) + |----T~AggregateMapReduce(aggrOp=TopK, aggrParams=List(2.0), without=List(), by=List()) + |-----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1469117252],raw) + |----T~AggregateMapReduce(aggrOp=TopK, aggrParams=List(2.0), without=List(), by=List()) + |-----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1469117252],raw) + |---E~LocalPartitionReduceAggregateExec(aggrOp=TopK, aggrParams=List(2.0)) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1469117252],downsample) + |----T~AggregateMapReduce(aggrOp=TopK, aggrParams=List(2.0), without=List(), by=List()) + |-----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1469117252],downsample) + |----T~AggregateMapReduce(aggrOp=TopK, aggrParams=List(2.0), without=List(), by=List()) + |-----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1469117252],downsample)""".stripMargin validatePlan(execPlan5, expectedPlan5) // Case 6: top k with regex, the resolved regex should all be two remote partition, both PromQLRemoteExec should fail @@ -3050,7 +2927,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS } val twoGrpcRemoteMpPlanner = new MultiPartitionPlanner(twoRemoteGrpcPartitionLocationProvider, singlePartitionPlanner, - "localPartition", dataset, queryConfig, shardKeyMatcher = twoRemoteShardKeyMatcherFn) + "localPartition", dataset, queryConfig) val twoGrpcRemoteShardKeyRegexPlanner = new ShardKeyRegexPlanner(dataset, twoGrpcRemoteMpPlanner, twoRemoteShardKeyMatcherFn, twoRemoteGrpcPartitionLocationProvider, queryConfig) val query8 = """topk(2, test{_ws_ = "demo", _ns_ =~ "remoteNs.*", instance = "Inst-1"})""" @@ -3064,9 +2941,9 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS val expectedPlan8 = """T~AggregatePresenter(aggrOp=TopK, aggrParams=List(2.0), rangeParams=RangeParams(1633913330,300,1634777330)) - |-E~MultiPartitionReduceAggregateExec(aggrOp=TopK, aggrParams=List(2.0)) on InProcessPlanDispatcher - |--E~PromQLGrpcRemoteExec(PromQlQueryParams(topk(2.0,test{_ws_="demo",_ns_=~"remoteNs.*",instance="Inst-1"}),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false,true), queryEndpoint=remotePartition-grpc-url2.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher - |--E~PromQLGrpcRemoteExec(PromQlQueryParams(topk(2.0,test{_ws_="demo",_ns_=~"remoteNs.*",instance="Inst-1"}),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false,true), queryEndpoint=remotePartition-grpc-url1.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin + |-E~MultiPartitionReduceAggregateExec(aggrOp=TopK, aggrParams=List(2.0)) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |--E~PromQLGrpcRemoteExec(PromQlQueryParams(topk(2.0,test{instance="Inst-1",_ws_="demo",_ns_="remoteNs1"}),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remotePartition-grpc-url2.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true)) + |--E~PromQLGrpcRemoteExec(PromQlQueryParams(topk(2.0,test{instance="Inst-1",_ws_="demo",_ns_="remoteNs0"}),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remotePartition-grpc-url1.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin validatePlan(execPlan8, expectedPlan8) // Case 9: top k with regex, the resolved regex should all be one remote partition, using PromQLRemoteExec, should be supported BUT fails @@ -3084,24 +2961,30 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS } val oneRemoteMpPlanner = new MultiPartitionPlanner(singleRemotePartitionLocationProvider, singlePartitionPlanner, - "localPartition", dataset, queryConfig, shardKeyMatcher = twoRemoteShardKeyMatcherFn) + "localPartition", dataset, queryConfig) val oneRemoteShardKeyRegexPlanner = new ShardKeyRegexPlanner(dataset, oneRemoteMpPlanner, twoRemoteShardKeyMatcherFn, singleRemotePartitionLocationProvider, queryConfig) val query9 = """topk(2, test{_ws_ = "demo", _ns_ =~ "remoteNs.*", instance = "Inst-1"})""" val lp9 = Parser.queryRangeToLogicalPlan(query9, TimeStepParams(startSeconds, step, endSeconds), Antlr) - val execPlan9 = oneRemoteShardKeyRegexPlanner.materialize(lp9, + + // TODO: Shardkey regex planner simply rejects based on multiple namespaces seen in the query and target partition does not + // support gRPC endpoint. The partition location Provider should be available with ShardKeyRegexPlanner as well to let it push + // down the entire plan to remote partition. This might be more involved as MultiPartitionPlanner will not be able to resolve the regex to + // individual namespaces. Even in general, there is a scope to push the query with regex resolving to a single + // partition to that partition completely. One solution is for the proxy to simply route to remote shard key regex planner and + // return the response as is to caller + intercept[UnsupportedOperationException] { + oneRemoteShardKeyRegexPlanner.materialize(lp9, QueryContext( origQueryParams = PromQlQueryParams(query9, startSeconds, step, endSeconds), plannerParams = PlannerParams(processMultiPartition = true))) - val expectedPlan9 = """E~PromQlRemoteExec(PromQlQueryParams(topk(2.0,test{_ws_="demo",_ns_=~"remoteNs.*",instance="Inst-1"}),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remotePartition-url0, requestTimeoutMs=10000) on InProcessPlanDispatcher""" - validatePlan(execPlan9, expectedPlan9) + } // Case 10: MPP configured not to use grpc but partition assignment has grpc endpoint configured val twoGrpcRemoteMpPlannerNoGrpc = new MultiPartitionPlanner(twoRemoteGrpcPartitionLocationProvider, singlePartitionPlanner, "localPartition", dataset, - queryConfig.copy(grpcPartitionsDenyList = Set("remotepartition1")), - shardKeyMatcher = twoRemoteShardKeyMatcherFn) + queryConfig.copy(grpcPartitionsDenyList = Set("remotepartition1"))) val twoGrpcRemoteShardKeyRegexPlannerNoGrpc = new ShardKeyRegexPlanner(dataset, twoGrpcRemoteMpPlannerNoGrpc, twoRemoteShardKeyMatcherFn, twoRemoteGrpcPartitionLocationProvider, queryConfig) @@ -3119,9 +3002,9 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS val expectedPlan10 = """T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634777330)) - |-E~MultiPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on InProcessPlanDispatcher - |--E~PromQLGrpcRemoteExec(PromQlQueryParams(sum(test{_ws_="demo",_ns_=~"remoteNs.*",instance="Inst-1"}),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false,true), queryEndpoint=remotePartition-grpc-url2.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher - |--E~PromQlRemoteExec(PromQlQueryParams(sum(test{_ws_="demo",_ns_=~"remoteNs.*",instance="Inst-1"}),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false,true), queryEndpoint=remotePartition-url1, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin + |-E~MultiPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set())) + |--E~PromQLGrpcRemoteExec(PromQlQueryParams(sum(test{instance="Inst-1",_ws_="demo",_ns_="remoteNs1"}),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remotePartition-grpc-url2.execStreaming, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set(remotepartition1))) + |--E~PromQlRemoteExec(PromQlQueryParams(sum(test{instance="Inst-1",_ws_="demo",_ns_="remoteNs0"}),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remotePartition-url1, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true,Set(remotepartition1)))""".stripMargin validatePlan(execPlan10, expectedPlan10) } @@ -3844,7 +3727,6 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |-E~PromQlRemoteExec(PromQlQueryParams((foo{tschemaLabel="bar",_ws_="demo",_ns_="oneRemote1"} + on(tschemaLabel) sum(baz{tschemaLabel="bar",_ws_="demo",_ns_="oneRemote1"}) by (tschemaLabel)),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,Some(FunctionalTargetSchemaProvider(~)),60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000),PerQueryLimits(50000,15000000,50000,50000,150000,500000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remote-url, requestTimeoutMs=10000) on InProcessPlanDispatcher |-E~PromQlRemoteExec(PromQlQueryParams((foo{tschemaLabel="bar",_ws_="demo",_ns_="oneRemote2"} + on(tschemaLabel) sum(baz{tschemaLabel="bar",_ws_="demo",_ns_="oneRemote2"}) by (tschemaLabel)),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,Some(FunctionalTargetSchemaProvider(~)),60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000),PerQueryLimits(50000,15000000,50000,50000,150000,500000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remote-url, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin) ) - val tests2 = new mutable.ArrayBuffer[(String, Set[String], ExecPlan)] for (test <- tests) { def tschemaProviderFunc(filters: Seq[ColumnFilter]): Seq[TargetSchemaChange] = { val ns = filters.find(f => f.column == "_ns_").get.filter.valuesStrings.toList.head.asInstanceOf[String] @@ -3858,7 +3740,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS if (shardColumnFilters.isEmpty) { return Nil } - val nsColValue = shardColumnFilters.find(_.column == "_ns_").get.filter.valuesStrings.head.toString + val nsColValue = shardColumnFilters.find(_.column == "_ns_").get.filter.asInstanceOf[EqualsRegex].pattern.toString nsColValue match { case LOCAL => Seq( @@ -3883,8 +3765,8 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS } } - val spp = getPlanners(2, dataset, shardKeyMatcherFunc).spp - val multiPartitionPlanner = new MultiPartitionPlanner(partitionLocationProvider, spp, "local", dataset, queryConfig, shardKeyMatcher = shardKeyMatcherFunc) + val spp = getPlanners(2, dataset).spp + val multiPartitionPlanner = new MultiPartitionPlanner(partitionLocationProvider, spp, "local", dataset, queryConfig) val shardKeyRegexPlanner = new ShardKeyRegexPlanner(dataset, multiPartitionPlanner, shardKeyMatcherFunc, partitionLocationProvider, queryConfig, targetSchemaProvider) val tschema = FunctionalTargetSchemaProvider(tschemaProviderFunc) @@ -3894,17 +3776,8 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS origQueryParams = PromQlQueryParams(test.query, timeParams.start, timeParams.step, timeParams.end), plannerParams = PlannerParams(processMultiPartition = true, targetSchemaProviderOverride = Some(tschema))) val ep = shardKeyRegexPlanner.materialize(lp, context) - tests2.append((test.query, test.tschemaEnabled, ep)) - try { - validatePlan(ep, test.expected, sort = true) - } catch { - case t: Throwable => { - println("failed: " + test.query) - throw t - } - } + validatePlan(ep, test.expected, sort = true) } - printTests2(tests2) } it("should generate correct plan for aggregations/joins without selectors") { @@ -3936,271 +3809,4 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS validatePlan(execPlan, expected) } } - - it ("should create one remote plan per remote partition for ns-regex queries") { - val dataset = MetricsTestData.timeseriesDatasetMultipleShardKeys - val shardKeyMatcher: Seq[ColumnFilter] => Seq[Seq[ColumnFilter]] = filters => Seq( - // Selectors will always match four keys. - // Note: _ws_ label is interpreted below as the partition name. - Seq(ColumnFilter("_ws_", Filter.Equals("local")), ColumnFilter("_ns_", Filter.Equals("local1"))), - Seq(ColumnFilter("_ws_", Filter.Equals("local")), ColumnFilter("_ns_", Filter.Equals("local2"))), - Seq(ColumnFilter("_ws_", Filter.Equals("remote")), ColumnFilter("_ns_", Filter.Equals("remote1"))), - Seq(ColumnFilter("_ws_", Filter.Equals("remote")), ColumnFilter("_ns_", Filter.Equals("remote2"))), - ) - val partitionLocationProvider = new PartitionLocationProvider { - override def getPartitions(routingKey: Map[String, String], timeRange: TimeRange): List[PartitionAssignment] = { - // using the _ws_ label as the partition name - List(PartitionAssignment(routingKey("_ws_"), "dummy-endpoint", timeRange)) - } - override def getMetadataPartitions(nonMetricShardKeyFilters: Seq[ColumnFilter], timeRange: TimeRange): List[PartitionAssignment] = - getPartitions(nonMetricShardKeyFilters.map(filter => (filter.column, filter.filter.valuesStrings.head.toString)).toMap, timeRange) - } - - val planners = getPlanners(2, dataset, shardKeyMatcher) - val mppPlanner = new MultiPartitionPlanner(partitionLocationProvider, planners.spp, "local", dataset, queryConfig, shardKeyMatcher = shardKeyMatcher) - val planner = new ShardKeyRegexPlanner(dataset, mppPlanner, shardKeyMatcher, partitionLocationProvider, queryConfig) - - val timeParams = TimeStepParams(startSeconds, step, endSeconds) - val qContext = QueryContext(origQueryParams = queryParams, - plannerParams = PlannerParams(processMultiPartition = true)) - - // Each should materialize plans for two partitions, and _ns_ labels should be filtered by EqualsRegex with pipe-separated values. - val queryExpectedPairs = Seq( - ("""foo{_ws_="dummy", _ns_=~"dummy.*", bar="hello"}""", - """E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192))) - |--E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#31671711],raw) - |---T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(dummy)), ColumnFilter(_ns_,EqualsRegex(dummy.*)), ColumnFilter(bar,Equals(hello)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#31671711],raw) - |---T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(dummy)), ColumnFilter(_ns_,EqualsRegex(dummy.*)), ColumnFilter(bar,Equals(hello)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#31671711],raw) - |--E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#31671711],downsample) - |---T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(dummy)), ColumnFilter(_ns_,EqualsRegex(dummy.*)), ColumnFilter(bar,Equals(hello)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#31671711],downsample) - |---T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(dummy)), ColumnFilter(_ns_,EqualsRegex(dummy.*)), ColumnFilter(bar,Equals(hello)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#31671711],downsample) - |-E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="dummy",_ns_=~"dummy.*",bar="hello"},100,1,1000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=dummy-endpoint, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - ("""sum(foo{_ws_="dummy", _ns_=~"dummy.*", bar="hello"})""", - """T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(100,1,1000)) - |-E~MultiPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |--E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192))) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#31671711],raw) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |-----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(dummy)), ColumnFilter(_ns_,EqualsRegex(dummy.*)), ColumnFilter(bar,Equals(hello)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#31671711],raw) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |-----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(dummy)), ColumnFilter(_ns_,EqualsRegex(dummy.*)), ColumnFilter(bar,Equals(hello)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#31671711],raw) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#31671711],downsample) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |-----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(dummy)), ColumnFilter(_ns_,EqualsRegex(dummy.*)), ColumnFilter(bar,Equals(hello)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#31671711],downsample) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |-----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(dummy)), ColumnFilter(_ns_,EqualsRegex(dummy.*)), ColumnFilter(bar,Equals(hello)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#31671711],downsample) - |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="dummy",_ns_=~"dummy.*",bar="hello"}),100,1,1000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false,true), queryEndpoint=dummy-endpoint, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - ("""foo{_ws_="dummy", _ns_=~"dummy.*", bar="hello"} + foo{_ws_="dummy", _ns_=~"dummy.*", bar="goodbye"}""", - """E~BinaryJoinExec(binaryOp=ADD, on=List(), ignoring=List()) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |--E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192))) - |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#31671711],raw) - |----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(dummy)), ColumnFilter(_ns_,EqualsRegex(dummy.*)), ColumnFilter(bar,Equals(hello)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#31671711],raw) - |----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(dummy)), ColumnFilter(_ns_,EqualsRegex(dummy.*)), ColumnFilter(bar,Equals(hello)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#31671711],raw) - |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#31671711],downsample) - |----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(dummy)), ColumnFilter(_ns_,EqualsRegex(dummy.*)), ColumnFilter(bar,Equals(hello)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#31671711],downsample) - |----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(dummy)), ColumnFilter(_ns_,EqualsRegex(dummy.*)), ColumnFilter(bar,Equals(hello)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#31671711],downsample) - |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="dummy",_ns_=~"dummy.*",bar="hello"},100,1,1000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=dummy-endpoint, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |--E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192))) - |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#31671711],raw) - |----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(dummy)), ColumnFilter(_ns_,EqualsRegex(dummy.*)), ColumnFilter(bar,Equals(goodbye)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#31671711],raw) - |----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(dummy)), ColumnFilter(_ns_,EqualsRegex(dummy.*)), ColumnFilter(bar,Equals(goodbye)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#31671711],raw) - |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#31671711],downsample) - |----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(dummy)), ColumnFilter(_ns_,EqualsRegex(dummy.*)), ColumnFilter(bar,Equals(goodbye)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#31671711],downsample) - |----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(dummy)), ColumnFilter(_ns_,EqualsRegex(dummy.*)), ColumnFilter(bar,Equals(goodbye)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#31671711],downsample) - |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="dummy",_ns_=~"dummy.*",bar="goodbye"},100,1,1000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=dummy-endpoint, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin) - ) - val plans = new ArrayBuffer[ExecPlan] - for ((query, expected) <- queryExpectedPairs) { - val lp = Parser.queryRangeToLogicalPlan(query, timeParams) - val exec = planner.materialize(lp, qContext) - validatePlan(exec, expected) - plans.append(exec) - } - printTests(queryExpectedPairs.map(_._1).zip(plans)) - } - - it ("should create one remote plan per remote partition for ns-regex queries with target-schemas") { - val dataset = MetricsTestData.timeseriesDatasetMultipleShardKeys - val shardKeyMatcher: Seq[ColumnFilter] => Seq[Seq[ColumnFilter]] = filters => Seq( - // Selectors will always match four keys. - // Note: _ws_ label is interpreted below as the partition name. - Seq(ColumnFilter("_ws_", Filter.Equals("local")), ColumnFilter("_ns_", Filter.Equals("local1"))), - Seq(ColumnFilter("_ws_", Filter.Equals("local")), ColumnFilter("_ns_", Filter.Equals("local2"))), - Seq(ColumnFilter("_ws_", Filter.Equals("remote")), ColumnFilter("_ns_", Filter.Equals("remote1"))), - Seq(ColumnFilter("_ws_", Filter.Equals("remote")), ColumnFilter("_ns_", Filter.Equals("remote2"))), - ) - val partitionLocationProvider = new PartitionLocationProvider { - override def getPartitions(routingKey: Map[String, String], timeRange: TimeRange): List[PartitionAssignment] = { - // using the _ws_ label as the partition name - List(PartitionAssignment(routingKey("_ws_"), "dummy-endpoint", timeRange)) - } - override def getMetadataPartitions(nonMetricShardKeyFilters: Seq[ColumnFilter], timeRange: TimeRange): List[PartitionAssignment] = - getPartitions(nonMetricShardKeyFilters.map(filter => (filter.column, filter.filter.valuesStrings.head.toString)).toMap, timeRange) - } - val tschemaProvider = FunctionalTargetSchemaProvider(cf => Seq(TargetSchemaChange(0L, Seq("_ws_", "_ns_", "bar")))) - - // Since two _ns_ labels per partition and tschemas are enabled, we should expect the result plan to contain leaves for at most two shards. - val nShards = 4 - val spreadProv = StaticSpreadProvider(SpreadChange(0L, Integer.numberOfTrailingZeros(nShards))) - - val singlePartitionPlanner = getPlanners(nShards, dataset, shardKeyMatcher).spp - val mppPlanner = new MultiPartitionPlanner(partitionLocationProvider, singlePartitionPlanner, "local", dataset, queryConfig, shardKeyMatcher = shardKeyMatcher) - val planner = new ShardKeyRegexPlanner(dataset, mppPlanner, shardKeyMatcher, partitionLocationProvider, queryConfig) - - val timeParams = TimeStepParams(startSeconds, step, endSeconds) - val qContext = QueryContext(origQueryParams = queryParams, - plannerParams = PlannerParams( - processMultiPartition = true, - targetSchemaProviderOverride = Some(tschemaProvider), - spreadOverride = Some(spreadProv))) - - // Each should materialize plans for two partitions, and _ns_ labels should be filtered by EqualsRegex with pipe-separated values. - // Leaves should be materialized for at most two shards per selector. - val queryExpectedPairs = Seq( - ("""foo{_ws_="dummy", _ns_=~"dummy.*", bar="hello"}""", - """E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192))) - |--E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#978006600],raw) - |---T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=2, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(dummy)), ColumnFilter(_ns_,EqualsRegex(dummy.*)), ColumnFilter(bar,Equals(hello)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#978006600],raw) - |---T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(dummy)), ColumnFilter(_ns_,EqualsRegex(dummy.*)), ColumnFilter(bar,Equals(hello)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#978006600],raw) - |---T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=3, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(dummy)), ColumnFilter(_ns_,EqualsRegex(dummy.*)), ColumnFilter(bar,Equals(hello)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#978006600],raw) - |--E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#978006600],downsample) - |---T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=2, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(dummy)), ColumnFilter(_ns_,EqualsRegex(dummy.*)), ColumnFilter(bar,Equals(hello)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#978006600],downsample) - |---T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(dummy)), ColumnFilter(_ns_,EqualsRegex(dummy.*)), ColumnFilter(bar,Equals(hello)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#978006600],downsample) - |---T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=3, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(dummy)), ColumnFilter(_ns_,EqualsRegex(dummy.*)), ColumnFilter(bar,Equals(hello)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#978006600],downsample) - |-E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="dummy",_ns_=~"dummy.*",bar="hello"},100,1,1000,None,false), PlannerParams(filodb,None,Some(StaticSpreadProvider(SpreadChange(0,2))),None,Some(FunctionalTargetSchemaProvider(~)),60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=dummy-endpoint, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - ("""sum(foo{_ws_="dummy", _ns_=~"dummy.*", bar="hello"})""", - """T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(100,1,1000)) - |-E~MultiPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |--E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192))) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#978006600],raw) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |-----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=2, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(dummy)), ColumnFilter(_ns_,EqualsRegex(dummy.*)), ColumnFilter(bar,Equals(hello)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#978006600],raw) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |-----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(dummy)), ColumnFilter(_ns_,EqualsRegex(dummy.*)), ColumnFilter(bar,Equals(hello)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#978006600],raw) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |-----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=3, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(dummy)), ColumnFilter(_ns_,EqualsRegex(dummy.*)), ColumnFilter(bar,Equals(hello)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#978006600],raw) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#978006600],downsample) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |-----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=2, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(dummy)), ColumnFilter(_ns_,EqualsRegex(dummy.*)), ColumnFilter(bar,Equals(hello)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#978006600],downsample) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |-----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(dummy)), ColumnFilter(_ns_,EqualsRegex(dummy.*)), ColumnFilter(bar,Equals(hello)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#978006600],downsample) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |-----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=3, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(dummy)), ColumnFilter(_ns_,EqualsRegex(dummy.*)), ColumnFilter(bar,Equals(hello)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#978006600],downsample) - |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="dummy",_ns_=~"dummy.*",bar="hello"}),100,1,1000,None,false), PlannerParams(filodb,None,Some(StaticSpreadProvider(SpreadChange(0,2))),None,Some(FunctionalTargetSchemaProvider(~)),60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false,true), queryEndpoint=dummy-endpoint, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin), - ("""foo{_ws_="dummy", _ns_=~"dummy.*", bar="hello"} + foo{_ws_="dummy", _ns_=~"dummy.*", bar="goodbye"}""", - """E~BinaryJoinExec(binaryOp=ADD, on=List(), ignoring=List()) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |--E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192))) - |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#978006600],raw) - |----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=2, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(dummy)), ColumnFilter(_ns_,EqualsRegex(dummy.*)), ColumnFilter(bar,Equals(hello)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#978006600],raw) - |----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(dummy)), ColumnFilter(_ns_,EqualsRegex(dummy.*)), ColumnFilter(bar,Equals(hello)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#978006600],raw) - |----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=3, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(dummy)), ColumnFilter(_ns_,EqualsRegex(dummy.*)), ColumnFilter(bar,Equals(hello)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#978006600],raw) - |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#978006600],downsample) - |----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=2, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(dummy)), ColumnFilter(_ns_,EqualsRegex(dummy.*)), ColumnFilter(bar,Equals(hello)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#978006600],downsample) - |----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(dummy)), ColumnFilter(_ns_,EqualsRegex(dummy.*)), ColumnFilter(bar,Equals(hello)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#978006600],downsample) - |----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=3, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(dummy)), ColumnFilter(_ns_,EqualsRegex(dummy.*)), ColumnFilter(bar,Equals(hello)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#978006600],downsample) - |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="dummy",_ns_=~"dummy.*",bar="hello"},100,1,1000,None,false), PlannerParams(filodb,None,Some(StaticSpreadProvider(SpreadChange(0,2))),None,Some(FunctionalTargetSchemaProvider(~)),60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=dummy-endpoint, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |--E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192))) - |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#978006600],raw) - |----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(dummy)), ColumnFilter(_ns_,EqualsRegex(dummy.*)), ColumnFilter(bar,Equals(goodbye)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#978006600],raw) - |----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(dummy)), ColumnFilter(_ns_,EqualsRegex(dummy.*)), ColumnFilter(bar,Equals(goodbye)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#978006600],raw) - |----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=2, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(dummy)), ColumnFilter(_ns_,EqualsRegex(dummy.*)), ColumnFilter(bar,Equals(goodbye)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#978006600],raw) - |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#978006600],downsample) - |----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(dummy)), ColumnFilter(_ns_,EqualsRegex(dummy.*)), ColumnFilter(bar,Equals(goodbye)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#978006600],downsample) - |----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(dummy)), ColumnFilter(_ns_,EqualsRegex(dummy.*)), ColumnFilter(bar,Equals(goodbye)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#978006600],downsample) - |----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=2, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(dummy)), ColumnFilter(_ns_,EqualsRegex(dummy.*)), ColumnFilter(bar,Equals(goodbye)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#978006600],downsample) - |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="dummy",_ns_=~"dummy.*",bar="goodbye"},100,1,1000,None,false), PlannerParams(filodb,None,Some(StaticSpreadProvider(SpreadChange(0,2))),None,Some(FunctionalTargetSchemaProvider(~)),60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=dummy-endpoint, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin) - ) - val plans = new mutable.ArrayBuffer[ExecPlan] - for ((query, expected) <- queryExpectedPairs) { - val lp = Parser.queryRangeToLogicalPlan(query, timeParams) - val exec = planner.materialize(lp, qContext) - validatePlan(exec, expected) - plans.append(exec) - } - printTests(queryExpectedPairs.map(_._1).zip(plans)) - } - - it("should materialize a plan per shard-key when reduceShardKeyRegexFanout=false") { - val dataset = MetricsTestData.timeseriesDatasetMultipleShardKeys - val shardKeyMatcher: Seq[ColumnFilter] => Seq[Seq[ColumnFilter]] = filters => Seq( - // Selectors will always match four keys. - Seq(ColumnFilter("_ws_", Filter.Equals("foo_1")), ColumnFilter("_ns_", Filter.Equals("ns1"))), - Seq(ColumnFilter("_ws_", Filter.Equals("foo_1")), ColumnFilter("_ns_", Filter.Equals("ns2"))), - Seq(ColumnFilter("_ws_", Filter.Equals("bar_2")), ColumnFilter("_ns_", Filter.Equals("ns3"))), - Seq(ColumnFilter("_ws_", Filter.Equals("baz_2")), ColumnFilter("_ns_", Filter.Equals("ns4"))), - ) - val partitionLocationProvider = new PartitionLocationProvider { - override def getPartitions(routingKey: Map[String, String], timeRange: TimeRange): List[PartitionAssignment] = { - // using the last char of the _ws_ label as the partition name - List(PartitionAssignment(routingKey("_ws_").last.toString, "dummy-endpoint", timeRange)) - } - - override def getMetadataPartitions(nonMetricShardKeyFilters: Seq[ColumnFilter], timeRange: TimeRange): List[PartitionAssignment] = - getPartitions(nonMetricShardKeyFilters.map(filter => (filter.column, filter.filter.valuesStrings.head.toString)).toMap, timeRange) - } - - val mppPlanner = new MultiPartitionPlanner(partitionLocationProvider, singlePartitionPlanner, "local", dataset, queryConfig) - val planner = new ShardKeyRegexPlanner(dataset, mppPlanner, shardKeyMatcher, partitionLocationProvider, queryConfig) - - val timeParams = TimeStepParams(startSeconds, step, endSeconds) - val qContext = QueryContext(origQueryParams = queryParams, - plannerParams = PlannerParams(processMultiPartition = true, reduceShardKeyRegexFanout = false)) - - val query = """foo{_ws_="dummy", _ns_=~"dummy.*", bar="hello"}""" - val expected = - """E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(foo{bar="hello",_ws_="foo_1",_ns_="ns1"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false,false), queryEndpoint=dummy-endpoint, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(foo{bar="hello",_ws_="foo_1",_ns_="ns2"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false,false), queryEndpoint=dummy-endpoint, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(foo{bar="hello",_ws_="bar_2",_ns_="ns3"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false,false), queryEndpoint=dummy-endpoint, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~PromQlRemoteExec(PromQlQueryParams(foo{bar="hello",_ws_="baz_2",_ns_="ns4"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false,false), queryEndpoint=dummy-endpoint, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin - val lp = Parser.queryRangeToLogicalPlan(query, timeParams) - val exec = planner.materialize(lp, qContext) - validatePlan(exec, expected) - } } diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlannerSpec.scala index 5ee9a229da..3f554a3392 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlannerSpec.scala @@ -3,20 +3,22 @@ package filodb.coordinator.queryplanner import akka.actor.ActorSystem import akka.testkit.TestProbe import com.typesafe.config.ConfigFactory -import filodb.coordinator.client.QueryCommands.StaticSpreadProvider import org.scalatest.concurrent.ScalaFutures import org.scalatest.funspec.AnyFunSpec import org.scalatest.matchers.should.Matchers + import filodb.coordinator.{ActorPlanDispatcher, ShardMapper} -import filodb.core.{MetricsTestData, SpreadChange} +import filodb.core.MetricsTestData import filodb.core.metadata.Schemas import filodb.prometheus.ast.TimeStepParams -import filodb.query.{InstantFunctionId, LogicalPlan, PlanValidationSpec, TsCardinalities} +import filodb.query.{BinaryOperator, InstantFunctionId, LogicalPlan, MiscellaneousFunctionId, PlanValidationSpec, SortFunctionId, TsCardinalities} import filodb.core.query.{ColumnFilter, PlannerParams, PromQlQueryParams, QueryConfig, QueryContext} import filodb.core.query.Filter.Equals import filodb.prometheus.parse.Parser +import filodb.query.InstantFunctionId.{Exp, HistogramQuantile, Ln} import filodb.query.exec._ - +import filodb.query.AggregationOperator._ +import scala.language.postfixOps class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFutures with PlanValidationSpec { @@ -37,12 +39,9 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture private val localMapper = new ShardMapper(32) for {i <- 0 until 32} localMapper.registerNode(Seq(i), node) - def makeLocalPlanner(shardKeyMatcher: Seq[ColumnFilter] => Seq[Seq[ColumnFilter]]): SingleClusterPlanner = { - new SingleClusterPlanner( - dataset, schemas, localMapper, earliestRetainedTimestampFn = 0, - queryConfig, "raw", shardKeyMatcher = shardKeyMatcher, - spreadProvider = StaticSpreadProvider(SpreadChange(time = Long.MinValue))) - } + + val localPlanner = new SingleClusterPlanner(dataset, schemas, localMapper, earliestRetainedTimestampFn = 0, + queryConfig, "raw") def partitions(timeRange: TimeRange): List[PartitionAssignment] = List(PartitionAssignment("remote", "remote-url", TimeRange(timeRange.startMs, timeRange.endMs))) @@ -64,7 +63,9 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture getPartitions(nonMetricShardKeyFilters.map(f => (f.column, f.filter.valuesStrings.head.toString)).toMap, timeRange) } val c = QueryConfig(config).copy(plannerSelector = Some("plannerSelector")) - + val mpp = new MultiPartitionPlanner( + mppPartitionLocationProvider, localPlanner, "local", dataset, c + ) val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq( Seq( @@ -77,11 +78,6 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture ) ) } - - val localPlanner = makeLocalPlanner(shardKeyMatcherFn) - val mpp = new MultiPartitionPlanner( - mppPartitionLocationProvider, localPlanner, "local", dataset, c, shardKeyMatcher = shardKeyMatcherFn - ) val skrp = new ShardKeyRegexPlanner(dataset, mpp, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) it("should generate Exec plan for simple query") { @@ -89,19 +85,14 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-2"))))} - val localPlanner = makeLocalPlanner(shardKeyMatcherFn) val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) - val expected = """E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1342019804],raw) - |-T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |--E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1342019804],raw) - |-T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |--E~MultiSchemaPartitionsExec(dataset=timeseries, shard=16, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1342019804],raw) - |-T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |--E~MultiSchemaPartitionsExec(dataset=timeseries, shard=6, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1342019804],raw) - |-T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |--E~MultiSchemaPartitionsExec(dataset=timeseries, shard=22, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1342019804],raw)""".stripMargin - validatePlan(execPlan, expected) + execPlan.isInstanceOf[MultiPartitionDistConcatExec] shouldEqual(true) + execPlan.children(0).children.head.isInstanceOf[MultiSchemaPartitionsExec] + execPlan.children(1).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", Equals("App-1"))) shouldEqual(true) + execPlan.children(0).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", Equals("App-2"))) shouldEqual(true) } it("should generate Exec plan for implicit ws query") { @@ -109,27 +100,25 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-2"))))} - val localPlanner = makeLocalPlanner(shardKeyMatcherFn) val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = PromQlQueryParams( """test{_ns_ =~ "App.*", instance = "Inst-1" }""", 100, 1, 1000))) - val expected = """E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |--E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |--E~MultiSchemaPartitionsExec(dataset=timeseries, shard=16, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |--E~MultiSchemaPartitionsExec(dataset=timeseries, shard=6, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |--E~MultiSchemaPartitionsExec(dataset=timeseries, shard=22, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw)""".stripMargin - validatePlan(execPlan, expected) + execPlan.isInstanceOf[MultiPartitionDistConcatExec] shouldEqual(true) + execPlan.children(0).children.head.isInstanceOf[MultiSchemaPartitionsExec] + execPlan.children(1).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", Equals("App-1"))) shouldEqual(true) + execPlan.children(0).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", Equals("App-2"))) shouldEqual(true) + execPlan.children(0).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ws_", Equals("demo"))) shouldEqual(true) + execPlan.children(1).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ws_", Equals("demo"))) shouldEqual(true) } it("should check for required non metric shard key filters") { val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-2"))))} - val localPlanner = makeLocalPlanner(shardKeyMatcherFn) val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val nonMetricShardColumns = dataset.options.nonMetricShardColumns @@ -145,18 +134,14 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture it("should generate Exec plan for subquery with windowing") { val expected = - """T~PeriodicSamplesMapper(start=1000000, step=0, end=1000000, window=Some(300000), functionId=Some(AvgOverTime), rawSource=false, offsetMs=None) - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |--E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |---T~PeriodicSamplesMapper(start=720000, step=60000, end=960000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(420000,960000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |---T~PeriodicSamplesMapper(start=720000, step=60000, end=960000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=16, chunkMethod=TimeRangeChunkScan(420000,960000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |---T~PeriodicSamplesMapper(start=720000, step=60000, end=960000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=6, chunkMethod=TimeRangeChunkScan(420000,960000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |---T~PeriodicSamplesMapper(start=720000, step=60000, end=960000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=22, chunkMethod=TimeRangeChunkScan(420000,960000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--E~PromQlRemoteExec(PromQlQueryParams(test{_ws_="demo",_ns_=~"App.*",instance="Inst-1"},100,1,1000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote-url, requestTimeoutMs=60000) on InProcessPlanDispatcher""".stripMargin + """|T~PeriodicSamplesMapper(start=1000000, step=0, end=1000000, window=Some(300000), functionId=Some(AvgOverTime), rawSource=false, offsetMs=None) + |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher + |--E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |---T~PeriodicSamplesMapper(start=720000, step=60000, end=960000, window=None, functionId=None, rawSource=true, offsetMs=None) + |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=6, chunkMethod=TimeRangeChunkScan(420000,960000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(App-2)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |---T~PeriodicSamplesMapper(start=720000, step=60000, end=960000, window=None, functionId=None, rawSource=true, offsetMs=None) + |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=22, chunkMethod=TimeRangeChunkScan(420000,960000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(App-2)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |--E~PromQlRemoteExec(PromQlQueryParams(test{instance="Inst-1",_ws_="demo",_ns_="App-1"},720,60,960,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remote-url, requestTimeoutMs=60000) on InProcessPlanDispatcher""".stripMargin val lp = Parser.queryToLogicalPlan( """avg_over_time(test{_ws_ = "demo", _ns_ =~ "App.*", instance = "Inst-1" }[5m:1m])""", 1000, 1000 @@ -167,7 +152,6 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture lp, QueryContext(origQueryParams = promQlQueryParams, plannerParams = PlannerParams(processMultiPartition = true)) ) - println(execPlan.printTree()) execPlan.isInstanceOf[MultiPartitionDistConcatExec] shouldEqual(true) execPlan.children(0).children.head.isInstanceOf[MultiSchemaPartitionsExec] validatePlan(execPlan, expected) @@ -181,21 +165,14 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-2"))))} - val localPlanner = makeLocalPlanner(shardKeyMatcherFn) val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) - val expected = """T~PeriodicSamplesMapper(start=1000000, step=0, end=1000000, window=Some(600000), functionId=Some(MaxOverTime), rawSource=false, offsetMs=None) - |-T~PeriodicSamplesMapper(start=420000, step=60000, end=960000, window=Some(300000), functionId=Some(AvgOverTime), rawSource=false, offsetMs=None) - |--E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#613989854],raw) - |---T~PeriodicSamplesMapper(start=120000, step=60000, end=960000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(-180000,960000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#613989854],raw) - |---T~PeriodicSamplesMapper(start=120000, step=60000, end=960000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=16, chunkMethod=TimeRangeChunkScan(-180000,960000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#613989854],raw) - |---T~PeriodicSamplesMapper(start=120000, step=60000, end=960000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=6, chunkMethod=TimeRangeChunkScan(-180000,960000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#613989854],raw) - |---T~PeriodicSamplesMapper(start=120000, step=60000, end=960000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=22, chunkMethod=TimeRangeChunkScan(-180000,960000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#613989854],raw)""".stripMargin - validatePlan(execPlan, expected) + execPlan.isInstanceOf[MultiPartitionDistConcatExec] shouldEqual(true) + execPlan.children(0).children.head.isInstanceOf[MultiSchemaPartitionsExec] + execPlan.children(1).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", Equals("App-1"))) shouldEqual(true) + execPlan.children(0).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", Equals("App-2"))) shouldEqual(true) } it("should generate Exec plan for subquery with windowing with aggregate") { @@ -206,29 +183,15 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-2"))))} - val localPlanner = makeLocalPlanner(shardKeyMatcherFn) val engine = new ShardKeyRegexPlanner( dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = PromQlQueryParams("sum(heap_usage)", 100, 1, 1000))) - val expected = """T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1000,0,1000)) - |-E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#25320684],raw) - |--T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |---T~PeriodicSamplesMapper(start=1000000, step=0, end=1000000, window=Some(300000), functionId=Some(AvgOverTime), rawSource=false, offsetMs=None) - |----T~PeriodicSamplesMapper(start=720000, step=60000, end=960000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(420000,960000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#25320684],raw) - |--T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |---T~PeriodicSamplesMapper(start=1000000, step=0, end=1000000, window=Some(300000), functionId=Some(AvgOverTime), rawSource=false, offsetMs=None) - |----T~PeriodicSamplesMapper(start=720000, step=60000, end=960000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=16, chunkMethod=TimeRangeChunkScan(420000,960000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#25320684],raw) - |--T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |---T~PeriodicSamplesMapper(start=1000000, step=0, end=1000000, window=Some(300000), functionId=Some(AvgOverTime), rawSource=false, offsetMs=None) - |----T~PeriodicSamplesMapper(start=720000, step=60000, end=960000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=6, chunkMethod=TimeRangeChunkScan(420000,960000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#25320684],raw) - |--T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |---T~PeriodicSamplesMapper(start=1000000, step=0, end=1000000, window=Some(300000), functionId=Some(AvgOverTime), rawSource=false, offsetMs=None) - |----T~PeriodicSamplesMapper(start=720000, step=60000, end=960000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=22, chunkMethod=TimeRangeChunkScan(420000,960000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#25320684],raw)""".stripMargin - validatePlan(execPlan, expected) + execPlan.isInstanceOf[MultiPartitionReduceAggregateExec] shouldEqual(true) + execPlan.children(0).children.head.isInstanceOf[MultiSchemaPartitionsExec] + execPlan.children(1).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", Equals("App-1"))) shouldEqual(true) + execPlan.children(0).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", Equals("App-2"))) shouldEqual(true) } it("should generate Exec plan for top level subquery") { @@ -236,19 +199,17 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture """E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher |-E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) |--T~PeriodicSamplesMapper(start=720000, step=60000, end=960000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(420000,960000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--T~PeriodicSamplesMapper(start=720000, step=60000, end=960000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=16, chunkMethod=TimeRangeChunkScan(420000,960000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--T~PeriodicSamplesMapper(start=720000, step=60000, end=960000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=6, chunkMethod=TimeRangeChunkScan(420000,960000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=6, chunkMethod=TimeRangeChunkScan(420000,960000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(App-2)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) |--T~PeriodicSamplesMapper(start=720000, step=60000, end=960000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=22, chunkMethod=TimeRangeChunkScan(420000,960000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-E~PromQlRemoteExec(PromQlQueryParams(test{_ws_="demo",_ns_=~"App.*",instance="Inst-1"},100,1,1000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true), queryEndpoint=remote-url, requestTimeoutMs=60000) on InProcessPlanDispatcher""".stripMargin + |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=22, chunkMethod=TimeRangeChunkScan(420000,960000), filters=List(ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(App-2)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |-E~PromQlRemoteExec(PromQlQueryParams(test{instance="Inst-1",_ws_="demo",_ns_="App-1"},720,60,960,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remote-url, requestTimeoutMs=60000) on InProcessPlanDispatcher""".stripMargin val lp = Parser.queryToLogicalPlan( """test{_ws_ = "demo", _ns_ =~ "App.*", instance = "Inst-1" }[5m:1m]""", 1000, 1000 ) val execPlan = skrp.materialize(lp, QueryContext(origQueryParams = promQlQueryParams, plannerParams = PlannerParams(processMultiPartition = true))) + execPlan.isInstanceOf[MultiPartitionDistConcatExec] shouldEqual(true) + execPlan.children(0).children.head.isInstanceOf[MultiSchemaPartitionsExec] validatePlan(execPlan, expected) } @@ -258,35 +219,23 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-2"))))} - val localPlanner = makeLocalPlanner(shardKeyMatcherFn) val engine = new ShardKeyRegexPlanner( dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = PromQlQueryParams("sum(heap_usage)", 100, 1, 1000))) - val expected = """T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1000,1000,1000)) - |-E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-2117041601],raw) - |--T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |---T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-2117041601],raw) - |--T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |---T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=16, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-2117041601],raw) - |--T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |---T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=6, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-2117041601],raw) - |--T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |---T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=22, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-2117041601],raw)""".stripMargin - validatePlan(execPlan, expected) + execPlan.isInstanceOf[MultiPartitionReduceAggregateExec] shouldEqual(true) + execPlan.children(0).children.head.isInstanceOf[MultiSchemaPartitionsExec] + execPlan.children(1).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", Equals("App-1"))) shouldEqual(true) + execPlan.children(0).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", Equals("App-2"))) shouldEqual(true) } it("should generate Exec plan for time()") { val lp = Parser.queryToLogicalPlan("time()", 1000, 1000) val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq((Seq.empty)) } - val localPlanner = makeLocalPlanner(shardKeyMatcherFn) val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) - val expected = """E~TimeScalarGeneratorExec(params = RangeParams(1000,1000,1000), function = Time) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,25,true,false,true,Set(),None,Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""" - validatePlan(execPlan, expected) + execPlan.isInstanceOf[TimeScalarGeneratorExec] shouldEqual(true) } it("should generate Exec plan for Scalar Binary Operation") { @@ -295,22 +244,23 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-2"))))} - val localPlanner = makeLocalPlanner(shardKeyMatcherFn) val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = PromQlQueryParams( """1 + test{_ws_ = \"demo\",_ns_ =~ \"App.*\", instance = \"Inst-1\" }""", 100, 1, 1000))) - val expected = """T~ScalarOperationMapper(operator=ADD, scalarOnLhs=true) - |-FA1~StaticFuncArgs(1.0,RangeParams(1000,1000,1000)) - |-E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1364224076],raw) - |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1364224076],raw) - |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=16, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1364224076],raw) - |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=6, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1364224076],raw) - |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=22, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1364224076],raw)""".stripMargin - validatePlan(execPlan, expected) + execPlan.isInstanceOf[MultiPartitionDistConcatExec] shouldEqual(true) + execPlan.rangeVectorTransformers(0).isInstanceOf[ScalarOperationMapper] shouldEqual true + execPlan.children(0).children.head.isInstanceOf[MultiSchemaPartitionsExec] + + // Child plans should have only inner periodic query in PromQlQueryParams + execPlan.children(1).children.head.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams].promQl shouldEqual + "test{instance=\"Inst-1\",_ws_=\"demo\",_ns_=\"App-1\"}" + execPlan.children(0).children.head.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams].promQl shouldEqual + "test{instance=\"Inst-1\",_ws_=\"demo\",_ns_=\"App-2\"}" + execPlan.children(0).children.head.rangeVectorTransformers(0).isInstanceOf[PeriodicSamplesMapper] shouldEqual true + execPlan.children(1).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", Equals("App-1"))) shouldEqual(true) + execPlan.children(0).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", Equals("App-2"))) shouldEqual(true) } it("should generate Exec plan for Binary join without regex") { @@ -319,7 +269,6 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-2"))))} - val localPlanner = makeLocalPlanner(shardKeyMatcherFn) val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = PromQlQueryParams("""test1{_ws_="demo",_ns_="App"} + test2{_ws_="demo",_ns_="App"}""".stripMargin, 100,1, 1000))) @@ -334,11 +283,8 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-2"))))} - val localPlanner = makeLocalPlanner(shardKeyMatcherFn) val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) -// val expected = """""" -// validatePlan(execPlan, expected) execPlan.isInstanceOf[PartKeysDistConcatExec] shouldEqual (true) } @@ -348,26 +294,32 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-2"))))} - val localPlanner = makeLocalPlanner(shardKeyMatcherFn) val engine = new ShardKeyRegexPlanner( dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) - val expected = """T~InstantVectorFunctionMapper(function=HistogramQuantile) - |-FA1~StaticFuncArgs(0.2,RangeParams(1000,1000,1000)) - |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1000,1000,1000)) - |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1364224076],raw) - |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |----T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1364224076],raw) - |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |----T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=16, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1364224076],raw) - |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |----T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=6, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1364224076],raw) - |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |----T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=22, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1364224076],raw)""".stripMargin - validatePlan(execPlan, expected) + execPlan.isInstanceOf[MultiPartitionReduceAggregateExec] shouldEqual(true) + execPlan.asInstanceOf[MultiPartitionReduceAggregateExec].rangeVectorTransformers(0). + isInstanceOf[AggregatePresenter] shouldEqual true + execPlan.asInstanceOf[MultiPartitionReduceAggregateExec].rangeVectorTransformers(1). + isInstanceOf[InstantVectorFunctionMapper] shouldEqual true + + execPlan.asInstanceOf[MultiPartitionReduceAggregateExec].rangeVectorTransformers(1). + asInstanceOf[InstantVectorFunctionMapper].function shouldEqual HistogramQuantile + execPlan.children(0).children.head.isInstanceOf[MultiSchemaPartitionsExec] + + // Plan for each map should not have histogram quantile + execPlan.children(0).children.head.rangeVectorTransformers.length shouldEqual 2 + execPlan.children(0).children.head.rangeVectorTransformers(0).isInstanceOf[PeriodicSamplesMapper] shouldEqual true + execPlan.children(0).children.head.rangeVectorTransformers(1).isInstanceOf[AggregateMapReduce] shouldEqual true + execPlan.children(1).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", Equals("App-1"))) shouldEqual(true) + execPlan.children(0).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", Equals("App-2"))) shouldEqual(true) + + // Child plans should have only sum query in PromQlQueryParams + execPlan.children(1).children.head.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams].promQl shouldEqual + """sum(test{_ws_="demo",_ns_="App-1"})""" + execPlan.children(0).children.head.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams].promQl shouldEqual + """sum(test{_ws_="demo",_ns_="App-2"})""" } it("should generate Exec plan for exp for Aggregate query") { @@ -376,40 +328,44 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-2"))))} - val localPlanner = makeLocalPlanner(shardKeyMatcherFn) val engine = new ShardKeyRegexPlanner( dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) - val expected = """T~InstantVectorFunctionMapper(function=Exp) - |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1000,1000,1000)) - |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1364224076],raw) - |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |----T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1364224076],raw) - |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |----T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=16, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1364224076],raw) - |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |----T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=6, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1364224076],raw) - |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |----T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=22, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1364224076],raw)""".stripMargin - validatePlan(execPlan, expected) + execPlan.isInstanceOf[MultiPartitionReduceAggregateExec] shouldEqual(true) + execPlan.asInstanceOf[MultiPartitionReduceAggregateExec].rangeVectorTransformers(0). + isInstanceOf[AggregatePresenter] shouldEqual true + execPlan.asInstanceOf[MultiPartitionReduceAggregateExec].rangeVectorTransformers(1). + isInstanceOf[InstantVectorFunctionMapper] shouldEqual true + + execPlan.asInstanceOf[MultiPartitionReduceAggregateExec].rangeVectorTransformers(1). + asInstanceOf[InstantVectorFunctionMapper].function shouldEqual Exp + execPlan.children(0).children.head.isInstanceOf[MultiSchemaPartitionsExec] + + // Plan for each map should not have exp + execPlan.children(0).children.head.rangeVectorTransformers.length shouldEqual 2 + execPlan.children(0).children.head.rangeVectorTransformers(0).isInstanceOf[PeriodicSamplesMapper] shouldEqual true + execPlan.children(0).children.head.rangeVectorTransformers(1).isInstanceOf[AggregateMapReduce] shouldEqual true + execPlan.children(1).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", Equals("App-1"))) shouldEqual(true) + execPlan.children(0).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", Equals("App-2"))) shouldEqual(true) + + // Child plans should have only sum query in PromQlQueryParams + execPlan.children(1).children.head.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams].promQl shouldEqual + """sum(test{_ws_="demo",_ns_="App-1"})""" + execPlan.children(0).children.head.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams].promQl shouldEqual + """sum(test{_ws_="demo",_ns_="App-2"})""" } it("should generate local Exec plan for query without regex") { val lp = Parser.queryToLogicalPlan("""test{_ws_ = "demo", _ns_ = "App-1" }""", 1000, 1000) val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-1"))))} - val localPlanner = makeLocalPlanner(shardKeyMatcherFn) val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) - val expected = """E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-882843538],raw) - |-T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |--E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(App-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-882843538],raw) - |-T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |--E~MultiSchemaPartitionsExec(dataset=timeseries, shard=16, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(App-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-882843538],raw)""".stripMargin - validatePlan(execPlan, expected) + execPlan.isInstanceOf[LocalPartitionDistConcatExec] shouldEqual(true) + execPlan.children(0).isInstanceOf[MultiSchemaPartitionsExec] + execPlan.children(0).asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", Equals("App-1"))) shouldEqual(true) } it("should generate Exec plan for scalar - time()") { @@ -419,28 +375,30 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-2"))))} - val localPlanner = makeLocalPlanner(shardKeyMatcherFn) val engine = new ShardKeyRegexPlanner( dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) - val expected = """T~ScalarOperationMapper(operator=SUB, scalarOnLhs=true) - |-FA1~ - |-T~ScalarFunctionMapper(function=Scalar, funcParams=List()) - |--E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |---T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |---T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=16, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |---T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=6, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |---T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=22, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-E~TimeScalarGeneratorExec(params = RangeParams(1000,1000,1000), function = Time) on InProcessPlanDispatcher""".stripMargin - validatePlan(execPlan, expected) + execPlan.isInstanceOf[TimeScalarGeneratorExec] shouldEqual(true) + execPlan.rangeVectorTransformers.head.isInstanceOf[ScalarOperationMapper] shouldEqual true + execPlan.rangeVectorTransformers.head.asInstanceOf[ScalarOperationMapper].funcParams.head. + isInstanceOf[ExecPlanFuncArgs] shouldEqual true + execPlan.rangeVectorTransformers.head.asInstanceOf[ScalarOperationMapper].funcParams.head. + isInstanceOf[ExecPlanFuncArgs] shouldEqual true + execPlan.rangeVectorTransformers.head.asInstanceOf[ScalarOperationMapper].funcParams.head. + asInstanceOf[ExecPlanFuncArgs].execPlan.isInstanceOf[MultiPartitionDistConcatExec] shouldEqual true + + val multiPartitionExec = execPlan.rangeVectorTransformers.head.asInstanceOf[ScalarOperationMapper].funcParams.head. + asInstanceOf[ExecPlanFuncArgs].execPlan + multiPartitionExec.rangeVectorTransformers.head.isInstanceOf[ScalarFunctionMapper] shouldEqual true + + // Child plans should have only inner query in PromQlQueryParams + multiPartitionExec.children(1).children.head.queryContext.origQueryParams + .asInstanceOf[PromQlQueryParams].promQl shouldEqual """test{_ws_="demo",_ns_="App-1"}""" + multiPartitionExec.children(0).children.head.queryContext.origQueryParams + .asInstanceOf[PromQlQueryParams].promQl shouldEqual """test{_ws_="demo",_ns_="App-2"}""" } it ("should generate Exec plan for Metadata Label values query") { val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => Seq.empty - val localPlanner = makeLocalPlanner(shardKeyMatcherFn) val engine = new ShardKeyRegexPlanner( dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val lp = Parser.labelValuesQueryToLogicalPlan(Seq("""__metric__"""), Some("""_ws_="demo", _ns_=~".*" """), TimeStepParams(1000, 20, 5000) ) @@ -456,7 +414,6 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture it ("should generate ExecPlan for TsCardinalities") { val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => Nil - val localPlanner = makeLocalPlanner(shardKeyMatcherFn) val engine = new ShardKeyRegexPlanner( dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val lp = TsCardinalities(Seq("ws_foo", "ns_bar"), 3) val promQlQueryParams = PromQlQueryParams( @@ -474,29 +431,23 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-2"))))} - val localPlanner = makeLocalPlanner(shardKeyMatcherFn) val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) - val expected = """E~BinaryJoinExec(binaryOp=ADD, on=List(), ignoring=List()) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,25,true,false,true,Set(),None,Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-2074499967],raw) - |--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(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(_metric_,Equals(test1))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-2074499967],raw) - |--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(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(_metric_,Equals(test1))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-2074499967],raw) - |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=11, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(_metric_,Equals(test1))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-2074499967],raw) - |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=27, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(_metric_,Equals(test1))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-2074499967],raw) - |-E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-2074499967],raw) - |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=14, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(_metric_,Equals(test2))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-2074499967],raw) - |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=30, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(_metric_,Equals(test2))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-2074499967],raw) - |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=4, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(_metric_,Equals(test2))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-2074499967],raw) - |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=20, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(_metric_,Equals(test2))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-2074499967],raw)""".stripMargin - validatePlan(execPlan, expected) + execPlan.isInstanceOf[BinaryJoinExec] shouldEqual(true) + execPlan.children(0).isInstanceOf[MultiPartitionDistConcatExec] shouldEqual(true) + val lhs = execPlan.children(0).asInstanceOf[MultiPartitionDistConcatExec] + lhs.children.length shouldEqual 2 + lhs.children(0).children.head.isInstanceOf[MultiSchemaPartitionsExec] shouldEqual true + lhs.children(1).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", Equals("App-1"))) shouldEqual(true) + lhs.children(0).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", Equals("App-2"))) shouldEqual(true) + val rhs = execPlan.children(1).asInstanceOf[MultiPartitionDistConcatExec] + rhs.children(0).children.head.isInstanceOf[MultiSchemaPartitionsExec] shouldEqual true + rhs.children(1).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", Equals("App-1"))) shouldEqual(true) + rhs.children(0).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", Equals("App-2"))) shouldEqual(true) } @@ -509,31 +460,23 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-2"))))} - val localPlanner = makeLocalPlanner(shardKeyMatcherFn) val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) - val expected = """E~BinaryJoinExec(binaryOp=ADD, on=List(), ignoring=List()) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,25,true,false,true,Set(),None,Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-T~PeriodicSamplesMapper(start=1000000, step=0, end=1000000, window=Some(300000), functionId=Some(AvgOverTime), rawSource=false, offsetMs=None) - |--E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1923678294],raw) - |---T~PeriodicSamplesMapper(start=720000, step=60000, end=960000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(420000,960000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1923678294],raw) - |---T~PeriodicSamplesMapper(start=720000, step=60000, end=960000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=16, chunkMethod=TimeRangeChunkScan(420000,960000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1923678294],raw) - |---T~PeriodicSamplesMapper(start=720000, step=60000, end=960000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=6, chunkMethod=TimeRangeChunkScan(420000,960000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1923678294],raw) - |---T~PeriodicSamplesMapper(start=720000, step=60000, end=960000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=22, chunkMethod=TimeRangeChunkScan(420000,960000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1923678294],raw) - |-T~PeriodicSamplesMapper(start=1000000, step=0, end=1000000, window=Some(300000), functionId=Some(AvgOverTime), rawSource=false, offsetMs=None) - |--E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1923678294],raw) - |---T~PeriodicSamplesMapper(start=720000, step=60000, end=960000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(420000,960000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1923678294],raw) - |---T~PeriodicSamplesMapper(start=720000, step=60000, end=960000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=16, chunkMethod=TimeRangeChunkScan(420000,960000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1923678294],raw) - |---T~PeriodicSamplesMapper(start=720000, step=60000, end=960000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=6, chunkMethod=TimeRangeChunkScan(420000,960000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1923678294],raw) - |---T~PeriodicSamplesMapper(start=720000, step=60000, end=960000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=22, chunkMethod=TimeRangeChunkScan(420000,960000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1923678294],raw)""".stripMargin - validatePlan(execPlan, expected) + execPlan.isInstanceOf[BinaryJoinExec] shouldEqual(true) + execPlan.children(0).isInstanceOf[MultiPartitionDistConcatExec] shouldEqual(true) + val lhs = execPlan.children(0).asInstanceOf[MultiPartitionDistConcatExec] + lhs.children.length shouldEqual 2 + lhs.children(0).children.head.isInstanceOf[MultiSchemaPartitionsExec] shouldEqual true + lhs.children(1).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", Equals("App-1"))) shouldEqual(true) + lhs.children(0).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", Equals("App-2"))) shouldEqual(true) + val rhs = execPlan.children(1).asInstanceOf[MultiPartitionDistConcatExec] + rhs.children(0).children.head.isInstanceOf[MultiSchemaPartitionsExec] shouldEqual true + rhs.children(1).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", Equals("App-1"))) shouldEqual(true) + rhs.children(0).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", Equals("App-2"))) shouldEqual(true) } it("should generate Exec plan for Binary join with regex only on one side") { @@ -545,25 +488,19 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-2")))) } - val localPlanner = makeLocalPlanner(shardKeyMatcherFn) val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) - val expected = """E~BinaryJoinExec(binaryOp=ADD, on=List(), ignoring=List()) on InProcessPlanDispatcher - |-E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=10, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(App-0)), ColumnFilter(_metric_,Equals(test1))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=26, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(App-0)), ColumnFilter(_metric_,Equals(test1))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=14, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(_metric_,Equals(test2))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=30, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(_metric_,Equals(test2))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=4, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(_metric_,Equals(test2))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=20, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(_metric_,Equals(test2))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw)""".stripMargin - validatePlan(execPlan, expected) + execPlan.isInstanceOf[BinaryJoinExec] shouldEqual (true) + execPlan.children(0).isInstanceOf[LocalPartitionDistConcatExec] shouldEqual (true) + val lhs = execPlan.children(0).asInstanceOf[LocalPartitionDistConcatExec] + lhs.children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", Equals("App-0"))) shouldEqual (true) + val rhs = execPlan.children(1).asInstanceOf[MultiPartitionDistConcatExec] + rhs.children(0).children.head.isInstanceOf[MultiSchemaPartitionsExec] shouldEqual true + rhs.children(1).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", Equals("App-1"))) shouldEqual (true) + rhs.children(0).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", Equals("App-2"))) shouldEqual (true) } it("should generate Exec plan for topk query with single matching value for regex") { @@ -573,18 +510,9 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture Seq(Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-1")))) } - val localPlanner = makeLocalPlanner(shardKeyMatcherFn) val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) - val expected = """T~AggregatePresenter(aggrOp=TopK, aggrParams=List(2.0), rangeParams=RangeParams(1000,1000,1000)) - |-E~LocalPartitionReduceAggregateExec(aggrOp=TopK, aggrParams=List(2.0)) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--T~AggregateMapReduce(aggrOp=TopK, aggrParams=List(2.0), without=List(), by=List()) - |---T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--T~AggregateMapReduce(aggrOp=TopK, aggrParams=List(2.0), without=List(), by=List()) - |---T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=16, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw)""".stripMargin - validatePlan(execPlan, expected) + execPlan.isInstanceOf[LocalPartitionReduceAggregateExec] shouldEqual (true) } it("should support for topk query with multiple matching values for regex if they are all local") { @@ -596,24 +524,9 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-2")))) } - val localPlanner = makeLocalPlanner(shardKeyMatcherFn) val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) - val expected = """T~AggregatePresenter(aggrOp=TopK, aggrParams=List(2.0), rangeParams=RangeParams(1000,1000,1000)) - |-E~LocalPartitionReduceAggregateExec(aggrOp=TopK, aggrParams=List(2.0)) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#143127322],raw) - |--T~AggregateMapReduce(aggrOp=TopK, aggrParams=List(2.0), without=List(), by=List()) - |---T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#143127322],raw) - |--T~AggregateMapReduce(aggrOp=TopK, aggrParams=List(2.0), without=List(), by=List()) - |---T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=16, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#143127322],raw) - |--T~AggregateMapReduce(aggrOp=TopK, aggrParams=List(2.0), without=List(), by=List()) - |---T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=6, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#143127322],raw) - |--T~AggregateMapReduce(aggrOp=TopK, aggrParams=List(2.0), without=List(), by=List()) - |---T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=22, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#143127322],raw)""".stripMargin - validatePlan(execPlan, expected) + execPlan.isInstanceOf[MultiPartitionReduceAggregateExec] shouldEqual true } @@ -624,20 +537,22 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture Seq(Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-1")))) } - val localPlanner = makeLocalPlanner(shardKeyMatcherFn) val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) - val expected = """T~InstantVectorFunctionMapper(function=HistogramQuantile) - |-FA1~StaticFuncArgs(0.2,RangeParams(1000,1000,1000)) - |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1000,1000,1000)) - |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |----T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |----T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=16, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw)""".stripMargin - validatePlan(execPlan, expected) + execPlan.isInstanceOf[LocalPartitionReduceAggregateExec] shouldEqual (true) + execPlan.asInstanceOf[LocalPartitionReduceAggregateExec].rangeVectorTransformers(0). + isInstanceOf[AggregatePresenter] shouldEqual true + execPlan.asInstanceOf[LocalPartitionReduceAggregateExec].rangeVectorTransformers(1). + isInstanceOf[InstantVectorFunctionMapper] shouldEqual true + + execPlan.asInstanceOf[LocalPartitionReduceAggregateExec].rangeVectorTransformers(1). + asInstanceOf[InstantVectorFunctionMapper].function shouldEqual HistogramQuantile + execPlan.children.head.isInstanceOf[MultiSchemaPartitionsExec] + + execPlan.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams].promQl shouldEqual + """histogram_quantile(0.2,sum(test{_ws_="demo",_ns_="App-1"}))""" + + execPlan.queryContext.plannerParams.skipAggregatePresent shouldEqual (false) } it("should generate Exec plan for Binary join with single regex match") { @@ -648,19 +563,11 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture Seq(Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App")))) } - val localPlanner = makeLocalPlanner(shardKeyMatcherFn) val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) - val expected = """E~BinaryJoinExec(binaryOp=ADD, on=List(), ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |--E~MultiSchemaPartitionsExec(dataset=timeseries, shard=11, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App)), ColumnFilter(_metric_,Equals(test1))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |--E~MultiSchemaPartitionsExec(dataset=timeseries, shard=27, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App)), ColumnFilter(_metric_,Equals(test1))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |--E~MultiSchemaPartitionsExec(dataset=timeseries, shard=4, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App)), ColumnFilter(_metric_,Equals(test2))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |--E~MultiSchemaPartitionsExec(dataset=timeseries, shard=20, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App)), ColumnFilter(_metric_,Equals(test2))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw)""".stripMargin - validatePlan(execPlan, expected) + execPlan.isInstanceOf[BinaryJoinExec] shouldEqual (true) + execPlan.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams].promQl shouldEqual + ("""(test1{_ws_="demo",_ns_="App"} + test2{_ws_="demo",_ns_="App"})""") } it("should preserve brackets in Binary join with regex query") { @@ -671,37 +578,11 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture Seq(Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App")))) } - val localPlanner = makeLocalPlanner(shardKeyMatcherFn) val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) - val expected = """E~BinaryJoinExec(binaryOp=DIV, on=List(), ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-2041966083],raw) - |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1000,1000,1000)) - |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-2041966083],raw) - |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |----T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=11, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App)), ColumnFilter(_metric_,Equals(test1))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-2041966083],raw) - |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |----T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=27, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App)), ColumnFilter(_metric_,Equals(test1))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-2041966083],raw) - |-E~BinaryJoinExec(binaryOp=ADD, on=List(), ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-2041966083],raw) - |--T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1000,1000,1000)) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-2041966083],raw) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |-----T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=4, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App)), ColumnFilter(_metric_,Equals(test2))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-2041966083],raw) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |-----T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=20, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App)), ColumnFilter(_metric_,Equals(test2))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-2041966083],raw) - |--T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1000,1000,1000)) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-2041966083],raw) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |-----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(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App)), ColumnFilter(_metric_,Equals(test3))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-2041966083],raw) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |-----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(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App)), ColumnFilter(_metric_,Equals(test3))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-2041966083],raw)""".stripMargin - validatePlan(execPlan, expected) - + execPlan.isInstanceOf[BinaryJoinExec] shouldEqual (true) + execPlan.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams].promQl shouldEqual + ("""(sum(test1{_ws_="demo",_ns_="App"}) / (sum(test2{_ws_="demo",_ns_="App"}) + sum(test3{_ws_="demo",_ns_="App"})))""") } it("should handle push down aggregation to right level case 1") { @@ -716,27 +597,28 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture ) ) } - val localPlanner = makeLocalPlanner(shardKeyMatcherFn) val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) - val expected = """T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1000,1000,1000)) - |-E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |---T~AggregatePresenter(aggrOp=Count, aggrParams=List(), rangeParams=RangeParams(1000,1000,1000)) - |----E~LocalPartitionReduceAggregateExec(aggrOp=Count, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-----T~AggregateMapReduce(aggrOp=Count, aggrParams=List(), without=List(), by=List(foo)) - |------T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=4, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App-.*)), ColumnFilter(_metric_,Equals(test1))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-----T~AggregateMapReduce(aggrOp=Count, aggrParams=List(), without=List(), by=List(foo)) - |------T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=20, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App-.*)), ColumnFilter(_metric_,Equals(test1))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-----T~AggregateMapReduce(aggrOp=Count, aggrParams=List(), without=List(), by=List(foo)) - |------T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=8, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App-.*)), ColumnFilter(_metric_,Equals(test1))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-----T~AggregateMapReduce(aggrOp=Count, aggrParams=List(), without=List(), by=List(foo)) - |------T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=24, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App-.*)), ColumnFilter(_metric_,Equals(test1))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw)""".stripMargin - validatePlan(execPlan, expected) + execPlan.isInstanceOf[LocalPartitionReduceAggregateExec] shouldEqual true + execPlan.asInstanceOf[LocalPartitionReduceAggregateExec].aggrOp shouldEqual Sum + val mpExec = execPlan.children.head + mpExec.isInstanceOf[MultiPartitionReduceAggregateExec] shouldEqual true + mpExec.asInstanceOf[MultiPartitionReduceAggregateExec].aggrOp shouldEqual Count + mpExec.rangeVectorTransformers.find(_.isInstanceOf[AggregateMapReduce]) match { + case Some(AggregateMapReduce(op, _, _, _)) => op shouldEqual Sum + case _ => fail("Expected AggregateMapReduce for the sum operation") + } + mpExec.children match { + case children: Seq[ExecPlan] if children.size == 2 => + val plan1 = children(0) + val plan2 = children(1) + (plan2.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams].promQl :: + plan1.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams].promQl :: Nil toSet) shouldEqual + Set("""count(test1{_ws_="demo",_ns_="App2"}) by (foo)""", + """count(test1{_ws_="demo",_ns_="App1"}) by (foo)""") + plan1.isInstanceOf[LocalPartitionReduceAggregateExec] shouldEqual true + case _ => fail("Expected two children") + } } @@ -749,21 +631,13 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture ColumnFilter("_ns_", Equals("App1"))) ) } - val localPlanner = makeLocalPlanner(shardKeyMatcherFn) val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) - val expected = """T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1000,1000,1000)) - |-E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#572058289],raw) - |--T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |---T~AggregatePresenter(aggrOp=Count, aggrParams=List(), rangeParams=RangeParams(1000,1000,1000)) - |----E~LocalPartitionReduceAggregateExec(aggrOp=Count, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#572058289],raw) - |-----T~AggregateMapReduce(aggrOp=Count, aggrParams=List(), without=List(), by=List(foo)) - |------T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=4, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App-.*)), ColumnFilter(_metric_,Equals(test1))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#572058289],raw) - |-----T~AggregateMapReduce(aggrOp=Count, aggrParams=List(), without=List(), by=List(foo)) - |------T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=20, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App-.*)), ColumnFilter(_metric_,Equals(test1))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#572058289],raw)""".stripMargin - validatePlan(execPlan, expected) + // Since we resolve to just one namespace, the entire plan should be materialized by the wrapped planner + execPlan.isInstanceOf[LocalPartitionReduceAggregateExec] shouldEqual true + execPlan.dispatcher.isInstanceOf[ActorPlanDispatcher] shouldEqual true + execPlan.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams].promQl shouldEqual + """sum(count(test1{_ws_="demo",_ns_="App1"}) by (foo))""" } it("should handle multiple binary joins appropriately, case 1") { @@ -779,21 +653,15 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture ColumnFilter("_ns_", Equals("App1"))) ) } - val localPlanner = makeLocalPlanner(shardKeyMatcherFn) val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext( origQueryParams = PromQlQueryParams("""test1{_ws_="demo",_ns_="App-1"} + test2{_ws_="demo",_ns_="App-1"}""", 100, 1, 1000))) - val expected = """E~BinaryJoinExec(binaryOp=ADD, on=List(), ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-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(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(App-1)), ColumnFilter(_metric_,Equals(test1))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-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(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(App-1)), ColumnFilter(_metric_,Equals(test1))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |--E~MultiSchemaPartitionsExec(dataset=timeseries, shard=14, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(App-1)), ColumnFilter(_metric_,Equals(test2))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |--E~MultiSchemaPartitionsExec(dataset=timeseries, shard=30, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(App-1)), ColumnFilter(_metric_,Equals(test2))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw)""".stripMargin - validatePlan(execPlan, expected) + execPlan.isInstanceOf[BinaryJoinExec] shouldEqual (true) + // Since the dispatcher is ActorDispatcher, its materialized by the underlying queryPlanner + execPlan.dispatcher.isInstanceOf[ActorPlanDispatcher] shouldEqual (true) + execPlan.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams].promQl shouldEqual + ("""test1{_ws_="demo",_ns_="App-1"} + test2{_ws_="demo",_ns_="App-1"}""") } @@ -808,25 +676,24 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => Seq(shardColumnFilters) // Top level plan to be done in-process, both left & right operations will be executed by the wrapped query planner - val localPlanner = makeLocalPlanner(shardKeyMatcherFn) val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) - val expected = """T~InstantVectorFunctionMapper(function=Ln) - |-E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=4, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(_metric_,Equals(test1))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=20, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(_metric_,Equals(test1))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=8, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(_metric_,Equals(test1))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=24, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(_metric_,Equals(test1))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw)""".stripMargin + execPlan.isInstanceOf[BinaryJoinExec] shouldEqual true + val inJoin = execPlan.asInstanceOf[BinaryJoinExec] + val (lhs, rhs) = (inJoin.lhs.head, inJoin.rhs.head) + execPlan.asInstanceOf[BinaryJoinExec].binaryOp shouldEqual BinaryOperator.ADD + execPlan.dispatcher.isInstanceOf[ActorPlanDispatcher] shouldEqual true + lhs.isInstanceOf[MultiSchemaPartitionsExec] shouldEqual true + lhs.dispatcher.isInstanceOf[ActorPlanDispatcher] shouldEqual true + rhs.isInstanceOf[BinaryJoinExec] shouldEqual true + rhs.asInstanceOf[BinaryJoinExec].binaryOp shouldEqual BinaryOperator.MUL + rhs.dispatcher.isInstanceOf[ActorPlanDispatcher] shouldEqual true } it("should handle multi partition multi level binary join appropriately with instant fn") { - val lp = Parser.queryToLogicalPlan("""ln(test1{_ws_ = "demo", _ns_ =~ "App.*"})""", 1000, 1000) + val lp = Parser.queryToLogicalPlan("""ln(test1{_ws_ = "demo", _ns_ =~ "App-1"})""", 1000, 1000) val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), @@ -836,20 +703,27 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture ) ) } - val localPlanner = makeLocalPlanner(shardKeyMatcherFn) + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) - val expected = """T~InstantVectorFunctionMapper(function=Ln) - |-E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=4, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(_metric_,Equals(test1))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=20, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(_metric_,Equals(test1))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=8, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(_metric_,Equals(test1))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=24, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App.*)), ColumnFilter(_metric_,Equals(test1))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw)""".stripMargin - validatePlan(execPlan, expected) + execPlan.isInstanceOf[MultiPartitionDistConcatExec] shouldEqual (true) + // Since we get data from multiple partitions, the dispatcher will be in process + execPlan.dispatcher.isInstanceOf[InProcessPlanDispatcher] shouldEqual (true) + execPlan.children match { + case plan1::plan2::Nil => + (plan2.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams].promQl :: + plan1.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams].promQl :: Nil toSet) shouldEqual + Set("""test1{_ws_="demo",_ns_="App1"}""", + """test1{_ws_="demo",_ns_="App2"}""") + plan1.isInstanceOf[LocalPartitionDistConcatExec] shouldEqual true + case _ => fail("Expected two children") + } + execPlan.rangeVectorTransformers.size shouldEqual 1 + val transformer = execPlan.rangeVectorTransformers.head + transformer.isInstanceOf[InstantVectorFunctionMapper] shouldEqual true + transformer.asInstanceOf[InstantVectorFunctionMapper].function shouldEqual InstantFunctionId.Ln + + } @@ -865,45 +739,52 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture ) ) } - val localPlanner = makeLocalPlanner(shardKeyMatcherFn) val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) - val expected = """T~InstantVectorFunctionMapper(function=Ln) - |-E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--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(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App-*)), ColumnFilter(_metric_,Equals(test1))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--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(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App-*)), ColumnFilter(_metric_,Equals(test1))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=11, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App-*)), ColumnFilter(_metric_,Equals(test1))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=27, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App-*)), ColumnFilter(_metric_,Equals(test1))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw)""".stripMargin - validatePlan(execPlan, expected) + execPlan.isInstanceOf[MultiPartitionDistConcatExec] shouldEqual (true) + // Since the dispatcher is ActorDispatcher, its materialized by the underlying queryPlanner + execPlan.dispatcher.isInstanceOf[InProcessPlanDispatcher] shouldEqual (true) + execPlan.rangeVectorTransformers.size shouldEqual 1 + execPlan.rangeVectorTransformers.head.isInstanceOf[InstantVectorFunctionMapper] shouldEqual true + execPlan.rangeVectorTransformers.head.asInstanceOf[InstantVectorFunctionMapper].function shouldEqual + InstantFunctionId.Ln + execPlan.children match { + case plan1::plan2::Nil => + (plan2.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams].promQl :: + plan1.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams].promQl :: Nil toSet) shouldEqual + Set("""test1{_ws_="demo",_ns_="App-1"}""", + """test1{_ws_="demo",_ns_="App-2"}""") + plan1.isInstanceOf[LocalPartitionDistConcatExec] shouldEqual true + case _ => fail("Expected two children") + } } it("should handle simple instant function applied to single partition raw series w/o regex") { val lp = Parser.queryToLogicalPlan("""ln(test1{_ws_ = "demo", _ns_ = "App-1"})""", 1000, 1000) val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => Seq(shardColumnFilters) - val localPlanner = makeLocalPlanner(shardKeyMatcherFn) + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = PromQlQueryParams("""ln(test1{_ws_="demo",_ns_="App-1"})""", 100, 1, 1000))) - val expected = """E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-T~InstantVectorFunctionMapper(function=Ln) - |--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(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(App-1)), ColumnFilter(_metric_,Equals(test1))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-T~InstantVectorFunctionMapper(function=Ln) - |--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(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(App-1)), ColumnFilter(_metric_,Equals(test1))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw)""".stripMargin - validatePlan(execPlan, expected) + execPlan.isInstanceOf[LocalPartitionDistConcatExec] shouldEqual (true) + // Since the dispatcher is ActorDispatcher, its materialized by the underlying queryPlanner + execPlan.dispatcher.isInstanceOf[ActorPlanDispatcher] shouldEqual (true) + execPlan.children.forall( child => { + child.rangeVectorTransformers.find(_.isInstanceOf[InstantVectorFunctionMapper]) match { + case Some(InstantVectorFunctionMapper(Ln, Nil)) => true + case _ => false + } + }) shouldEqual true + execPlan.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams].promQl shouldEqual + ("""ln(test1{_ws_="demo",_ns_="App-1"})""") } it("should handle simple instant function applied to single partition raw series w/o regex and sum") { val lp = Parser.queryToLogicalPlan("""sum(ln(test1{_ws_ = "demo", _ns_ = "App-1"}))""", 1000, 1000) val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => Seq(shardColumnFilters) - val localPlanner = makeLocalPlanner(shardKeyMatcherFn) + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = PromQlQueryParams("""sum(ln(test1{_ws_="demo",_ns_="App-1"}))""", 100, 1, 1000))) @@ -932,20 +813,31 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture ) ) } - val localPlanner = makeLocalPlanner(shardKeyMatcherFn) + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) - val expected = """T~MiscellaneousFunctionMapper(function=LabelJoin, funcParams=List() funcStringParam=List(foo, ,, instance, job)) - |-E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=6, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(Demo)), ColumnFilter(_ns_,EqualsRegex(App*)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=22, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(Demo)), ColumnFilter(_ns_,EqualsRegex(App*)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=12, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(Demo)), ColumnFilter(_ns_,EqualsRegex(App*)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=28, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(Demo)), ColumnFilter(_ns_,EqualsRegex(App*)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw)""".stripMargin - validatePlan(execPlan, expected) + execPlan.isInstanceOf[MultiPartitionDistConcatExec] shouldEqual (true) + execPlan.dispatcher.isInstanceOf[InProcessPlanDispatcher] shouldEqual (true) + execPlan.children.size shouldEqual 2 + execPlan.rangeVectorTransformers.find(_.isInstanceOf[MiscellaneousFunctionMapper]) match { + case Some(x: MiscellaneousFunctionMapper) => x.function shouldEqual MiscellaneousFunctionId.LabelJoin + case _ => fail("Expected to see an MiscellaneousFunctionMapper") + } + execPlan.children.foreach( x => { + x.isInstanceOf[LocalPartitionDistConcatExec] shouldEqual true + x.dispatcher.isInstanceOf[ActorPlanDispatcher] shouldEqual (true) + // TODO: Should the LocalPartitionDistConcatExec have range transformers and just send the required data? + x.rangeVectorTransformers.isEmpty shouldEqual true + }) + execPlan.children match { + case plan1::plan2::Nil => + (plan2.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams].promQl :: + plan1.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams].promQl :: Nil toSet) shouldEqual + Set("""foo{_ws_="demo",_ns_="App-2"}""", + """foo{_ws_="demo",_ns_="App-1"}""") + plan1.isInstanceOf[LocalPartitionDistConcatExec] shouldEqual true + case _ => fail("Expected two children") + } } it("should generate the appropriate plan with sort") { val lp = Parser.queryToLogicalPlan("""sort(foo{_ws_="Demo", _ns_ =~ "App*"}, "foo", ",", "instance", "job")""", @@ -958,20 +850,31 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture ) ) } - val localPlanner = makeLocalPlanner(shardKeyMatcherFn) + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) - val expected = """T~SortFunctionMapper(function=Sort) - |-E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=6, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(Demo)), ColumnFilter(_ns_,EqualsRegex(App*)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=22, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(Demo)), ColumnFilter(_ns_,EqualsRegex(App*)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=12, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(Demo)), ColumnFilter(_ns_,EqualsRegex(App*)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=28, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(Demo)), ColumnFilter(_ns_,EqualsRegex(App*)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw)""".stripMargin - validatePlan(execPlan, expected) + + execPlan.isInstanceOf[MultiPartitionDistConcatExec] shouldEqual (true) + execPlan.dispatcher.isInstanceOf[InProcessPlanDispatcher] shouldEqual (true) + execPlan.children.size shouldEqual 2 + execPlan.rangeVectorTransformers.find(_.isInstanceOf[SortFunctionMapper]) match { + case Some(x: SortFunctionMapper) => x.function shouldEqual SortFunctionId.Sort + case _ => fail("Expected to see an SortFunctionMapper") + } + execPlan.children.foreach( x => { + x.isInstanceOf[LocalPartitionDistConcatExec] shouldEqual true + x.dispatcher.isInstanceOf[ActorPlanDispatcher] shouldEqual (true) + x.rangeVectorTransformers.isEmpty shouldEqual true + }) + execPlan.children match { + case plan1::plan2::Nil => + (plan2.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams].promQl :: + plan1.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams].promQl :: Nil toSet) shouldEqual + Set("""foo{_ws_="demo",_ns_="App-2"}""", + """foo{_ws_="demo",_ns_="App-1"}""") + plan1.isInstanceOf[LocalPartitionDistConcatExec] shouldEqual true + case _ => fail("Expected two children") + } } @@ -986,28 +889,36 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture ) ) } - val localPlanner = makeLocalPlanner(shardKeyMatcherFn) + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) // val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) - val expected = """T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1000,1000,1000)) - |-E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |---T~AbsentFunctionMapper(columnFilter=List(ColumnFilter(_ws_,Equals(Demo)), ColumnFilter(_ns_,EqualsRegex(App*)), ColumnFilter(__name__,Equals(foo))) rangeParams=RangeParams(1000,1000,1000) metricColumn=_metric_) - |----E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(job)) - |------T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=6, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(Demo)), ColumnFilter(_ns_,EqualsRegex(App*)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(job)) - |------T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=22, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(Demo)), ColumnFilter(_ns_,EqualsRegex(App*)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(job)) - |------T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=12, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(Demo)), ColumnFilter(_ns_,EqualsRegex(App*)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(job)) - |------T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=28, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(Demo)), ColumnFilter(_ns_,EqualsRegex(App*)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw)""".stripMargin - validatePlan(execPlan, expected) + + execPlan.isInstanceOf[MultiPartitionReduceAggregateExec] shouldEqual (true) + execPlan.dispatcher.isInstanceOf[InProcessPlanDispatcher] shouldEqual (true) + execPlan.asInstanceOf[ReduceAggregateExec].aggrOp shouldEqual Sum + execPlan.children.size shouldEqual 2 + execPlan.rangeVectorTransformers.find(_.isInstanceOf[AggregatePresenter]) match { + case Some(x: AggregatePresenter) => x.aggrOp shouldEqual Sum + case _ => fail("Expected to see an AggregatePresenter") + } + execPlan.children.foreach( x => { + x.isInstanceOf[LocalPartitionReduceAggregateExec] shouldEqual true + x.dispatcher.isInstanceOf[ActorPlanDispatcher] shouldEqual (true) + x.rangeVectorTransformers.isEmpty shouldEqual true + x.asInstanceOf[ReduceAggregateExec].aggrOp shouldEqual Sum + + }) + execPlan.children match { + case children: Seq[ExecPlan] if children.size == 2 => + val plan1 = children(0) + val plan2 = children(1) + (plan2.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams].promQl :: + plan1.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams].promQl :: Nil toSet) shouldEqual + Set("""sum(absent(foo{_ws_="demo",_ns_="App-2"}))""", + """sum(absent(foo{_ws_="demo",_ns_="App-1"}))""") + case _ => fail("Expected two children") + } } @@ -1022,22 +933,34 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture ) ) } - val localPlanner = makeLocalPlanner(shardKeyMatcherFn) + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) - val expected = """T~AbsentFunctionMapper(columnFilter=List(ColumnFilter(_ws_,Equals(Demo)), ColumnFilter(_ns_,EqualsRegex(App*)), ColumnFilter(__name__,Equals(foo))) rangeParams=RangeParams(1000,1000,1000) metricColumn=_metric_) - |-E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(job)) - |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |----T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=6, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(Demo)), ColumnFilter(_ns_,EqualsRegex(App*)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |----T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=22, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(Demo)), ColumnFilter(_ns_,EqualsRegex(App*)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |----T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=12, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(Demo)), ColumnFilter(_ns_,EqualsRegex(App*)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |----T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=28, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(Demo)), ColumnFilter(_ns_,EqualsRegex(App*)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw)""".stripMargin - validatePlan(execPlan, expected) + + execPlan.isInstanceOf[LocalPartitionReduceAggregateExec] shouldEqual (true) + execPlan.dispatcher.isInstanceOf[InProcessPlanDispatcher] shouldEqual (true) + execPlan.asInstanceOf[ReduceAggregateExec].aggrOp shouldEqual Sum + execPlan.children.size shouldEqual 1 + execPlan.rangeVectorTransformers.find(_.isInstanceOf[AbsentFunctionMapper]) match { + case Some(_: AbsentFunctionMapper) => + case _ => fail("Expected to see an AbsentFunctionMapper") + } + val child = execPlan.children.head + child.isInstanceOf[MultiPartitionDistConcatExec] shouldEqual true + child.dispatcher.isInstanceOf[InProcessPlanDispatcher] shouldEqual (true) + child.rangeVectorTransformers.find(_.isInstanceOf[AggregateMapReduce]) match { + case Some(x: AggregateMapReduce) => x.aggrOp shouldEqual Sum + case _ => fail("Expected to see an AggregateMapReduce") + } + + child.children match { + case plan1::plan2::Nil => + (plan2.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams].promQl :: + plan1.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams].promQl :: Nil toSet) shouldEqual + Set("""foo{_ws_="demo",_ns_="App-2"}""", + """foo{_ws_="demo",_ns_="App-1"}""") + case _ => fail("Expected two children") + } } @@ -1048,41 +971,43 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture Seq(Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-1")))) } - val localPlanner = makeLocalPlanner(shardKeyMatcherFn) + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = PromQlQueryParams("""absent(foo{_ws_="Demo", _ns_ =~ "App-1"} , "foo", "," "instance", "job")""", 100, 1, 1000))) - val expected = """T~AbsentFunctionMapper(columnFilter=List(ColumnFilter(_ws_,Equals(Demo)), ColumnFilter(_ns_,EqualsRegex(App-1)), ColumnFilter(__name__,Equals(foo))) rangeParams=RangeParams(1000,1000,1000) metricColumn=_metric_) - |-E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(job)) - |---T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=6, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(Demo)), ColumnFilter(_ns_,EqualsRegex(App-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(job)) - |---T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=22, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(Demo)), ColumnFilter(_ns_,EqualsRegex(App-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw)""".stripMargin - validatePlan(execPlan, expected) + + execPlan.isInstanceOf[LocalPartitionReduceAggregateExec] shouldEqual (true) + execPlan.dispatcher.isInstanceOf[ActorPlanDispatcher] shouldEqual (true) + execPlan.asInstanceOf[ReduceAggregateExec].aggrOp shouldEqual Sum + execPlan.rangeVectorTransformers.find(_.isInstanceOf[AbsentFunctionMapper]) match { + case Some(x: AbsentFunctionMapper) => + case _ => fail("Expected to see an AggregatePresenter") + } } it("should materialize vector appropriately for single partition") { val lp = Parser.queryToLogicalPlan("""vector(scalar(sum(foo{_ws_="demo", _ns_ = "App-1"})))""", 1000, 1000) val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => Seq(shardColumnFilters) - val localPlanner = makeLocalPlanner(shardKeyMatcherFn) + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = PromQlQueryParams("""vector(scalar(sum(foo{_ws_="demo",_ns_="App-1"})))""", 100, 1, 1000))) - val expected = """T~VectorFunctionMapper(funcParams=List()) - |-T~ScalarFunctionMapper(function=Scalar, funcParams=List()) - |--T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1000,1000,1000)) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |-----T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=6, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(App-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |-----T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=22, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(App-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw)""".stripMargin - validatePlan(execPlan, expected) + + // Aggregate operation will be materialized on the wrapped planner, everything else in process + + execPlan.isInstanceOf[LocalPartitionReduceAggregateExec] shouldEqual (true) + execPlan.dispatcher.isInstanceOf[ActorPlanDispatcher] shouldEqual (true) + execPlan.asInstanceOf[ReduceAggregateExec].aggrOp shouldEqual Sum + execPlan.children.size shouldEqual 2 + execPlan.rangeVectorTransformers.contains(VectorFunctionMapper()) shouldEqual true + execPlan.rangeVectorTransformers.find(_.isInstanceOf[ScalarFunctionMapper]) match { + case Some(x: ScalarFunctionMapper) => + case _ => fail("Expected to see an ScalarFunctionMapper") + } + execPlan.children.head.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams].promQl shouldEqual + """vector(scalar(sum(foo{_ws_="demo",_ns_="App-1"})))""" } it("should materialize vector appropriately for multi partition") { @@ -1096,30 +1021,34 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture ) ) } - val localPlanner = makeLocalPlanner(shardKeyMatcherFn) + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) - val expected = """T~VectorFunctionMapper(funcParams=List()) - |-T~ScalarFunctionMapper(function=Scalar, funcParams=List()) - |--T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1000,1000,1000)) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1837220465],raw) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |-----T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=6, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App*)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1837220465],raw) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |-----T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=22, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App*)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1837220465],raw) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |-----T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=12, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App*)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1837220465],raw) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |-----T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=28, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(App*)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1837220465],raw)""".stripMargin - validatePlan(execPlan, expected) + + // Entre Plan should be materialized on the wrapped + execPlan.isInstanceOf[MultiPartitionReduceAggregateExec] shouldEqual (true) + execPlan.dispatcher.isInstanceOf[InProcessPlanDispatcher] shouldEqual (true) + execPlan.asInstanceOf[ReduceAggregateExec].aggrOp shouldEqual Sum + execPlan.children.size shouldEqual 2 + execPlan.rangeVectorTransformers.contains(VectorFunctionMapper()) shouldEqual true + execPlan.rangeVectorTransformers.find(_.isInstanceOf[ScalarFunctionMapper]) match { + case Some(x: ScalarFunctionMapper) => + case _ => fail("Expected to see an ScalarFunctionMapper") + } + + execPlan.children match { + case children: Seq[ExecPlan] if children.size == 2 => + val plan1 = children(0) + val plan2 = children(1) + (plan2.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams].promQl :: + plan1.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams].promQl :: Nil toSet) shouldEqual + Set("""sum(foo{_ws_="demo",_ns_="App-1"})""", + """sum(foo{_ws_="demo",_ns_="App-2"})""") + case _ => fail("Expected two children") + } } it("should materialize absent function mapper correctly with implicit WS") { - // TODO(a_theimer): need to verify it's correct to remove these tests. { // absent of sum_over_time: _ws_ in query val lp = Parser.queryToLogicalPlan( @@ -1128,25 +1057,23 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ns_", Equals("App-1")), ColumnFilter("_ws_", Equals("demo")))) } - val localPlanner = makeLocalPlanner(shardKeyMatcherFn) val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) execPlan.asInstanceOf[LocalPartitionReduceAggregateExec].rangeVectorTransformers.head .asInstanceOf[AbsentFunctionMapper].columnFilter.isEmpty shouldEqual true } -// { -// // absent of sum_over_time: _ws_ NOT in query -// val lp = Parser.queryToLogicalPlan("absent(sum_over_time(test{_ns_ = \"App-1\", instance = \"Inst-1\" }[5m]))", -// 1000, 1000) -// val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { -// Seq(Seq(ColumnFilter("_ns_", Equals("App-1")), ColumnFilter("_ws_", Equals("demo")))) -// } -// val localPlanner = makeLocalPlanner(shardKeyMatcherFn) -// val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) -// val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) -// execPlan.asInstanceOf[LocalPartitionReduceAggregateExec].rangeVectorTransformers.head -// .asInstanceOf[AbsentFunctionMapper].columnFilter.isEmpty shouldEqual true -// } + { + // absent of sum_over_time: _ws_ NOT in query + val lp = Parser.queryToLogicalPlan("absent(sum_over_time(test{_ns_ = \"App-1\", instance = \"Inst-1\" }[5m]))", + 1000, 1000) + val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { + Seq(Seq(ColumnFilter("_ns_", Equals("App-1")), ColumnFilter("_ws_", Equals("demo")))) + } + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + execPlan.asInstanceOf[LocalPartitionReduceAggregateExec].rangeVectorTransformers.head + .asInstanceOf[AbsentFunctionMapper].columnFilter.isEmpty shouldEqual true + } { // absent: _ws_ in query val lp = Parser.queryToLogicalPlan( @@ -1155,24 +1082,23 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ns_", Equals("App-1")), ColumnFilter("_ws_", Equals("demo")))) } - val localPlanner = makeLocalPlanner(shardKeyMatcherFn) val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) execPlan.asInstanceOf[LocalPartitionReduceAggregateExec].rangeVectorTransformers.head .asInstanceOf[AbsentFunctionMapper].columnFilter.size shouldEqual 4 // _ws_, _ns_, __name__ & instance } -// { -// // absent: _ws_ NOT in query -// val lp = Parser.queryToLogicalPlan("absent(test{_ns_ = \"App-1\", instance = \"Inst-1\" })", -// 1000, 1000) -// val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { -// Seq(Seq(ColumnFilter("_ns_", Equals("App-1")), ColumnFilter("_ws_", Equals("demo")))) -// } -// val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) -// val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) -// execPlan.asInstanceOf[LocalPartitionReduceAggregateExec].rangeVectorTransformers.head -// .asInstanceOf[AbsentFunctionMapper].columnFilter.size shouldEqual 4 // _ws_, _ns_, __name__ & instance -// } + { + // absent: _ws_ NOT in query + val lp = Parser.queryToLogicalPlan("absent(test{_ns_ = \"App-1\", instance = \"Inst-1\" })", + 1000, 1000) + val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { + Seq(Seq(ColumnFilter("_ns_", Equals("App-1")), ColumnFilter("_ws_", Equals("demo")))) + } + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + execPlan.asInstanceOf[LocalPartitionReduceAggregateExec].rangeVectorTransformers.head + .asInstanceOf[AbsentFunctionMapper].columnFilter.size shouldEqual 4 // _ws_, _ns_, __name__ & instance + } { // absent_over_time: _ws_ in query val lp = Parser.queryToLogicalPlan( @@ -1181,27 +1107,25 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ns_", Equals("App-1")), ColumnFilter("_ws_", Equals("demo")))) } - val localPlanner = makeLocalPlanner(shardKeyMatcherFn) val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) execPlan.asInstanceOf[LocalPartitionReduceAggregateExec].rangeVectorTransformers.head .asInstanceOf[AbsentFunctionMapper].columnFilter.size shouldEqual 4 // _ws_, _ns_, __name__ & instance } -// { -// // absent_over_time: _ws_ NOT in query -// val lp = Parser.queryToLogicalPlan("absent_over_time(test{_ns_ = \"App-1\", instance = \"Inst-1\" }[5m])", -// 1000, 1000) -// val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { -// Seq(Seq(ColumnFilter("_ns_", Equals("App-1")), ColumnFilter("_ws_", Equals("demo")))) -// } -// val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) -// val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) -// execPlan.asInstanceOf[LocalPartitionReduceAggregateExec].rangeVectorTransformers.head -// .asInstanceOf[AbsentFunctionMapper].columnFilter.size shouldEqual 4 // _ws_, _ns_, __name__ & instance -// } + { + // absent_over_time: _ws_ NOT in query + val lp = Parser.queryToLogicalPlan("absent_over_time(test{_ns_ = \"App-1\", instance = \"Inst-1\" }[5m])", + 1000, 1000) + val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { + Seq(Seq(ColumnFilter("_ns_", Equals("App-1")), ColumnFilter("_ws_", Equals("demo")))) + } + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + execPlan.asInstanceOf[LocalPartitionReduceAggregateExec].rangeVectorTransformers.head + .asInstanceOf[AbsentFunctionMapper].columnFilter.size shouldEqual 4 // _ws_, _ns_, __name__ & instance + } } - // TODO(a_theimer): need to confirm whether-or-not safe to remove this test. it("should materialize instant functions with args correctly with implicit WS") { // "expected" string is the printTree() of the first child (isInstanceOf[DistConcat] is asserted below-- // other children are identical except for their source shards) @@ -1218,23 +1142,22 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=30, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(App-1)), ColumnFilter(_metric_,Equals(sc_test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1006757749],raw) |-T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) |--E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(App-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1006757749],raw)""".stripMargin), -// // inst func with scalar() arg: _ws_ NOT in query -// ("""clamp_max(test{_ns_="App-1"}, scalar(sc_test{_ns_="App-1"}))""", -// """T~InstantVectorFunctionMapper(function=ClampMax) -// |-FA1~ -// |-T~ScalarFunctionMapper(function=Scalar, funcParams=List()) -// |--E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) -// |---T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) -// |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=14, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(App-1)), ColumnFilter(_metric_,Equals(sc_test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) -// |---T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) -// |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=30, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(App-1)), ColumnFilter(_metric_,Equals(sc_test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) -// |-T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) -// |--E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(App-1)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw)""".stripMargin) + // inst func with scalar() arg: _ws_ NOT in query + ("""clamp_max(test{_ns_="App-1"}, scalar(sc_test{_ns_="App-1"}))""", + """T~InstantVectorFunctionMapper(function=ClampMax) + |-FA1~ + |-T~ScalarFunctionMapper(function=Scalar, funcParams=List()) + |--E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#739940931],raw) + |---T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) + |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=14, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ns_,Equals(App-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_metric_,Equals(sc_test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#739940931],raw) + |---T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) + |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=30, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ns_,Equals(App-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_metric_,Equals(sc_test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#739940931],raw) + |-T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) + |--E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(_ns_,Equals(App-1)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_metric_,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#739940931],raw)""".stripMargin) ) val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ns_", Equals("App-1")), ColumnFilter("_ws_", Equals("demo")))) } - val localPlanner = makeLocalPlanner(shardKeyMatcherFn) val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) queryExpectedPairs.foreach{ case (query, expected) => diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/SingleClusterPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/SingleClusterPlannerSpec.scala index 5703887d04..3d9bad3489 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/SingleClusterPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/SingleClusterPlannerSpec.scala @@ -24,7 +24,6 @@ import filodb.query.LogicalPlan.getRawSeriesFilters import filodb.query.exec.aggregator.{CountRowAggregator, SumRowAggregator} import org.scalatest.exceptions.TestFailedException - import scala.concurrent.duration._ class SingleClusterPlannerSpec extends AnyFunSpec with Matchers with ScalaFutures with PlanValidationSpec { @@ -41,19 +40,11 @@ class SingleClusterPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture private val dsRef = dataset.ref private val schemas = Schemas(dataset.schema) - private def regexPipeShardKeyMatcher(filters: Seq[ColumnFilter]) = { - val values = filters.map { filter => - filter.column -> filter.filter.valuesStrings.toList.head.toString.split('|') - } - val triplets = QueryUtils.combinations(values.map(_._2.toSeq)) - triplets.map(triplet => values.map(_._1).zip(triplet).map(p => ColumnFilter(p._1, Equals(p._2)))) - } - private val config = ConfigFactory.load("application_test.conf") private val queryConfig = QueryConfig(config.getConfig("filodb.query")) private val engine = new SingleClusterPlanner(dataset, schemas, mapperRef, earliestRetainedTimestampFn = 0, - queryConfig, "raw", shardKeyMatcher = regexPipeShardKeyMatcher) + queryConfig, "raw") /* This is the PromQL @@ -382,7 +373,7 @@ class SingleClusterPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture val execPlan = engine.materialize(lp, QueryContext(promQlQueryParams, plannerParams = PlannerParams (spreadOverride = Some(FunctionalSpreadProvider(spread)), targetSchemaProviderOverride = Some(FunctionalTargetSchemaProvider(targetSchema)), queryTimeoutMillis = 1000000))) - println(execPlan) + println(execPlan.children.size) execPlan.rangeVectorTransformers.head.isInstanceOf[StitchRvsMapper] shouldEqual true } @@ -425,7 +416,7 @@ class SingleClusterPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture ColumnFilter("__name__", Equals("name")), ColumnFilter("foo", Equals("abcdefg")), ColumnFilter("bar", Equals("hijklmnop")), - ), qContext, start, end, useTargetSchemaForShards = (filts) => true) + ), qContext, start, end, useTargetSchemaForShards = filters => true) } should contain theSameElementsAs Seq(13, 20, 13, 20) } @@ -1219,7 +1210,6 @@ class SingleClusterPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture } catch { case e: TestFailedException => println(s"Plan validation failed for query: $query") - println(execPlan.printTree()) throw e } } diff --git a/core/src/main/scala/filodb.core/query/QueryUtils.scala b/core/src/main/scala/filodb.core/query/QueryUtils.scala index b6c0438b6d..7cf04a4346 100644 --- a/core/src/main/scala/filodb.core/query/QueryUtils.scala +++ b/core/src/main/scala/filodb.core/query/QueryUtils.scala @@ -1,5 +1,8 @@ package filodb.core.query +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer + /** * Storage for utility functions. */ @@ -23,4 +26,55 @@ object QueryUtils { def containsPipeOnlyRegex(str: String): Boolean = { str.forall(!regexCharsMinusPipe.contains(_)) } + + /** + * Returns all possible sets of elements where exactly one element is + * chosen from each of the argument sequences. + * + * @param choices: all sequences should have at least one element. + * @return ordered sequences; each sequence is ordered such that the element + * at index i is chosen from the ith argument sequence. + */ + def combinations[T](choices: Seq[Seq[T]]): Seq[Seq[T]] = { + val running = new mutable.ArraySeq[T](choices.size) + val result = new mutable.ArrayBuffer[Seq[T]] + def helper(iChoice: Int): Unit = { + if (iChoice == choices.size) { + result.append(Nil ++ running) + return + } + for (choice <- choices(iChoice)) { + running(iChoice) = choice + helper(iChoice + 1) + } + } + helper(0) + result + } + + /** + * Splits a string on unescaped pipe characters. + */ + def splitAtUnescapedPipes(str: String): Seq[String] = { + // match pipes preceded by any count of backslash pairs and either + // some non-backslash character or the beginning of a line. + val regex = "(?<=(^|[^\\\\]))(\\\\\\\\)*(\\|)".r + // get pipe indices -- third capture group + val pipeIndices = regex.findAllMatchIn(str).map(_.start(3)).toSeq + + var offset = 0 + val splits = new ArrayBuffer[String](pipeIndices.size + 1) + var remaining = str + for (i <- pipeIndices) { + // split at the pipe and remove it + val left = remaining.substring(0, i - offset) + val right = remaining.substring(i - offset + 1) + splits.append(left) + remaining = right + // count of all characters before the remaining suffix (+1 to account for pipe) + offset = offset + left.size + 1 + } + splits.append(remaining) + splits + } } diff --git a/core/src/test/scala/filodb.core/query/QueryUtilsSpec.scala b/core/src/test/scala/filodb.core/query/QueryUtilsSpec.scala index 8e5f2039f8..a5b8fb0824 100644 --- a/core/src/test/scala/filodb.core/query/QueryUtilsSpec.scala +++ b/core/src/test/scala/filodb.core/query/QueryUtilsSpec.scala @@ -56,4 +56,18 @@ class QueryUtilsSpec extends AnyFunSpec with Matchers{ QueryUtils.containsPipeOnlyRegex(test) shouldEqual false } } + + it("should correctly split strings at unescaped pipes") { + val tests = Seq( + ("this|is|a|test", Seq("this", "is", "a", "test")), + ("this|is|a||test", Seq("this", "is", "a", "", "test")), + ("this\\|is|a|test", Seq("this\\|is", "a", "test")), + ("this\\\\|is|a|test", Seq("this\\\\", "is", "a", "test")), + ("||this\\|is|\\+a|test||", Seq("", "", "this\\|is", "\\+a", "test", "", "")), + ) + for ((string, expected) <- tests) { + val res = QueryUtils.splitAtUnescapedPipes(string) + res shouldEqual expected + } + } } diff --git a/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala b/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala index 3f079d2093..9f655dff0b 100644 --- a/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala +++ b/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala @@ -991,6 +991,37 @@ class ParserSpec extends AnyFunSpec with Matchers { } } + it ("should correctly enforce filter length limit for EqualsRegex") { + val lim = Parser.REGEX_MAX_LEN + + // true -> should not throw an exception; else false + val succeedQueryPairs = Seq( + (true, s"""foo{job=~"${"a".repeat(lim)}"}"""), + (true, s"""foo{job=~"${".".repeat(lim)}"}"""), + // May want to enforce this (no regex chars, but length limit exceeded). + (true, s"""foo{job=~"${"a".repeat(lim + 1)}"}"""), + (true, s"""foo{job=~"${"a".repeat(lim + 1).split("").mkString("|")}"}"""), + (true, s"""foo{job=~"${"a|".repeat(10) + "a".repeat(lim)}"}"""), + // May want to enforce this (single "|"-joined value exceeds limit). + (true, s"""foo{job=~"${"a|".repeat(10) + "a".repeat(lim + 1)}"}"""), + + (false, s"""foo{job=~"${".".repeat(lim + 1)}"}"""), + (false, s"""foo{job=~"${".".repeat(lim + 1).split("").mkString("|")}"}"""), + (false, s"""foo{job=~"${"a".repeat(lim + 1)}.*"}"""), + (false, s"""foo{job=~"${"a".repeat(lim + 1).split("").mkString("|")}.*"}"""), + ) + for ((shouldSucceed, query) <- succeedQueryPairs) { + val materialize = () => { + Parser.queryToLogicalPlan(query, 1000, 1000) + } + if (shouldSucceed) { + materialize.apply() + } else { + intercept[IllegalArgumentException] {materialize.apply()} + } + } + } + private def printBinaryJoin( lp: LogicalPlan, level: Int = 0) : scala.Unit = { if (!lp.isInstanceOf[BinaryJoin]) { info(s"${" "*level}" + lp.toString)