From 60921cb6bab31358152f22909187f9b3bebf041d Mon Sep 17 00:00:00 2001 From: Alex Theimer Date: Fri, 29 Sep 2023 00:01:08 -0700 Subject: [PATCH 01/27] perf(query): reduce query fanout --- .../queryplanner/LogicalPlanUtils.scala | 46 ++- .../queryplanner/MultiPartitionPlanner.scala | 277 +++++++++--------- .../PartitionLocationPlanner.scala | 104 +++++++ .../queryplanner/QueryUtils.scala | 35 +++ .../queryplanner/ShardKeyRegexPlanner.scala | 97 ++++-- .../queryplanner/SingleClusterPlanner.scala | 259 +++++++++------- .../filodb.core/query/QueryContext.scala | 3 +- 7 files changed, 535 insertions(+), 286 deletions(-) create mode 100644 coordinator/src/main/scala/filodb.coordinator/queryplanner/PartitionLocationPlanner.scala create 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 8a6bcd603e..565493de47 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala @@ -6,7 +6,7 @@ import scala.collection.mutable.ArrayBuffer import com.typesafe.scalalogging.StrictLogging import filodb.core.TargetSchemaProvider -import filodb.core.query.{ColumnFilter, QueryContext, RangeParams} +import filodb.core.query.{ColumnFilter, Filter, QueryContext, RangeParams} import filodb.prometheus.ast.SubqueryUtils import filodb.prometheus.ast.Vectors.PromMetricLabel import filodb.prometheus.ast.WindowConstants @@ -28,17 +28,17 @@ object LogicalPlanUtils extends StrictLogging { } /** - * Given a LogicalPlan check if any descendent of plan is an aggregate operation + * Given a LogicalPlan check if any descendent of plan is either an aggregate or join operation * @param lp the LogicalPlan instance - * @return true if a descendent is an aggregate else false + * @return true if a descendent is an aggregate or join else false */ - def hasDescendantAggregate(lp: LogicalPlan): Boolean = lp match { + def hasDescendantAggregateOrJoin(lp: LogicalPlan): Boolean = lp match { case _: Aggregate => true // consider this BinaryJoin example foo + on(h) + bar. // partition1 has foo{h=1}, bar{h1=2} and partition2 has foo{h=2}, bar{h1=1} // the binary join cannot happen on a partition locally. InProcessPlanDispatcher is required. case _: BinaryJoin => true - case nonLeaf: NonLeafLogicalPlan => nonLeaf.children.exists(hasDescendantAggregate(_)) + case nonLeaf: NonLeafLogicalPlan => nonLeaf.children.exists(hasDescendantAggregateOrJoin(_)) case _ => false } /** @@ -406,6 +406,17 @@ 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 resolveShardKeyFiltersIfNotEquals(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. @@ -413,6 +424,7 @@ 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-- @@ -420,13 +432,16 @@ object LogicalPlanUtils extends StrictLogging { val rsTschemaOpts = LogicalPlan.findLeafLogicalPlans(plan) .filter(_.isInstanceOf[RawSeries]) .map(_.asInstanceOf[RawSeries]).flatMap{ rs => - val shardKeyFilters = getShardKeyFilters(rs) - val interval = LogicalPlanUtils.getSpanningIntervalSelector(rs) - shardKeyFilters.map{ shardKey => - val filters = LogicalPlanUtils.upsertFilters(rs.filters, shardKey) - LogicalPlanUtils.getTargetSchemaIfUnchanging(targetSchemaProvider, filters, interval) + val interval = LogicalPlanUtils.getSpanningIntervalSelector(rs) + val shardKeyFilters = getShardKeyFilters(rs) + .flatMap(LogicalPlanUtils.resolveShardKeyFiltersIfNotEquals(_, 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) + } } - } if (rsTschemaOpts.isEmpty) { return None } @@ -446,6 +461,7 @@ 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 { @@ -471,7 +487,7 @@ object LogicalPlanUtils extends StrictLogging { // return true // } - val tschema = sameRawSeriesTargetSchemaColumns(plan, targetSchemaProvider, getShardKeyFilters) + val tschema = sameRawSeriesTargetSchemaColumns(plan, targetSchemaProvider, shardKeyMatcher, getShardKeyFilters) if (tschema.isEmpty) { return false } @@ -493,6 +509,7 @@ 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]] = { @@ -524,12 +541,13 @@ 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, nonMetricShardKeyCols, getShardKeyFilters) + canPushdown(bj, targetSchemaProvider, shardKeyMatcher, 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, nonMetricShardKeyCols, getShardKeyFilters) + val canPushdownAgg = + canPushdown(agg, targetSchemaProvider, shardKeyMatcher, 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 f3c7a46d99..a0f83b925e 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala @@ -2,16 +2,13 @@ package filodb.coordinator.queryplanner import java.util.concurrent.ConcurrentHashMap - import scala.collection.concurrent.{Map => ConcurrentMap} import scala.jdk.CollectionConverters._ - import com.typesafe.scalalogging.StrictLogging import io.grpc.ManagedChannel - import filodb.coordinator.queryplanner.LogicalPlanUtils._ import filodb.core.metadata.{Dataset, DatasetOptions, Schemas} -import filodb.core.query.{ColumnFilter, PromQlQueryParams, QueryConfig, QueryContext, RvRange} +import filodb.core.query.{ColumnFilter, PromQlQueryParams, QueryConfig, QueryContext, RangeParams, RvRange} import filodb.grpc.GrpcCommonUtils import filodb.query._ import filodb.query.LogicalPlan._ @@ -49,10 +46,12 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider 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 QueryPlanner with StrictLogging with DefaultPlanner { + extends PartitionLocationPlanner(dataset, partitionLocationProvider, shardKeyMatcher) with StrictLogging { override val schemas: Schemas = Schemas(dataset.schema) override val dsOptions: DatasetOptions = schemas.part.options @@ -96,7 +95,12 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider } else logicalPlan match { case mqp: MetadataQueryPlan => materializeMetadataQueryPlan(mqp, qContext).plans.head case lp: TsCardinalities => materializeTsCardinalities(lp, qContext).plans.head - case _ => walkLogicalPlanTree(logicalPlan, qContext).plans.head + case _ => + val result = walkLogicalPlanTree(logicalPlan, qContext) + if (result.plans.size > 1) { + val dispatcher = PlannerUtil.pickDispatcher(result.plans) + MultiPartitionDistConcatExec(qContext, dispatcher, result.plans) + } else result.plans.head } } @@ -127,18 +131,19 @@ 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, instantTime, stepSecs, instantTime) + generateRemoteExecParamsWithStep(qContext, promQl, instantTime, stepSecs, instantTime) case psp: PeriodicSeriesPlan => val startSecs = psp.startMs / 1000 val stepSecs = psp.stepMs / 1000 val endSecs = psp.endMs / 1000 - generateRemoteExecParamsWithStep(qContext, startSecs, stepSecs, endSecs) + generateRemoteExecParamsWithStep(qContext, promQl, startSecs, stepSecs, endSecs) case _ => - generateRemoteExecParams(qContext, startMs, endMs) + generateRemoteExecParams(qContext, promQl, startMs, endMs) } // Single partition but remote, send the entire plan remotely if (grpcEndpoint.isDefined && !(queryConfig.grpcPartitionsDenyList.contains("*") || @@ -169,110 +174,62 @@ 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 lp: PeriodicSeriesWithWindowing => materializePlanHandleSplitLeaf(lp, qContext) - case _: PeriodicSeries | + case _: BinaryJoin | + _: ApplyInstantFunction | + _: Aggregate | + _: ScalarVectorBinaryOperation | + _: ScalarVaryingDoublePlan | + _: ApplyAbsentFunction | + _: SubqueryWithWindowing | + _: PeriodicSeriesWithWindowing | + _: PeriodicSeries | _: RawChunkMeta | - _: RawSeries => materializePeriodicAndRawSeries(logicalPlan, qContext) + _: RawSeries => materializePlanHandleSplitLeaf(logicalPlan, qContext) } } // scalastyle:on cyclomatic.complexity private def getRoutingKeys(logicalPlan: LogicalPlan) = { - val columnFilterGroup = LogicalPlan.getColumnFilterGroup(logicalPlan) + val shardKeyFilters = LogicalPlan.getNonMetricShardKeyFilters(logicalPlan, dataset.options.nonMetricShardColumns) + val columnFilterGroups = + shardKeyFilters.flatMap(LogicalPlanUtils.resolveShardKeyFiltersIfNotEquals(_, shardKeyMatcher)) val routingKeys = dataset.options.nonMetricShardColumns - .map(x => (x, LogicalPlan.getColumnValues(columnFilterGroup, x))) + .map(x => (x, LogicalPlan.getColumnValues(columnFilterGroups.map(_.toSet), x))) if (routingKeys.flatMap(_._2).isEmpty) Seq.empty else routingKeys.filter(x => x._2.nonEmpty) } - private def generateRemoteExecParams(queryContext: QueryContext, startMs: Long, endMs: Long) = { + private def generateRemoteExecParams(queryContext: QueryContext, promQl: String, startMs: Long, endMs: Long) = { val queryParams = queryContext.origQueryParams.asInstanceOf[PromQlQueryParams] - queryContext.copy(origQueryParams = queryParams.copy(startSecs = startMs/1000, endSecs = endMs / 1000), + queryContext.copy( + origQueryParams = queryParams.copy(promQl = promQl, startSecs = startMs/1000, endSecs = endMs / 1000), plannerParams = queryContext.plannerParams.copy(processMultiPartition = false)) } private def generateRemoteExecParamsWithStep( - queryContext: QueryContext, startSecs: Long, stepSecs: Long, endSecs: Long + queryContext: QueryContext, promQl: String, startSecs: Long, stepSecs: Long, endSecs: Long ) = { val queryParams = queryContext.origQueryParams.asInstanceOf[PromQlQueryParams] queryContext.copy( - origQueryParams = queryParams.copy(startSecs = startSecs, stepSecs = stepSecs, endSecs = endSecs), + origQueryParams = + queryParams.copy(promQl = promQl, startSecs = startSecs, stepSecs = stepSecs, endSecs = endSecs), plannerParams = queryContext.plannerParams.copy(processMultiPartition = false) ) } - /** - * Gets the partition Assignment for the given plan - */ - private def getPartitions(logicalPlan: LogicalPlan, - queryParams: PromQlQueryParams, - infiniteTimeRange: Boolean = false) : Seq[PartitionAssignment] = { - - //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 = leafFilters.filter(_.nonEmpty).map(cf => { - cf.filter(col => nonMetricColumnSet.contains(col.column)).map( - x => (x.column, x.filter.valuesStrings.head.toString)).toMap - }) - - // 3. Determine the query time range - val queryTimeRange = if (infiniteTimeRange) { - TimeRange(0, Long.MaxValue) - } else { - // 3a. Get the start and end time is ms based on the lookback, offset and the user provided start and end time - val (maxOffsetMs, minOffsetMs) = LogicalPlanUtils.getOffsetMillis(logicalPlan) - .foldLeft((Long.MinValue, Long.MaxValue)) { - case ((accMax, accMin), currValue) => (accMax.max(currValue), accMin.min(currValue)) - } - - val periodicSeriesTimeWithOffset = TimeRange((queryParams.startSecs * 1000) - maxOffsetMs, - (queryParams.endSecs * 1000) - minOffsetMs) - val lookBackMs = getLookBackMillis(logicalPlan).max - - //3b Get the Query time range based on user provided range, offsets in previous steps and lookback - TimeRange(periodicSeriesTimeWithOffset.startMs - lookBackMs, - periodicSeriesTimeWithOffset.endMs) - } - - //4. Based on the map in 2 and time range in 5, get the partitions to query - routingKeyMap.flatMap(metricMap => - partitionLocationProvider.getPartitions(metricMap, queryTimeRange)) - } - - /** - * Checks if all the PartitionAssignments belong to same partition - */ - private def isSinglePartition(partitions: Seq[PartitionAssignment]) : Boolean = { - if (partitions.isEmpty) - true - else { - val partName = partitions.head.partitionName - partitions.forall(_.partitionName.equals(partName)) - } - } - /** * * @param logicalPlan Logical plan @@ -294,12 +251,15 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider val queryTimeRange = TimeRange(periodicSeriesTimeWithOffset.startMs - lookBackMs, periodicSeriesTimeWithOffset.endMs) - val partitions = if (routingKeys.isEmpty) List.empty - else { - val routingKeyMap = routingKeys.map(x => (x._1, x._2.head)).toMap - partitionLocationProvider.getPartitions(routingKeyMap, queryTimeRange). - sortBy(_.timeRange.startMs) - } + // create sets of routing-key key->value pairs for all combinations of values, + // then use these sets to determine which partitions own the data. + val keys = routingKeys.map(_._1) + val values = routingKeys.map(_._2.toSeq) + val partitions = QueryUtils.combinations(values) + .map(valueCombo => keys.zip(valueCombo)) + .flatMap(shardKey => partitionLocationProvider.getPartitions(shardKey.toMap, queryTimeRange)) + .distinct + .sortBy(_.timeRange.startMs) if (partitions.isEmpty && routingKeys.nonEmpty) logger.warn(s"No partitions found for routing keys: $routingKeys") @@ -307,47 +267,21 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider } /** - * Materialize all queries except Binary Join and Metadata - */ - def materializePeriodicAndRawSeries(logicalPlan: LogicalPlan, qContext: QueryContext): PlanResult = { + * Materialize plans with leaves that are not split across time. + */ + def materializeNonSplitPeriodicAndRawSeries(logicalPlan: LogicalPlan, qContext: QueryContext): PlanResult = { val queryParams = qContext.origQueryParams.asInstanceOf[PromQlQueryParams] val (partitions, lookBackMs, offsetMs, routingKeys) = resolvePartitionsAndRoutingKeys(logicalPlan, queryParams) val execPlan = if (partitions.isEmpty || routingKeys.forall(_._2.isEmpty)) localPartitionPlanner.materialize(logicalPlan, qContext) else { - 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))) - } + val execPlans = partitions.map(materializeForPartition(logicalPlan, _, qContext)) if (execPlans.size == 1) execPlans.head else { // TODO: Do we pass in QueryContext in LogicalPlan's helper rvRangeForPlan? - StitchRvsExec(qContext, inProcessPlanDispatcher, rvRangeFromPlan(logicalPlan), - execPlans.sortWith((x, _) => !x.isInstanceOf[PromQlRemoteExec])) + MultiPartitionDistConcatExec( + qContext, inProcessPlanDispatcher, execPlans.sortWith((x, _) => !x.isInstanceOf[PromQlRemoteExec])) } - // ^^ Stitch RemoteExec plan results with local using InProcessPlanDispatcher // Sort to move RemoteExec in end as it does not have schema } PlanResult(execPlan:: Nil) @@ -367,10 +301,16 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider val timeRange = timeRangeOverride.getOrElse(TimeRange(1000 * queryParams.startSecs, 1000 * queryParams.endSecs)) val (partitionName, grpcEndpoint) = (partition.partitionName, partition.grpcEndPoint) if (partitionName.equals(localPartitionName)) { - val lpWithUpdatedTime = copyLogicalPlanWithUpdatedTimeRange(logicalPlan, timeRange) - localPartitionPlanner.materialize(lpWithUpdatedTime, queryContext) + // FIXME: subquery tests fail when their time-ranges are updated. + val lpWithUpdatedTime = if (timeRangeOverride.isDefined) { + copyLogicalPlanWithUpdatedTimeRange(logicalPlan, timeRange) + } else logicalPlan + val ctx = queryContext.copy( + origQueryParams = queryParams.copy(promQl = LogicalPlanParser.convertToQuery(logicalPlan))) + localPartitionPlanner.materialize(lpWithUpdatedTime, ctx) } else { - val ctx = generateRemoteExecParams(queryContext, timeRange.startMs, timeRange.endMs) + val promQL = LogicalPlanParser.convertToQuery(logicalPlan) + val ctx = generateRemoteExecParams(queryContext, promQL, timeRange.startMs, timeRange.endMs) if (grpcEndpoint.isDefined && !(queryConfig.grpcPartitionsDenyList.contains("*") || queryConfig.grpcPartitionsDenyList.contains(partitionName.toLowerCase))) { @@ -389,44 +329,95 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider /** * Given a sequence of assignments and a time-range to query, returns (assignment, range) * pairs that describe the time-ranges to be queried for each assignment such that: - * (a) the returned ranges span the argument time-range, and + * (a) the returned ranges span only 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 range the complete time-range. Does not include the offset. + * @param queryRange 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) */ - def getAssignmentQueryRanges(assignments: Seq[PartitionAssignment], range: TimeRange, + def getAssignmentQueryRanges(assignments: Seq[PartitionAssignment], queryRange: TimeRange, lookbackMs: Long = 0L, offsetMs: Long = 0L, stepMsOpt: Option[Long] = None): Seq[(PartitionAssignment, TimeRange)] = { // Construct a sequence of Option[TimeRange]; the ith range is None iff the ith partition has no range to query. // First partition doesn't need its start snapped to a periodic step, so deal with it separately. + val offsetRange = TimeRange(queryRange.startMs - offsetMs, queryRange.endMs - offsetMs) + val filteredAssignments = assignments + // drop until ranges end beyond the first (offset) query timestamp + .dropWhile(range => range.timeRange.endMs < offsetRange.startMs) + // remove ranges smaller than a single lookback + .filter(range => (range.timeRange.endMs - range.timeRange.startMs) >= lookbackMs) + // take ranges until they can't possibly be used to evaluate this query + .takeWhile(range => range.timeRange.startMs < offsetRange.endMs - lookbackMs) + if (filteredAssignments.isEmpty) { + return Nil + } val headRange = { - val partRange = assignments.head.timeRange - Some(TimeRange(math.max(range.startMs, partRange.startMs), - math.min(partRange.endMs + offsetMs, range.endMs))) + val range = filteredAssignments.head.timeRange + Some(TimeRange(math.max(queryRange.startMs, range.startMs), + math.min(range.endMs + offsetMs, queryRange.endMs))) } // Snap remaining range starts to a step (if a step is provided). - val tailRanges = assignments.tail.map { part => + val tailRanges = filteredAssignments.tail.takeWhile(_.timeRange.startMs < queryRange.endMs).map { assign => val startMs = if (stepMsOpt.nonEmpty) { - snapToStep(timestamp = part.timeRange.startMs + lookbackMs + offsetMs, + snapToStep(timestamp = assign.timeRange.startMs + lookbackMs + offsetMs, step = stepMsOpt.get, - origin = range.startMs) + origin = queryRange.startMs) } else { - part.timeRange.startMs + lookbackMs + offsetMs + assign.timeRange.startMs + lookbackMs + offsetMs } - val endMs = math.min(range.endMs, part.timeRange.endMs + offsetMs) + val endMs = math.min(queryRange.endMs, assign.timeRange.endMs + offsetMs) if (startMs <= endMs) { Some(TimeRange(startMs, endMs)) } else None } // Filter out the Nones and flatten the Somes. - (Seq(headRange) ++ tailRanges).zip(assignments).filter(_._1.nonEmpty).map{ case (rangeOpt, part) => + (Seq(headRange) ++ tailRanges).zip(filteredAssignments).filter(_._1.nonEmpty).map{ case (rangeOpt, part) => (part, rangeOpt.get) } } + // FIXME: this is a copy-paste of a method in the ShardKeyRegexPlanner -- + // more evidence that these two classes should be merged. + private def canSupportMultiPartitionCalls(execPlans: Seq[ExecPlan]): Boolean = + execPlans.forall { + case _: PromQlRemoteExec => false + case _ => true + } + + // FIXME: this is a near-exact copy-paste of a method in the ShardKeyRegexPlanner -- + // more evidence that these two classes should be merged. + private def materializeAggregate(aggregate: Aggregate, queryContext: QueryContext): PlanResult = { + val plan = if (LogicalPlanUtils.hasDescendantAggregateOrJoin(aggregate.vectors)) { + val childPlan = materialize(aggregate.vectors, queryContext) + addAggregator(aggregate, queryContext, PlanResult(Seq(childPlan))) + } else { + val queryParams = queryContext.origQueryParams.asInstanceOf[PromQlQueryParams] + val (partitions, _, _, _) = resolvePartitionsAndRoutingKeys(aggregate, queryParams) + val childQueryContext = queryContext.copy( + plannerParams = queryContext.plannerParams.copy(skipAggregatePresent = true)) + val execPlans = partitions.map(p => materializeForPartition(aggregate, p, childQueryContext)) + val exec = if (execPlans.size == 1) execPlans.head + else { + if ((aggregate.operator.equals(AggregationOperator.TopK) + || aggregate.operator.equals(AggregationOperator.BottomK) + || aggregate.operator.equals(AggregationOperator.CountValues) + ) && !canSupportMultiPartitionCalls(execPlans)) + throw new UnsupportedOperationException(s"Shard Key regex not supported for ${aggregate.operator}") + else { + val reducer = MultiPartitionReduceAggregateExec(queryContext, inProcessPlanDispatcher, + execPlans.sortWith((x, _) => !x.isInstanceOf[PromQlRemoteExec]), aggregate.operator, aggregate.params) + reducer.addRangeVectorTransformer(AggregatePresenter(aggregate.operator, aggregate.params, + RangeParams(queryParams.startSecs, queryParams.stepSecs, queryParams.endSecs))) + reducer + } + } + exec + } + PlanResult(Seq(plan)) + } + /** * Materialize any plan whose materialization strategy is governed by whether-or-not it * contains leaves that individually span partitions. @@ -434,19 +425,31 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider private def materializePlanHandleSplitLeaf(logicalPlan: LogicalPlan, qContext: QueryContext): PlanResult = { val qParams = qContext.origQueryParams.asInstanceOf[PromQlQueryParams] - val hasMultiPartitionLeaves = LogicalPlan.findLeafLogicalPlans(logicalPlan) - .exists(getPartitions(_, qParams).size > 1) + // Create one plan per RawSeries/shard-key pair, then resolve its partitions. + // If any resides on more than one partition, the leaf is "split". + 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) if (hasMultiPartitionLeaves) { materializeSplitLeafPlan(logicalPlan, qContext) } else { logicalPlan match { - case agg: Aggregate => super.materializeAggregate(qContext, agg) - case psw: PeriodicSeriesWithWindowing => materializePeriodicAndRawSeries(psw, qContext) + case agg: Aggregate => materializeAggregate(agg, qContext) + case psw: PeriodicSeriesWithWindowing => materializeNonSplitPeriodicAndRawSeries(psw, qContext) case sqw: SubqueryWithWindowing => super.materializeSubqueryWithWindowing(qContext, sqw) case bj: BinaryJoin => materializeMultiPartitionBinaryJoinNoSplitLeaf(bj, qContext) case sv: ScalarVectorBinaryOperation => super.materializeScalarVectorBinOp(qContext, sv) case aif: ApplyInstantFunction => super.materializeApplyInstantFunction(qContext, aif) case svdp: ScalarVaryingDoublePlan => super.materializeScalarPlan(qContext, svdp) case aaf: ApplyAbsentFunction => super.materializeAbsentFunction(qContext, aaf) + case ps: PeriodicSeries => materializeNonSplitPeriodicAndRawSeries(ps, qContext) + case rcm: RawChunkMeta => materializeNonSplitPeriodicAndRawSeries(rcm, qContext) + case rs: RawSeries => materializeNonSplitPeriodicAndRawSeries(rs, qContext) case x => throw new IllegalArgumentException(s"unhandled type: ${x.getClass}") }} } @@ -505,7 +508,9 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider materializeForPartition(logicalPlan, part, newContext) } // stitch if necessary - val resPlan = if (plans.size == 1) { + val resPlan = if (plans.isEmpty) { + EmptyResultExec(qContext, dataset.ref, inProcessPlanDispatcher) + } else if (plans.size == 1) { plans.head } else { // returns NaNs for missing timestamps @@ -641,7 +646,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 new file mode 100644 index 0000000000..d220ab1e88 --- /dev/null +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/PartitionLocationPlanner.scala @@ -0,0 +1,104 @@ +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.query.LogicalPlan + +object PartitionLocationPlanner { + // Can be used as a default matcher function. + 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. + * + * FIXME: the ShardKeyRegexPlanner and MultiPartitionPlanner share purpose/responsibility + * and should eventually be merged. Currently, the SKRP needs getPartitions to group + * resolved shard-keys by partition before it individually materializes each of these + * groups with the MPP. The MPP will again find the corresponding partition for each group + * 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) + extends QueryPlanner with DefaultPlanner { + + private val nonMetricColumnSet = dataset.options.nonMetricShardColumns.toSet + + // scalastyle:off method.length + /** + * Gets the partition Assignment for the given plan + */ + protected def getPartitions(logicalPlan: LogicalPlan, + queryParams: PromQlQueryParams, + infiniteTimeRange: Boolean = false) : Seq[PartitionAssignment] = { + + //1. Get a Seq of all Leaf node filters + val leafFilters = LogicalPlan.getColumnFilterGroup(logicalPlan) + //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 + case _ => throw new IllegalArgumentException( + s"""shard keys must be filtered by equality. filter=${filter}""") + } + (filter.column, value) + }.toMap + } + + // 3. Determine the query time range + val queryTimeRange = if (infiniteTimeRange) { + TimeRange(0, Long.MaxValue) + } else { + // 3a. Get the start and end time is ms based on the lookback, offset and the user provided start and end time + val (maxOffsetMs, minOffsetMs) = LogicalPlanUtils.getOffsetMillis(logicalPlan) + .foldLeft((Long.MinValue, Long.MaxValue)) { + case ((accMax, accMin), currValue) => (accMax.max(currValue), accMin.min(currValue)) + } + + val periodicSeriesTimeWithOffset = TimeRange((queryParams.startSecs * 1000) - maxOffsetMs, + (queryParams.endSecs * 1000) - minOffsetMs) + val lookBackMs = getLookBackMillis(logicalPlan).max + + //3b Get the Query time range based on user provided range, offsets in previous steps and lookback + TimeRange(periodicSeriesTimeWithOffset.startMs - lookBackMs, + periodicSeriesTimeWithOffset.endMs) + } + + //4. Based on the map in 2 and time range in 5, get the partitions to query + routingKeyMap.flatMap(metricMap => + partitionLocationProvider.getPartitions(metricMap, queryTimeRange)) + } + // scalastyle:on method.length + + /** + * Checks if all the PartitionAssignments belong to same partition + */ + protected def isSinglePartition(partitions: Seq[PartitionAssignment]) : Boolean = { + if (partitions.isEmpty) + true + else { + val partName = partitions.head.partitionName + partitions.forall(_.partitionName.equals(partName)) + } + } +} diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/QueryUtils.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/QueryUtils.scala new file mode 100644 index 0000000000..425eeabb00 --- /dev/null +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/QueryUtils.scala @@ -0,0 +1,35 @@ +package filodb.coordinator.queryplanner + +import scala.collection.mutable + +/** + * Storage for miscellaneous utility functions. + */ +object QueryUtils { + /** + * 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 aaf988a459..161f52feaf 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlanner.scala @@ -1,7 +1,5 @@ package filodb.coordinator.queryplanner -import scala.collection.Seq - import filodb.core.{StaticTargetSchemaProvider, TargetSchemaProvider} import filodb.core.metadata.{Dataset, DatasetOptions, Schemas} import filodb.core.query.{ColumnFilter, PromQlQueryParams, QueryConfig, QueryContext, RangeParams} @@ -31,9 +29,10 @@ case class ShardKeyMatcher(columnFilters: Seq[ColumnFilter], query: String) class ShardKeyRegexPlanner(val dataset: Dataset, queryPlanner: QueryPlanner, shardKeyMatcher: Seq[ColumnFilter] => Seq[Seq[ColumnFilter]], + partitionLocationProvider: PartitionLocationProvider, config: QueryConfig, _targetSchemaProvider: TargetSchemaProvider = StaticTargetSchemaProvider()) - extends QueryPlanner with DefaultPlanner { + extends PartitionLocationPlanner(dataset, partitionLocationProvider, shardKeyMatcher) { override def queryConfig: QueryConfig = config override val schemas: Schemas = Schemas(dataset.schema) @@ -59,6 +58,7 @@ class ShardKeyRegexPlanner(val dataset: Dataset, LogicalPlanUtils.getPushdownKeys( plan, targetSchemaProvider(qContext), + shardKeyMatcher, dataset.options.nonMetricShardColumns, getRawShardKeys, rs => getShardKeys(rs)) @@ -71,7 +71,7 @@ class ShardKeyRegexPlanner(val dataset: Dataset, private def attemptPushdown(logicalPlan: LogicalPlan, qContext: QueryContext): Option[PlanResult] = { val pushdownKeys = getPushdownKeys(qContext, logicalPlan) if (pushdownKeys.isDefined) { - val plans = generateExecForEachKey(logicalPlan, pushdownKeys.get.map(_.toSeq).toSeq, qContext) + val plans = generateExec(logicalPlan, pushdownKeys.get.map(_.toSeq).toSeq, qContext) .sortWith((x, _) => !x.isInstanceOf[PromQlRemoteExec]) Some(PlanResult(plans)) } else None @@ -142,6 +142,23 @@ class ShardKeyRegexPlanner(val dataset: Dataset, override def walkLogicalPlanTree(logicalPlan: LogicalPlan, qContext: QueryContext, forceInProcess: Boolean = false): PlanResult = { + // Materialize for each key with the inner planner if: + // - all plan data lies on one partition + // - all RawSeries filters are identical + val qParams = qContext.origQueryParams.asInstanceOf[PromQlQueryParams] + val shardKeyFilterGroups = LogicalPlan.getRawSeriesFilters(logicalPlan) + .map(_.filter(cf => dataset.options.nonMetricShardColumns.contains(cf.column))) + val headFilters = shardKeyFilterGroups.headOption.map(_.toSet) + // Note: unchecked .get is OK here since it will only be called for each tail element. + val sameFilters = shardKeyFilterGroups.tail.forall(_.toSet == headFilters.get) + val partitions = getShardKeys(logicalPlan) + .flatMap(filters => getPartitions(logicalPlan.replaceFilters(filters), qParams)) + if (partitions.isEmpty) { + return PlanResult(Seq(queryPlanner.materialize(logicalPlan, qContext))) + } else if (sameFilters && isSinglePartition(partitions)) { + val plans = generateExec(logicalPlan, getShardKeys(logicalPlan), qContext) + return PlanResult(plans) + } logicalPlan match { case lp: ApplyMiscellaneousFunction => materializeApplyMiscellaneousFunction(qContext, lp) case lp: ApplyInstantFunction => materializeApplyInstantFunction(qContext, lp) @@ -162,9 +179,34 @@ class ShardKeyRegexPlanner(val dataset: Dataset, private def generateExecWithoutRegex(logicalPlan: LogicalPlan, nonMetricShardKeyFilters: Seq[ColumnFilter], qContext: QueryContext): Seq[ExecPlan] = { val shardKeyMatches = shardKeyMatcher(nonMetricShardKeyFilters) - generateExecForEachKey(logicalPlan, shardKeyMatches, qContext) + generateExec(logicalPlan, shardKeyMatches, qContext) + } + + private def updateQueryParams(logicalPlan: LogicalPlan, queryParams: PromQlQueryParams): PromQlQueryParams = { + val promQl = LogicalPlanParser.convertToQuery(logicalPlan) + logicalPlan match { + case tls: TopLevelSubquery => { + val instantTime = queryParams.startSecs + queryParams.copy( + promQl = promQl, + startSecs = instantTime, + endSecs = instantTime + ) + } + case psp: PeriodicSeriesPlan => { + queryParams.copy( + promQl = promQl, + startSecs = psp.startMs / 1000, + endSecs = psp.endMs / 1000, + stepSecs = psp.stepMs / 1000 + ) + } + case _ => queryParams.copy(promQl = promQl) + } } + // This will be deprecated to reduce query fanout. + @Deprecated private def generateExecForEachKey(logicalPlan: LogicalPlan, keys: Seq[Seq[ColumnFilter]], qContext: QueryContext): Seq[ExecPlan] = { @@ -175,31 +217,36 @@ class ShardKeyRegexPlanner(val dataset: Dataset, // 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 = logicalPlan match { - case tls: TopLevelSubquery => { - val instantTime = queryParams.startSecs - queryParams.copy( - promQl = LogicalPlanParser.convertToQuery(newLogicalPlan), - startSecs = instantTime, - endSecs = instantTime - ) - } - case psp: PeriodicSeriesPlan => { - queryParams.copy( - promQl = LogicalPlanParser.convertToQuery(newLogicalPlan), - startSecs = psp.startMs / 1000, - endSecs = psp.endMs / 1000, - stepSecs = psp.stepMs / 1000 - ) - } - case _ => queryParams.copy(promQl = LogicalPlanParser.convertToQuery(newLogicalPlan)) - } + val newQueryParams = updateQueryParams(newLogicalPlan, queryParams) val newQueryContext = qContext.copy(origQueryParams = newQueryParams, plannerParams = qContext.plannerParams. copy(skipAggregatePresent = skipAggregatePresentValue)) queryPlanner.materialize(newLogicalPlan, newQueryContext) } } + private def generateExec(logicalPlan: LogicalPlan, + keys: Seq[Seq[ColumnFilter]], + qContext: QueryContext): Seq[ExecPlan] = { + // Create one plan per RawSeries/shard-key pair, then resolve its partitions. + // If any resides on more than one partition, the leaf is "split". + val qParams = qContext.origQueryParams.asInstanceOf[PromQlQueryParams] + lazy 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) + if (qContext.plannerParams.reduceShardKeyRegexFanout && !hasMultiPartitionLeaves) { + val newQueryParams = updateQueryParams(logicalPlan, qParams) + val newQueryContext = qContext.copy(origQueryParams = newQueryParams) + Seq(queryPlanner.materialize(logicalPlan, newQueryContext)) + } else { + generateExecForEachKey(logicalPlan, keys, qContext) + } + } + /** * For binary join queries like test1{_ws_ = "demo", _ns_ =~ "App.*"} + test2{_ws_ = "demo", _ns_ =~ "App.*"}) * LHS and RHS could be across multiple partitions @@ -261,7 +308,7 @@ class ShardKeyRegexPlanner(val dataset: Dataset, // locally and then run count on the results. The following implementation checks for descendant aggregates, if // there are any, the provided aggregation needs to be done using inProcess, else we can materialize the aggregate // using the wrapped planner - val plan = if (LogicalPlanUtils.hasDescendantAggregate(aggregate.vectors)) { + val plan = if (LogicalPlanUtils.hasDescendantAggregateOrJoin(aggregate.vectors)) { val childPlan = materialize(aggregate.vectors, queryContext) // We are here because we have descendent aggregate, if that was multi-partition, the dispatcher will // be InProcessPlanDispatcher and adding the current aggregate using addAggregate will use the same dispatcher diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala index abc0cf2827..d53d244096 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala @@ -23,6 +23,7 @@ import filodb.query.LogicalPlan._ import filodb.query.exec.{LocalPartitionDistConcatExec, _} import filodb.query.exec.InternalRangeFunction.Last + // scalastyle:off file.size.limit object SingleClusterPlanner { @@ -42,20 +43,6 @@ object SingleClusterPlanner { else None } - - /** - * If TargetSchema exists and all of the target-schema label filters (equals) are provided in the query, - * then return true. - * @param filters Query Column Filters - * @param targetSchema TargetSchema - * @return useTargetSchema - use target-schema to calculate query shards - */ - private def useTargetSchemaForShards(filters: Seq[ColumnFilter], targetSchema: Option[TargetSchemaChange]): Boolean = - targetSchema match { - case Some(ts) if ts.schema.nonEmpty => ts.schema - .forall(s => filters.exists(cf => cf.column == s && cf.filter.isInstanceOf[Filter.Equals])) - case _ => false - } } /** @@ -76,6 +63,8 @@ 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) @@ -109,6 +98,26 @@ class SingleClusterPlanner(val dataset: Dataset, ActorPlanDispatcher(targetActor, clusterName) } + /** + * If TargetSchema exists and all of the target-schema label filters are provided in the query, then return true. + * + * @param filters Query Column Filters + * @param targetSchema TargetSchema + * @return useTargetSchema - use target-schema to calculate query shards + */ + private def useTargetSchemaForShards(filters: Seq[ColumnFilter], + targetSchema: Option[TargetSchemaChange]): Boolean = { + if (targetSchema.isEmpty || targetSchema.get.schema.isEmpty) { + return false + } + // Make sure each target-schema column has a filter. + targetSchema.get.schema + .forall(tschemaCol => filters.exists(cf => { + cf.column == tschemaCol && + (shardColumns.contains(tschemaCol) || cf.filter.isInstanceOf[Equals]) + })) + } + /** * Change start time of logical plan according to earliestRetainedTimestampFn */ @@ -208,57 +217,79 @@ class SingleClusterPlanner(val dataset: Dataset, materialized } + def shardsFromValues(shardPairs: Seq[(String, String)], + filters: Seq[ColumnFilter], + qContext: QueryContext, + startMs: Long, + endMs: Long, + useTargetSchemaForShards: Seq[ColumnFilter] => Boolean = _ => false): Seq[Int] = { + val spreadProvToUse = qContext.plannerParams.spreadOverride.getOrElse(spreadProvider) + + val metric = shardPairs.find(_._1 == dsOptions.metricColumn) + .map(_._2) + .getOrElse(throw new BadQueryException(s"Could not find metric value")) + + val shardValues = shardPairs.filterNot(_._1 == dsOptions.metricColumn).map(_._2) + + logger.debug(s"For shardColumns $shardColumns, extracted metric $metric and shard values $shardValues") + val targetSchemaChange = targetSchemaProvider(qContext).targetSchemaFunc(filters) + val targetSchema = { + if (targetSchemaChange.nonEmpty) { + findTargetSchema(targetSchemaChange, startMs, endMs).map(tsc => tsc.schema).getOrElse(Seq.empty) + } else Seq.empty + } + val shardHash = RecordBuilder.shardKeyHash(shardValues, dsOptions.metricColumn, metric, targetSchema) + if(useTargetSchemaForShards(filters)) { + val nonShardKeyLabelPairs = filters.filter(f => !shardColumns.contains(f.column) + && f.filter.isInstanceOf[Filter.Equals]) + .map(cf => cf.column -> + cf.filter.asInstanceOf[Filter.Equals].value.toString).toMap + val partitionHash = RecordBuilder.partitionKeyHash(nonShardKeyLabelPairs, shardPairs.toMap, targetSchema, + dsOptions.metricColumn, metric) + // since target-schema filter is provided in the query, ingestionShard can be used to find the single shard + // that can answer the query. + Seq(shardMapperFunc.ingestionShard(shardHash, partitionHash, spreadProvToUse.spreadFunc(filters).last.spread)) + } else { + shardMapperFunc.queryShards(shardHash, spreadProvToUse.spreadFunc(filters).last.spread) + } + } + // scalastyle:off method.length - def shardsFromFilters(filters: Seq[ColumnFilter], + def shardsFromFilters(rawFilters: Seq[ColumnFilter], qContext: QueryContext, startMs: Long, endMs: Long, - useTargetSchemaForShards: Boolean = false): Seq[Int] = { - - val spreadProvToUse = qContext.plannerParams.spreadOverride.getOrElse(spreadProvider) + 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 shardVals = shardColumns.map { shardCol => - // So to compute the shard hash we need shardCol == value filter (exact equals) for each shardColumn - filters.find(f => f.column == shardCol) match { - case Some(ColumnFilter(_, Filter.Equals(filtVal: String))) => - shardCol -> RecordBuilder.trimShardColumn(dsOptions, shardCol, 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 hasNonEqualsShardKeyFilter = rawFilters.exists { filter => + shardColumns.contains(filter.column) && !filter.filter.isInstanceOf[Equals] } - val metric = shardVals.find(_._1 == dsOptions.metricColumn) - .map(_._2) - .getOrElse(throw new BadQueryException(s"Could not find metric value")) - val shardValues = shardVals.filterNot(_._1 == dsOptions.metricColumn).map(_._2) - logger.debug(s"For shardColumns $shardColumns, extracted metric $metric and shard values $shardValues") - val targetSchemaChange = targetSchemaProvider(qContext).targetSchemaFunc(filters) - val targetSchema = { - if (targetSchemaChange.nonEmpty) { - findTargetSchema(targetSchemaChange, startMs, endMs).map(tsc => tsc.schema).getOrElse(Seq.empty) - } else Seq.empty - } - val shardHash = RecordBuilder.shardKeyHash(shardValues, dsOptions.metricColumn, metric, targetSchema) - if(useTargetSchemaForShards) { - val nonShardKeyLabelPairs = filters.filter(f => !shardColumns.contains(f.column) - && f.filter.isInstanceOf[Filter.Equals]) - .map(cf => cf.column -> - cf.filter.asInstanceOf[Filter.Equals].value.toString).toMap - val partitionHash = RecordBuilder.partitionKeyHash(nonShardKeyLabelPairs, shardVals.toMap, targetSchema, - dsOptions.metricColumn, metric) - // since target-schema filter is provided in the query, ingestionShard can be used to find the single shard - // that can answer the query. - Seq(shardMapperFunc.ingestionShard(shardHash, partitionHash, spreadProvToUse.spreadFunc(filters).last.spread)) + val filterGroups = if (hasNonEqualsShardKeyFilter) { + shardKeyMatcher(rawFilters).map(LogicalPlanUtils.upsertFilters(rawFilters, _)) } else { - shardMapperFunc.queryShards(shardHash, spreadProvToUse.spreadFunc(filters).last.spread) + Seq(rawFilters) } + 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") + } + RecordBuilder.trimShardColumn(dsOptions, shardCol, value) + } + val shardPairs = shardColumns.zip(shardValues) + shardsFromValues(shardPairs, filters, qContext, startMs, endMs, useTargetSchemaForShards) + }.distinct } } // scalastyle:off method.length @@ -338,8 +369,7 @@ class SingleClusterPlanner(val dataset: Dataset, case class LeafInfo(lp: LogicalPlan, startMs: Long, endMs: Long, - renamedFilters: Seq[ColumnFilter], - targetSchemaLabels: Option[Seq[String]]) + renamedFilters: Seq[ColumnFilter]) // construct a LeafInfo for each leaf... val tsp = targetSchemaProvider(qContext) @@ -354,15 +384,8 @@ class SingleClusterPlanner(val dataset: Dataset, case is: IntervalSelector => Some((is.from, is.to)) case _ => None } - val tsLabels: Option[Seq[String]] = { - val tsFunc = tsp.targetSchemaFunc(filters.head) - startEndMsPairOpt match { - case Some((startMs, endMs)) => findTargetSchema(tsFunc, startMs, endMs).map(_.schema) - case _ => None - } - } val (startMs, endMs): (Long, Long) = startEndMsPairOpt.getOrElse((0, Long.MaxValue)) - leafInfos.append(LeafInfo(leafPlan, startMs, endMs, renameMetricFilter(filters.head), tsLabels)) + leafInfos.append(LeafInfo(leafPlan, startMs, endMs, renameMetricFilter(filters.head))) // Do nothing; not pulling data from any shards. case sc: ScalarPlan => { } // Note!! If an unrecognized plan type is encountered, this just pessimistically returns None. @@ -375,13 +398,18 @@ class SingleClusterPlanner(val dataset: Dataset, canGetShardsFromFilters(leaf.renamedFilters, qContext) }) return None - Some(leafInfos.flatMap{ leaf => - val useTargetSchema = leaf.targetSchemaLabels.isDefined && { - val equalColFilterLabels = leaf.renamedFilters.filter(_.filter.isInstanceOf[Filter.Equals]).map(_.column) - leaf.targetSchemaLabels.get.toSet.subsetOf(equalColFilterLabels.toSet) + val shards = leafInfos.flatMap { leaf => + val useTargetSchema = (filters: Seq[ColumnFilter]) => { + val targetSchemaChanges = targetSchemaProvider(qContext).targetSchemaFunc(filters) + val tsChangeExists = isTargetSchemaChanging(targetSchemaChanges, leaf.startMs, leaf.endMs) + val targetSchemaOpt = findTargetSchema(targetSchemaChanges, leaf.startMs, leaf.endMs) + val allTSLabelsPresent = useTargetSchemaForShards(filters, targetSchemaOpt) + !tsChangeExists && allTSLabelsPresent } shardsFromFilters(leaf.renamedFilters, qContext, leaf.startMs, leaf.endMs, useTargetSchema) - }.toSet) + }.toSet + + Some(shards) } // scalastyle:on method.length @@ -399,9 +427,10 @@ class SingleClusterPlanner(val dataset: Dataset, LogicalPlanUtils.getPushdownKeys( plan, targetSchemaProvider(qContext), + shardKeyMatcher, dataset.options.nonMetricShardColumns, getRawPushdownShards, - rs => LogicalPlan.getRawSeriesFilters(rs)) + rs => LogicalPlan.getRawSeriesFilters(rs).flatMap(shardKeyMatcher(_))) } /** @@ -712,7 +741,6 @@ class SingleClusterPlanner(val dataset: Dataset, val (renamedFilters, schemaOpt) = extractSchemaFilter(renameMetricFilter(lp.filters)) val spreadChanges = spreadProvToUse.spreadFunc(renamedFilters) - val targetSchemaChanges = targetSchemaProvider(qContext).targetSchemaFunc(renamedFilters) val rangeSelectorWithOffset = lp.rangeSelector match { case IntervalSelector(fromMs, toMs) => IntervalSelector(fromMs - offsetMillis - lp.lookbackMs.getOrElse( @@ -724,37 +752,52 @@ class SingleClusterPlanner(val dataset: Dataset, case _ => false } - // Change in Target Schema in query window, do not use target schema to find query shards - val tsChangeExists = (rangeSelectorWithOffset match { - case IntervalSelector(from, to) => isTargetSchemaChanging(targetSchemaChanges, from, to) - case _ => false - }) - val targetSchemaOpt = (rangeSelectorWithOffset match { - case IntervalSelector(from, to) => findTargetSchema(targetSchemaChanges, from, to) - case _ => None - }) - val allTSLabelsPresent = useTargetSchemaForShards(renamedFilters, targetSchemaOpt) - - // Whether to use target-schema for calculating the target shards. - // If there is change in target-schema or target-schema not defined or query doesn't have all the labels of - // target-schema, use shardKeyHash to find the target shards. - // 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 useTSForQueryShards = !tsChangeExists && allTSLabelsPresent + // Record whether-or-not (a) any tschema changes during the query, and (b) the changing tshema was applied. + // Use this info to help determine whether-or-not the results need to be stitched. + val tschemaChangesOverTime = new Array[Boolean](1) + tschemaChangesOverTime(0) = false + + val useTargetSchema = (filters: Seq[ColumnFilter]) => { + val targetSchemaChanges = targetSchemaProvider(qContext).targetSchemaFunc(filters) + + // Change in Target Schema in query window, do not use target schema to find query shards + val tsChangeExists = (rangeSelectorWithOffset match { + case IntervalSelector(from, to) => isTargetSchemaChanging(targetSchemaChanges, from, to) + case _ => false + }) + val targetSchemaOpt = (rangeSelectorWithOffset match { + case IntervalSelector(from, to) => findTargetSchema(targetSchemaChanges, from, to) + case _ => None + }) + val allTSLabelsPresent = useTargetSchemaForShards(filters, targetSchemaOpt) + + // Whether to use target-schema for calculating the target shards. + // If there is change in target-schema or target-schema not defined or query doesn't have all the labels of + // target-schema, use shardKeyHash to find the target shards. + // 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 + if (tsChangeExists) { + // Record that some tschema changes exist. This will later invoke result stitching. + tschemaChangesOverTime(0) = true + } + useTschema + } val (startMs, endMs): (Long, Long) = rangeSelectorWithOffset match { case IntervalSelector(from, to) => (from, to) case _ => (0, Long.MaxValue) } - val execPlans = shardsFromFilters(renamedFilters, qContext, startMs, endMs, useTSForQueryShards).map { shard => + val execPlans = shardsFromFilters(renamedFilters, qContext, startMs, endMs, useTargetSchema).map { shard => val dispatcher = dispatcherForShard(shard, forceInProcess, qContext) MultiSchemaPartitionsExec(qContext, dispatcher, dsRef, shard, renamedFilters, toChunkScanMethod(rangeSelectorWithOffset), dsOptions.metricColumn, schemaOpt, colName) } - // Stitch only if spread and/or target-schema changes during the query-window. - // when target-schema changes during query window, data might be ingested in different shards after the change. - PlanResult(execPlans, needsStitch || tsChangeExists) + // Stitch only if spread changes during the query-window. + // When a target-schema changes during query window, data might be ingested in + // different shards after the change. + PlanResult(execPlans, needsStitch || tschemaChangesOverTime(0)) } // scalastyle:on method.length @@ -781,19 +824,14 @@ class SingleClusterPlanner(val dataset: Dataset, PlanResult(metaExec) } - // allow metadataQueries to get list of shards from shardKeyFilters only if all shardCols have Equals filter + // allow metadataQueries to get list of shards from shardKeyFilters only if + // filters are given for all shard-key columns private def canGetShardsFromFilters(renamedFilters: Seq[ColumnFilter], qContext: QueryContext): Boolean = { - if (qContext.plannerParams.shardOverrides.isEmpty && shardColumns.nonEmpty) { + qContext.plannerParams.shardOverrides.isEmpty && + shardColumns.nonEmpty && shardColumns.toSet.subsetOf(renamedFilters.map(_.column).toSet) && - 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 _ => false - } - } - } else false + shardColumns.forall { shardCol => renamedFilters.exists(f => f.column == shardCol) } } private def materializeLabelNames(qContext: QueryContext, @@ -850,16 +888,17 @@ class SingleClusterPlanner(val dataset: Dataset, // NOTE: _type_ filter support currently isn't there in series keys queries val (renamedFilters, _) = extractSchemaFilter(renameMetricFilter(lp.filters)) - val targetSchemaChanges = targetSchemaProvider(qContext).targetSchemaFunc(renamedFilters) - - // Change in Target Schema in query window, do not use target schema to find query shards - val tsChangeExists = isTargetSchemaChanging(targetSchemaChanges, lp.startMs, lp.endMs) - val targetSchemaOpt = findTargetSchema(targetSchemaChanges, lp.startMs, lp.endMs) - val allTSLabelsPresent = useTargetSchemaForShards(renamedFilters, targetSchemaOpt) + val useTSForQueryShards = (filters: Seq[ColumnFilter]) => { + // Change in Target Schema in query window, do not use target schema to find query shards + val targetSchemaChanges = targetSchemaProvider(qContext).targetSchemaFunc(filters) + val tsChangeExists = isTargetSchemaChanging(targetSchemaChanges, lp.startMs, lp.endMs) + val targetSchemaOpt = findTargetSchema(targetSchemaChanges, lp.startMs, lp.endMs) + val allTSLabelsPresent = useTargetSchemaForShards(filters, targetSchemaOpt) + // No change in TargetSchema or not all target-schema labels are provided in the query + !tsChangeExists && allTSLabelsPresent + } - // No change in TargetSchema or not all target-schema labels are provided in the query - val useTSForQueryShards = !tsChangeExists && allTSLabelsPresent - val shardsToHit = if (useTSForQueryShards || canGetShardsFromFilters(renamedFilters, qContext)) { + val shardsToHit = if (canGetShardsFromFilters(renamedFilters, qContext)) { shardsFromFilters(renamedFilters, qContext, lp.startMs, lp.endMs, useTSForQueryShards) } else { mdNoShardKeyFilterRequests.increment() diff --git a/core/src/main/scala/filodb.core/query/QueryContext.scala b/core/src/main/scala/filodb.core/query/QueryContext.scala index 22037fb410..eda29689ee 100644 --- a/core/src/main/scala/filodb.core/query/QueryContext.scala +++ b/core/src/main/scala/filodb.core/query/QueryContext.scala @@ -154,7 +154,8 @@ case class PlannerParams(applicationId: String = "filodb", skipAggregatePresent: Boolean = false, processFailure: Boolean = true, processMultiPartition: Boolean = false, - allowPartialResults: Boolean = false) + allowPartialResults: Boolean = false, + reduceShardKeyRegexFanout: Boolean = true) object PlannerParams { def apply(constSpread: Option[SpreadProvider], sampleLimit: Int): PlannerParams = From 16dc0aaa3d34bf9f9ac1476057dbe98053c973aa Mon Sep 17 00:00:00 2001 From: Alex Theimer Date: Fri, 29 Sep 2023 00:01:22 -0700 Subject: [PATCH 02/27] update tests --- .../MultiPartitionPlannerSpec.scala | 236 +++--- .../queryplanner/PlannerHierarchySpec.scala | 514 ++++++------ .../ShardKeyRegexPlannerSpec.scala | 786 +++++++++--------- .../SingleClusterPlannerSpec.scala | 16 +- .../filodb/query/PlanValidationSpec.scala | 16 +- 5 files changed, 771 insertions(+), 797 deletions(-) diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala index 640db472f3..b5d623a648 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala @@ -117,7 +117,7 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers with PlanValida // expectedStarMs ends up to be 3 400 000, which does not look right to me, it is supposed to be 3 000 000 // kpetrov, 12/02/21 val expectedStartMs = ((startSeconds*1000) to (endSeconds*1000) by (step*1000)).find { instant => - instant - lookbackMs > (localPartitionStart * 1000) + instant - lookbackMs >= (localPartitionStart * 1000) }.get val remoteExec2 = stitchRvsExec.children(1).asInstanceOf[PromQlRemoteExec] @@ -324,13 +324,13 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers with PlanValida ) ) val expectedPlan = - """E~BinaryJoinExec(binaryOp=ADD, on=List(), ignoring=List()) on InProcessPlanDispatcher(filodb.core.query.QueryConfig@1dad01fe) - |-E~PromQlRemoteExec(PromQlQueryParams(test{job="app1"},3000,100,12000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,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=remote-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(filodb.core.query.QueryConfig@1dad01fe) - |-E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1232106303],raw) + """E~BinaryJoinExec(binaryOp=ADD, on=List(), ignoring=List()) on InProcessPlanDispatcher + |-E~PromQlRemoteExec(PromQlQueryParams(test{job="app1"},3000,100,12000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,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=10000) on InProcessPlanDispatcher + |-E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) |--T~PeriodicSamplesMapper(start=3000000, step=100000, end=12000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=15, chunkMethod=TimeRangeChunkScan(2700000,12000000), filters=List(ColumnFilter(job,Equals(app2)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1232106303],raw) + |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=15, chunkMethod=TimeRangeChunkScan(2700000,12000000), filters=List(ColumnFilter(job,Equals(app2)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) |--T~PeriodicSamplesMapper(start=3000000, step=100000, end=12000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=31, chunkMethod=TimeRangeChunkScan(2700000,12000000), filters=List(ColumnFilter(job,Equals(app2)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1232106303],raw)""".stripMargin + |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=31, chunkMethod=TimeRangeChunkScan(2700000,12000000), filters=List(ColumnFilter(job,Equals(app2)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw)""".stripMargin validatePlan(execPlan, expectedPlan) } @@ -373,9 +373,9 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers with PlanValida ) execPlan.printTree() val expectedPlan = { - """E~BinaryJoinExec(binaryOp=ADD, on=List(), ignoring=List()) on InProcessPlanDispatcher(filodb.core.query.QueryConfig@7c0f28f8) - |-E~PromQlRemoteExec(PromQlQueryParams(test{job="app1"},3000,100,12000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,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=remote-url1, requestTimeoutMs=10000) on InProcessPlanDispatcher(filodb.core.query.QueryConfig@7c0f28f8) - |-E~PromQlRemoteExec(PromQlQueryParams(test{job="app2"},3000,100,12000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,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=remote-url2, requestTimeoutMs=10000) on InProcessPlanDispatcher(filodb.core.query.QueryConfig@7c0f28f8)""".stripMargin + """E~BinaryJoinExec(binaryOp=ADD, on=List(), ignoring=List()) on InProcessPlanDispatcher + |-E~PromQlRemoteExec(PromQlQueryParams(test{job="app1"},3000,100,12000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,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-url1, requestTimeoutMs=10000) on InProcessPlanDispatcher + |-E~PromQlRemoteExec(PromQlQueryParams(test{job="app2"},3000,100,12000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,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-url2, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin } validatePlan(execPlan, expectedPlan) } @@ -419,15 +419,15 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers with PlanValida ) val expectedPlan = """T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(3000,100,12000)) - |-E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on InProcessPlanDispatcher(filodb.core.query.QueryConfig@2a8a3ada) + |-E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on InProcessPlanDispatcher |--T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |---E~BinaryJoinExec(binaryOp=ADD, on=List(), ignoring=List()) on InProcessPlanDispatcher(filodb.core.query.QueryConfig@2a8a3ada) - |----E~PromQlRemoteExec(PromQlQueryParams(test{job="app1"},3000,100,12000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,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=remote-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(filodb.core.query.QueryConfig@2a8a3ada) - |----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1705462153],raw) + |---E~BinaryJoinExec(binaryOp=ADD, on=List(), ignoring=List()) on InProcessPlanDispatcher + |----E~PromQlRemoteExec(PromQlQueryParams(test{job="app1"},3000,100,12000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,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=10000) on InProcessPlanDispatcher + |----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) |-----T~PeriodicSamplesMapper(start=3000000, step=100000, end=12000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=15, chunkMethod=TimeRangeChunkScan(2700000,12000000), filters=List(ColumnFilter(job,Equals(app2)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1705462153],raw) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=15, chunkMethod=TimeRangeChunkScan(2700000,12000000), filters=List(ColumnFilter(job,Equals(app2)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) |-----T~PeriodicSamplesMapper(start=3000000, step=100000, end=12000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=31, chunkMethod=TimeRangeChunkScan(2700000,12000000), filters=List(ColumnFilter(job,Equals(app2)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1705462153],raw)""".stripMargin + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=31, chunkMethod=TimeRangeChunkScan(2700000,12000000), filters=List(ColumnFilter(job,Equals(app2)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw)""".stripMargin validatePlan(execPlan, expectedPlan) } @@ -457,7 +457,7 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers with PlanValida QueryContext(origQueryParams = promQlQueryParams, plannerParams = PlannerParams(processMultiPartition = true)) ) val expectedPlan = - """E~PromQlRemoteExec(PromQlQueryParams(avg_over_time(test{job = "app"}[10m:1m]),1200,100,1800,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,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=remote-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(filodb.core.query.QueryConfig@4c82b5df)""" + """E~PromQlRemoteExec(PromQlQueryParams(avg_over_time(test{job="app"}[600s:60s]),1200,100,1800,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,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=10000) on InProcessPlanDispatcher""" validatePlan(execPlan, expectedPlan) } @@ -531,13 +531,13 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers with PlanValida ) val expectedPlan = """T~PeriodicSamplesMapper(start=1200000, step=120000, end=1800000, window=Some(600000), functionId=Some(AvgOverTime), rawSource=false, offsetMs=None) - |-E~BinaryJoinExec(binaryOp=ADD, on=List(), ignoring=List()) on InProcessPlanDispatcher(filodb.core.query.QueryConfig@4684240f) - |--E~PromQlRemoteExec(PromQlQueryParams(test{job="app1"},600,60,1800,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,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=remote-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(filodb.core.query.QueryConfig@4684240f) - |--E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1103787767],raw) + |-E~BinaryJoinExec(binaryOp=ADD, on=List(), ignoring=List()) on InProcessPlanDispatcher + |--E~PromQlRemoteExec(PromQlQueryParams(test{job="app1"},600,60,1800,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,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=10000) on InProcessPlanDispatcher + |--E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) |---T~PeriodicSamplesMapper(start=600000, step=60000, end=1800000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=15, chunkMethod=TimeRangeChunkScan(300000,1800000), filters=List(ColumnFilter(job,Equals(app2)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1103787767],raw) + |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=15, chunkMethod=TimeRangeChunkScan(300000,1800000), filters=List(ColumnFilter(job,Equals(app2)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) |---T~PeriodicSamplesMapper(start=600000, step=60000, end=1800000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=31, chunkMethod=TimeRangeChunkScan(300000,1800000), filters=List(ColumnFilter(job,Equals(app2)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1103787767],raw)""".stripMargin + |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=31, chunkMethod=TimeRangeChunkScan(300000,1800000), filters=List(ColumnFilter(job,Equals(app2)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw)""".stripMargin validatePlan(execPlan, expectedPlan) } @@ -572,7 +572,7 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers with PlanValida val expectedPlan = """T~PeriodicSamplesMapper(start=1200000, step=120000, end=1800000, window=Some(600000), functionId=Some(MinOverTime), rawSource=false, offsetMs=None) |-E~BinaryJoinExec(binaryOp=ADD, on=List(), ignoring=List()) on InProcessPlanDispatcher - |--E~PromQlRemoteExec(PromQlQueryParams(sum_over_time(sum(test{job="app1"})[600s:60s]),600,60,1800,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,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=remote-url, requestTimeoutMs=10000) on InProcessPlanDispatcher + |--E~PromQlRemoteExec(PromQlQueryParams(sum_over_time(sum(test{job="app1"})[600s:60s]),600,60,1800,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,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=10000) on InProcessPlanDispatcher |--T~PeriodicSamplesMapper(start=600000, step=60000, end=1800000, window=Some(600000), functionId=Some(SumOverTime), rawSource=false, offsetMs=None) |---T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(0,60,1800)) |----E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) @@ -873,11 +873,11 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers with PlanValida val remoteExec2 = stitchRvsExec.children(1).asInstanceOf[PromQlRemoteExec] val expectedStartMs1 = ((startSeconds*1000) to (endSeconds*1000) by (step*1000)).find { instant => - instant - lookbackMs > (secondPartitionStart * 1000) + instant - lookbackMs >= (secondPartitionStart * 1000) }.get val expectedStartMs2 = ((startSeconds*1000) to (endSeconds*1000) by (step*1000)).find { instant => - instant - lookbackMs > (thirdPartitionStart * 1000) + instant - lookbackMs >= (thirdPartitionStart * 1000) }.get val queryParams2 = remoteExec2.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams] @@ -899,10 +899,10 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers with PlanValida } - it ("should generate all PromQlRemoteExec plan for instant queries") { + it ("should generate at most one PromQlRemoteExec plan for non-shard-key-regex instant queries") { val startSeconds = 1000 val endSeconds = 1000 - val localPartitionStartSec= 950 + val localPartitionStartSec = 850 val lookbackMs = 100000 val step = 1 @@ -923,94 +923,82 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers with PlanValida val engine = new MultiPartitionPlanner(partitionLocationProvider, localPlanner, "local", dataset, queryConfig) val lp = Parser.queryRangeToLogicalPlan("test{job = \"app\"}[100s]", TimeStepParams(startSeconds, step, endSeconds)) - val promQlQueryParams = PromQlQueryParams("test{job = \"app\"}", startSeconds, step, endSeconds) + val promQlQueryParams = PromQlQueryParams("test{job = \"app\"}[100s]", startSeconds, step, endSeconds) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams, plannerParams = PlannerParams(processMultiPartition = true))) - val stitchRvsExec = execPlan.asInstanceOf[StitchRvsExec] - stitchRvsExec.children.size shouldEqual (2) - stitchRvsExec.children(0).isInstanceOf[PromQlRemoteExec] shouldEqual (true) - stitchRvsExec.children(1).isInstanceOf[PromQlRemoteExec] shouldEqual (true) - - - // Instant/Raw queries will have same start and end point in all partitions as we want to fetch raw data - val remoteExec1 = stitchRvsExec.children(0).asInstanceOf[PromQlRemoteExec] - val queryParams1 = remoteExec1.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams] - queryParams1.startSecs shouldEqual startSeconds - queryParams1.endSecs shouldEqual endSeconds - queryParams1.stepSecs shouldEqual step - remoteExec1.queryContext.plannerParams.processFailure shouldEqual true - remoteExec1.queryContext.plannerParams.processMultiPartition shouldEqual false - remoteExec1.queryEndpoint shouldEqual "remote-url1" - - val remoteExec2 = stitchRvsExec.children(1).asInstanceOf[PromQlRemoteExec] - val queryParams2 = remoteExec1.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams] - queryParams2.startSecs shouldEqual startSeconds - queryParams2.endSecs shouldEqual endSeconds - queryParams2.stepSecs shouldEqual step - remoteExec2.queryContext.plannerParams.processFailure shouldEqual true - remoteExec2.queryContext.plannerParams.processMultiPartition shouldEqual false - remoteExec2.queryEndpoint shouldEqual "remote-url2" - - } - - it ("should generate second Exec with start and end time equal to query end time when query duration is less" + - "than or equal to lookback ") { - - val startSeconds = 1594309980L - val endSeconds = 1594310280L - val localPartitionStartMs: Long = 1594309980001L - val step = 15L - - val partitionLocationProvider = new PartitionLocationProvider { - override def getPartitions(routingKey: Map[String, String], timeRange: TimeRange): List[PartitionAssignment] = { - if (routingKey.equals(Map("job" -> "app"))) List( - PartitionAssignment("remote1", "remote-url", TimeRange(startSeconds * 1000 - lookbackMs, - localPartitionStartMs - 1)), PartitionAssignment("remote2", "remote-url", - TimeRange(localPartitionStartMs, endSeconds * 1000))) - else Nil - } - - override def getMetadataPartitions(nonMetricShardKeyFilters: Seq[ColumnFilter], timeRange: TimeRange): List[PartitionAssignment] = List( - PartitionAssignment("remote1", "remote-url", TimeRange(startSeconds * 1000 - lookbackMs, - localPartitionStartMs - 1)), PartitionAssignment("remote2", "remote-url", - TimeRange(localPartitionStartMs, endSeconds * 1000))) - - } - val engine = new MultiPartitionPlanner(partitionLocationProvider, localPlanner, "local", dataset, queryConfig) - val lp = Parser.queryRangeToLogicalPlan("test{job = \"app\"}", TimeStepParams(startSeconds, step, endSeconds)) - - val promQlQueryParams = PromQlQueryParams("test{job = \"app\"}", startSeconds, step, endSeconds) - - val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams, plannerParams = - PlannerParams(processMultiPartition = true))) - val stitchRvsExec = execPlan.asInstanceOf[StitchRvsExec] - stitchRvsExec.children.size shouldEqual (2) - stitchRvsExec.children(0).isInstanceOf[PromQlRemoteExec] shouldEqual (true) - stitchRvsExec.children(1).isInstanceOf[PromQlRemoteExec] shouldEqual (true) - + execPlan.isInstanceOf[PromQlRemoteExec] shouldEqual (true) - val remoteExec = stitchRvsExec.children(0).asInstanceOf[PromQlRemoteExec] + // Instant/Raw queries will have same start and end point + val remoteExec = execPlan.asInstanceOf[PromQlRemoteExec] val queryParams = remoteExec.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams] queryParams.startSecs shouldEqual startSeconds - queryParams.endSecs shouldEqual (localPartitionStartMs - 1) / 1000 + queryParams.endSecs shouldEqual endSeconds queryParams.stepSecs shouldEqual step remoteExec.queryContext.plannerParams.processFailure shouldEqual true remoteExec.queryContext.plannerParams.processMultiPartition shouldEqual false - remoteExec.queryEndpoint shouldEqual "remote-url" - - val remoteExec2 = stitchRvsExec.children(1).asInstanceOf[PromQlRemoteExec] - val queryParams2 = remoteExec2.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams] - queryParams2.startSecs shouldEqual endSeconds - queryParams2.endSecs shouldEqual endSeconds - queryParams2.stepSecs shouldEqual step - remoteExec2.queryContext.plannerParams.processFailure shouldEqual true - remoteExec2.queryContext.plannerParams.processMultiPartition shouldEqual false - remoteExec2.queryEndpoint shouldEqual "remote-url" - + remoteExec.queryEndpoint shouldEqual "remote-url2" } +// // TODO: what is this the purpose of this test? The lookback is 300s, and the assignment +// // durations are-- in order-- 300s and 299s,999ms. +// it ("should generate second Exec with start and end time equal to query end time when query duration is less" + +// "than or equal to lookback ") { +// +// val startSeconds = 1594309980L +// val endSeconds = 1594310280L +// val localPartitionStartMs: Long = 1594309980001L +// val step = 15L +// +// val partitionLocationProvider = new PartitionLocationProvider { +// override def getPartitions(routingKey: Map[String, String], timeRange: TimeRange): List[PartitionAssignment] = { +// if (routingKey.equals(Map("job" -> "app"))) List( +// PartitionAssignment("remote1", "remote-url", TimeRange(startSeconds * 1000 - lookbackMs, +// localPartitionStartMs - 1)), PartitionAssignment("remote2", "remote-url", +// TimeRange(localPartitionStartMs, endSeconds * 1000))) +// else Nil +// } +// +// override def getMetadataPartitions(nonMetricShardKeyFilters: Seq[ColumnFilter], timeRange: TimeRange): List[PartitionAssignment] = List( +// PartitionAssignment("remote1", "remote-url", TimeRange(startSeconds * 1000 - lookbackMs, +// localPartitionStartMs - 1)), PartitionAssignment("remote2", "remote-url", +// TimeRange(localPartitionStartMs, endSeconds * 1000))) +// +// } +// val engine = new MultiPartitionPlanner(partitionLocationProvider, localPlanner, "local", dataset, queryConfig) +// val lp = Parser.queryRangeToLogicalPlan("test{job = \"app\"}", TimeStepParams(startSeconds, step, endSeconds)) +// +// val promQlQueryParams = PromQlQueryParams("test{job = \"app\"}", startSeconds, step, endSeconds) +// +// val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams, plannerParams = +// PlannerParams(processMultiPartition = true))) +// val stitchRvsExec = execPlan.asInstanceOf[StitchRvsExec] +// stitchRvsExec.children.size shouldEqual (2) +// stitchRvsExec.children(0).isInstanceOf[PromQlRemoteExec] shouldEqual (true) +// stitchRvsExec.children(1).isInstanceOf[PromQlRemoteExec] shouldEqual (true) +// +// +// val remoteExec = stitchRvsExec.children(0).asInstanceOf[PromQlRemoteExec] +// val queryParams = remoteExec.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams] +// queryParams.startSecs shouldEqual startSeconds +// queryParams.endSecs shouldEqual (localPartitionStartMs - 1) / 1000 +// queryParams.stepSecs shouldEqual step +// remoteExec.queryContext.plannerParams.processFailure shouldEqual true +// remoteExec.queryContext.plannerParams.processMultiPartition shouldEqual false +// remoteExec.queryEndpoint shouldEqual "remote-url" +// +// val remoteExec2 = stitchRvsExec.children(1).asInstanceOf[PromQlRemoteExec] +// val queryParams2 = remoteExec2.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams] +// queryParams2.startSecs shouldEqual endSeconds +// queryParams2.endSecs shouldEqual endSeconds +// queryParams2.stepSecs shouldEqual step +// remoteExec2.queryContext.plannerParams.processFailure shouldEqual true +// remoteExec2.queryContext.plannerParams.processMultiPartition shouldEqual false +// remoteExec2.queryEndpoint shouldEqual "remote-url" +// +// } + it ("should generate Exec plan for Metadata Label values query") { def partitions(timeRange: TimeRange): List[PartitionAssignment] = List(PartitionAssignment("remote", "remote-url", @@ -1463,28 +1451,28 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers with PlanValida // binary join for + should be done in process val expectedPlan = - """E~BinaryJoinExec(binaryOp=ADD, on=List(), ignoring=List()) on InProcessPlanDispatcher(filodb.core.query.QueryConfig@298b64f7) - |-E~BinaryJoinExec(binaryOp=MUL, on=List(), ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#450220728],raw) + """E~BinaryJoinExec(binaryOp=ADD, on=List(), ignoring=List()) on InProcessPlanDispatcher + |-E~BinaryJoinExec(binaryOp=MUL, on=List(), ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) |--T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1000,100,10000)) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#450220728],raw) + |---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=100000, end=10000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=12, chunkMethod=TimeRangeChunkScan(700000,10000000), filters=List(ColumnFilter(job,Equals(app1)), ColumnFilter(__name__,Equals(test1))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#450220728],raw) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=12, chunkMethod=TimeRangeChunkScan(700000,10000000), filters=List(ColumnFilter(job,Equals(app1)), ColumnFilter(__name__,Equals(test1))), 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=100000, end=10000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=28, chunkMethod=TimeRangeChunkScan(700000,10000000), filters=List(ColumnFilter(job,Equals(app1)), ColumnFilter(__name__,Equals(test1))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#450220728],raw) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=28, chunkMethod=TimeRangeChunkScan(700000,10000000), filters=List(ColumnFilter(job,Equals(app1)), ColumnFilter(__name__,Equals(test1))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) |--T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1000,100,10000)) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#450220728],raw) + |---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=100000, end=10000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,10000000), filters=List(ColumnFilter(job,Equals(app1)), ColumnFilter(__name__,Equals(test2))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#450220728],raw) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,10000000), filters=List(ColumnFilter(job,Equals(app1)), ColumnFilter(__name__,Equals(test2))), 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=100000, end=10000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,10000000), filters=List(ColumnFilter(job,Equals(app1)), ColumnFilter(__name__,Equals(test2))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#450220728],raw) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,10000000), filters=List(ColumnFilter(job,Equals(app1)), ColumnFilter(__name__,Equals(test2))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) |-T~InstantVectorFunctionMapper(function=Ln) - |--E~BinaryJoinExec(binaryOp=ADD, on=List(), ignoring=List()) on InProcessPlanDispatcher(filodb.core.query.QueryConfig@298b64f7) - |---E~PromQlRemoteExec(PromQlQueryParams(sum(test3{job="app2"}),1000,100,10000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,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=remote-url-1, requestTimeoutMs=10000) on InProcessPlanDispatcher(filodb.core.query.QueryConfig@298b64f7) - |---E~PromQlRemoteExec(PromQlQueryParams(sum(test4{job="app3"}),1000,100,10000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,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=remote-url-2, requestTimeoutMs=10000) on InProcessPlanDispatcher(filodb.core.query.QueryConfig@298b64f7)""".stripMargin + |--E~BinaryJoinExec(binaryOp=ADD, on=List(), ignoring=List()) on InProcessPlanDispatcher + |---E~PromQlRemoteExec(PromQlQueryParams(sum(test3{job="app2"}),1000,100,10000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,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-1, requestTimeoutMs=10000) on InProcessPlanDispatcher + |---E~PromQlRemoteExec(PromQlQueryParams(sum(test4{job="app3"}),1000,100,10000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,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-2, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin validatePlan(execPlan, expectedPlan) @@ -1527,25 +1515,25 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers with PlanValida // to this remote partitions. Finally the top level binary join for + should be done in process val expectedPlan = - """E~BinaryJoinExec(binaryOp=ADD, on=List(), ignoring=List()) on InProcessPlanDispatcher(filodb.core.query.QueryConfig@298b64f7) - |-E~BinaryJoinExec(binaryOp=MUL, on=List(), ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#450220728],raw) + """E~BinaryJoinExec(binaryOp=ADD, on=List(), ignoring=List()) on InProcessPlanDispatcher + |-E~BinaryJoinExec(binaryOp=MUL, on=List(), ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) |--T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1000,100,10000)) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#450220728],raw) + |---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=100000, end=10000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=12, chunkMethod=TimeRangeChunkScan(700000,10000000), filters=List(ColumnFilter(job,Equals(app1)), ColumnFilter(__name__,Equals(test1))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#450220728],raw) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=12, chunkMethod=TimeRangeChunkScan(700000,10000000), filters=List(ColumnFilter(job,Equals(app1)), ColumnFilter(__name__,Equals(test1))), 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=100000, end=10000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=28, chunkMethod=TimeRangeChunkScan(700000,10000000), filters=List(ColumnFilter(job,Equals(app1)), ColumnFilter(__name__,Equals(test1))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#450220728],raw) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=28, chunkMethod=TimeRangeChunkScan(700000,10000000), filters=List(ColumnFilter(job,Equals(app1)), ColumnFilter(__name__,Equals(test1))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) |--T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1000,100,10000)) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#450220728],raw) + |---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=100000, end=10000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,10000000), filters=List(ColumnFilter(job,Equals(app1)), ColumnFilter(__name__,Equals(test2))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#450220728],raw) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,10000000), filters=List(ColumnFilter(job,Equals(app1)), ColumnFilter(__name__,Equals(test2))), 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=100000, end=10000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,10000000), filters=List(ColumnFilter(job,Equals(app1)), ColumnFilter(__name__,Equals(test2))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#450220728],raw) - |-E~PromQlRemoteExec(PromQlQueryParams(ln((sum(test3{job="app2"}) + sum(test4{job="app2"}))),1000,100,10000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,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=remote-url-1, requestTimeoutMs=10000) on InProcessPlanDispatcher(filodb.core.query.QueryConfig@298b64f7)""".stripMargin + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,10000000), filters=List(ColumnFilter(job,Equals(app1)), ColumnFilter(__name__,Equals(test2))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |-E~PromQlRemoteExec(PromQlQueryParams(ln((sum(test3{job="app2"}) + sum(test4{job="app2"}))),1000,100,10000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,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-1, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin validatePlan(execPlan, expectedPlan) } @@ -1582,7 +1570,7 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers with PlanValida // Above Binary join should push the entire query to remote partition val expectedPlan = - """E~PromQlRemoteExec(PromQlQueryParams(sum(test1{job = "app2"}) * sum(test2{job = "app2"}) +ln(sum(test3{job = "app2"}) + sum(test4{job = "app2"})),1000,100,10000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,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=remote-url-1, requestTimeoutMs=10000) on InProcessPlanDispatcher(filodb.core.query.QueryConfig@64b0d1fa)""" + """E~PromQlRemoteExec(PromQlQueryParams(((sum(test1{job="app2"}) * sum(test2{job="app2"})) + ln((sum(test3{job="app2"}) + sum(test4{job="app2"})))),1000,100,10000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,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-1, requestTimeoutMs=10000) on InProcessPlanDispatcher""" validatePlan(execPlan, expectedPlan) } @@ -1631,7 +1619,7 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers with PlanValida PlannerParams(processMultiPartition = true))) - val expectedRemotePlan = """E~MetadataRemoteExec(PromQlQueryParams(test{job="app2"},1000,100,10000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,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=remote-url-1, requestTimeoutMs=10000) on InProcessPlanDispatcher(filodb.core.query.QueryConfig@73c48264)""" + val expectedRemotePlan = """E~MetadataRemoteExec(PromQlQueryParams(,1000,100,10000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,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-1, requestTimeoutMs=10000) on InProcessPlanDispatcher""" validatePlan(remoteExecPlan, expectedRemotePlan) } diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala index 3f194654db..f6e734a918 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala @@ -7,9 +7,9 @@ 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 +import filodb.coordinator.client.QueryCommands.{FunctionalTargetSchemaProvider} import filodb.core.{MetricsTestData, StaticTargetSchemaProvider, TargetSchemaChange} -import filodb.core.metadata.Schemas +import filodb.core.metadata.{Dataset, Schemas} import filodb.core.query._ import filodb.core.query.Filter.{Equals, EqualsRegex} import filodb.prometheus.ast.{TimeStepParams, WindowConstants} @@ -18,16 +18,13 @@ import filodb.prometheus.parse.Parser.Antlr import filodb.query.{BadQueryException, IntervalSelector, LabelCardinality, PlanValidationSpec, RawSeries} import filodb.query.exec._ + // scalastyle:off line.size.limit +// scalastyle:off number.of.methods class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationSpec { private implicit val system: ActorSystem = ActorSystem() private val node = TestProbe().ref - private val mapper = new ShardMapper(2) - for {i <- 0 until 2} mapper.registerNode(Seq(i), node) - - private def mapperRef = mapper - private val dataset = MetricsTestData.timeseriesDatasetWithMetric private val schemas = Schemas(dataset.schema) @@ -40,44 +37,74 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS private val now = 1634777330000L private val rawRetention = 7.days.toMillis - val rawPlanner = new SingleClusterPlanner(dataset, schemas, mapperRef, - earliestRetainedTimestampFn = now - rawRetention, queryConfig, "raw") - private val downsampleRetention = 30.days.toMillis - val downsamplePlanner = new SingleClusterPlanner(dataset, schemas, mapperRef, - earliestRetainedTimestampFn = now - downsampleRetention, queryConfig, "downsample") - - private def inProcessDispatcher = InProcessPlanDispatcher(QueryConfig.unitTestingQueryConfig) - + private val rrRetention = 30.days.toMillis private val timeToDownsample = 6.hours.toMillis - private val longTermPlanner = new LongTimeRangePlanner(rawPlanner, downsamplePlanner, - earliestRawTimestampFn = now - rawRetention, latestDownsampleTimestampFn = now - timeToDownsample, - inProcessDispatcher, queryConfig, dataset) - private val rrRetention = 30.days.toMillis - val recordingRulesPlanner = new SingleClusterPlanner(dataset, schemas, mapperRef, - earliestRetainedTimestampFn = now - rrRetention, - queryConfig, "recordingRules") + private def inProcessDispatcher = InProcessPlanDispatcher(QueryConfig.unitTestingQueryConfig) private val plannerSelector = (metricName: String) => { if (metricName.contains(":1m")) "recordingRules" else "longTerm" } - val planners = Map("longTerm" -> longTermPlanner, "recordingRules" -> recordingRulesPlanner) - val singlePartitionPlanner = new SinglePartitionPlanner(planners, plannerSelector, - dataset, queryConfig) + + 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 = { + 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) + val downsamplePlanner = new SingleClusterPlanner(dataset, schemas, mapperRef, + earliestRetainedTimestampFn = now - downsampleRetention, queryConfig, "downsample", shardKeyMatcher = shardKeyMatcher) + 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) + 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 + } private val oneRemotePartitionLocationProvider = new PartitionLocationProvider { override def getPartitions(routingKey: Map[String, String], timeRange: TimeRange): List[PartitionAssignment] = { - if (routingKey.contains("_ns_")) { - if (routingKey("_ns_") == "localNs" || routingKey("_ns_") == "localNs1") { - List(PartitionAssignment("localPartition", "localPartition-url", TimeRange(timeRange.startMs, timeRange.endMs))) + routingKey("_ns_").split('|').map{ ns => + if (ns == "localNs" || ns == "localNs1") { + PartitionAssignment("localPartition", "localPartition-url", TimeRange(timeRange.startMs, timeRange.endMs)) } else { - List(PartitionAssignment("remotePartition", "remotePartition-url", - TimeRange(timeRange.startMs, timeRange.endMs))) + PartitionAssignment("remotePartition", "remotePartition-url", TimeRange(timeRange.startMs, timeRange.endMs)) } - } else { - List.empty - } + }.toList } override def getMetadataPartitions(nonMetricShardKeyFilters: Seq[ColumnFilter], @@ -99,45 +126,13 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS case "remoteNs1" => List(PartitionAssignment("remotePartition1", "remote1Partition-url1", TimeRange(timeRange.startMs, timeRange.endMs))) - case _ => throw new IllegalArgumentException("nope") + case key => throw new IllegalArgumentException("unexpected routing key: " + key) } } 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 @@ -169,11 +164,26 @@ 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, oneRemoteMultiPartitionPlanner, shardKeyMatcherFn, queryConfig) - val oneRemoteRootPlanner = new ShardKeyRegexPlanner(dataset, oneRemoteMultiPartitionPlanner, oneRemoteShardKeyMatcherFn, queryConfig) - val twoRemoteRootPlanner = new ShardKeyRegexPlanner(dataset, twoRemoteMultiPartitionPlanner, twoRemoteShardKeyMatcherFn, queryConfig) + val rootPlanner = new ShardKeyRegexPlanner(dataset, defaultMultiPartitionPlanner, shardKeyMatcherFn, oneRemotePartitionLocationProvider, queryConfig) + val oneRemoteRootPlanner = new ShardKeyRegexPlanner(dataset, oneRemoteMultiPartitionPlanner, oneRemoteShardKeyMatcherFn, oneRemotePartitionLocationProvider, queryConfig) + val twoRemoteRootPlanner = new ShardKeyRegexPlanner(dataset, twoRemoteMultiPartitionPlanner, twoRemoteShardKeyMatcherFn, twoRemotePartitionLocationProvider, queryConfig) private val startSeconds = now / 1000 - 10.days.toSeconds @@ -256,43 +266,31 @@ 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(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) + |-E~LocalPartitionReduceAggregateExec(aggrOp=Count, aggrParams=List()) on InProcessPlanDispatcher |--T~AggregateMapReduce(aggrOp=Count, aggrParams=List(), without=List(), by=List()) - |---E~SetOperatorExec(binaryOp=LUnless, on=List(instance), ignoring=List()) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |----E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192))) - |-----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],raw) + |---E~SetOperatorExec(binaryOp=LUnless, on=List(instance), ignoring=List()) on InProcessPlanDispatcher + |----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_,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~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) |------T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],raw) - |-----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],downsample) + |-------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) |------T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],downsample) + |-------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) |------T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],downsample) - |----E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-----E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192))) - |------E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],raw) - |-------T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(instance,EqualsRegex(.*)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs1)), ColumnFilter(_metric_,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],raw) - |-------T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(instance,EqualsRegex(.*)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs1)), ColumnFilter(_metric_,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],raw) - |------E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],downsample) - |-------T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(instance,EqualsRegex(.*)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs1)), ColumnFilter(_metric_,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],downsample) - |-------T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(instance,EqualsRegex(.*)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs1)), ColumnFilter(_metric_,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],downsample) - |-----E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192))) - |------E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],raw) - |-------T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(instance,EqualsRegex(.*)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],raw) - |-------T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(instance,EqualsRegex(.*)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],raw) - |------E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],downsample) - |-------T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(instance,EqualsRegex(.*)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],downsample) - |-------T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(instance,EqualsRegex(.*)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(_metric_,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#766017432],downsample)""".stripMargin + |-------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 validatePlan(execPlan, expected) } @@ -306,23 +304,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(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) + |-E~LocalPartitionReduceAggregateExec(aggrOp=Count, aggrParams=List()) on InProcessPlanDispatcher |--T~AggregateMapReduce(aggrOp=Count, aggrParams=List(), without=List(), by=List()) - |---E~SetOperatorExec(binaryOp=LUnless, on=List(instance), ignoring=List()) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |----E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192))) - |-----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1681100802],raw) + |---E~SetOperatorExec(binaryOp=LUnless, on=List(instance), ignoring=List()) on InProcessPlanDispatcher + |----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_,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~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) |------T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1681100802],raw) - |-----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1681100802],downsample) + |-------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) |------T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1681100802],downsample) + |-------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) |------T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1681100802],downsample) - |----E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-----E~PromQlRemoteExec(PromQlQueryParams(bar{instance=~".*",_ws_="demo",_ns_="remoteNs0"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remotePartition-url0, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) - |-----E~PromQlRemoteExec(PromQlQueryParams(bar{instance=~".*",_ws_="demo",_ns_="remoteNs1"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,true,true,false,false), queryEndpoint=remote1Partition-url1, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536)))""".stripMargin + |-------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=~".*"},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 + |-----E~PromQlRemoteExec(PromQlQueryParams(bar{_ws_="demo",_ns_=~"remoteNs.*",instance=~".*"},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""".stripMargin validatePlan(execPlan, expected) } @@ -353,7 +351,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), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(filodb.core.query.QueryConfig@39de9bda)""".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,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin validatePlan(execPlan, expected) } @@ -909,16 +907,16 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS plannerParams = PlannerParams(processMultiPartition = true))) val expected = - """E~BinaryJoinExec(binaryOp=ADD, on=List(), ignoring=List()) on InProcessPlanDispatcher(filodb.core.query.QueryConfig@20c812c8) - |-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@20c812c8) + """E~BinaryJoinExec(binaryOp=ADD, on=List(), ignoring=List()) on InProcessPlanDispatcher + |-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 |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634777330)) - |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1684950125],recordingRules) + |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testActor],recordingRules) |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) |----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo:1m))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1684950125],recordingRules) + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo:1m))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],recordingRules) |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) |----T~PeriodicSamplesMapper(start=1633913330000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo:1m))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1684950125],recordingRules)""".stripMargin + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo:1m))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],recordingRules)""".stripMargin validatePlan(execPlan, expected) } @@ -935,27 +933,27 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS val expected = """T~AggregatePresenter(aggrOp=Count, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634777330)) - |-E~LocalPartitionReduceAggregateExec(aggrOp=Count, aggrParams=List()) on InProcessPlanDispatcher(filodb.core.query.QueryConfig@159424e2) + |-E~LocalPartitionReduceAggregateExec(aggrOp=Count, aggrParams=List()) on InProcessPlanDispatcher |--T~AggregateMapReduce(aggrOp=Count, aggrParams=List(), without=List(), by=List()) - |---E~BinaryJoinExec(binaryOp=ADD, on=List(), ignoring=List()) on InProcessPlanDispatcher(filodb.core.query.QueryConfig@159424e2) - |----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,false,true,false,false), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(filodb.core.query.QueryConfig@159424e2) - |----E~StitchRvsExec() on InProcessPlanDispatcher(filodb.core.query.EmptyQueryConfig$@1f736d00) + |---E~BinaryJoinExec(binaryOp=ADD, on=List(), ignoring=List()) on InProcessPlanDispatcher + |----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,false,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher + |----E~StitchRvsExec() on InProcessPlanDispatcher |-----T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1634173130,300,1634777330)) - |------E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1672762875],raw) + |------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=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1672762875],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/testActor],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_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1672762875],raw) + |---------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) |-----T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634172830)) - |------E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1672762875],downsample) + |------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=1633913330000, step=300000, end=1634172830000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1672762875],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/testActor],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_,Equals(localNs)), ColumnFilter(instance,Equals(Inst-1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1672762875],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/testActor],downsample)""".stripMargin validatePlan(execPlan, expected) @@ -972,23 +970,23 @@ 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(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) + |-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) |----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#1211350849],raw) + |------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) |----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#1211350849],raw) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1211350849],downsample) + |------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) |----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#1211350849],downsample) + |------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) |----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#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 + |------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 validatePlan(execPlan, expected) } @@ -1010,18 +1008,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(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~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) |----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(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~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~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(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~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) |----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(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 + |------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 validatePlan(execPlan, expected) } @@ -1035,24 +1033,24 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS plannerParams = PlannerParams(processMultiPartition = true))) val expected = - """T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1634775000,0,1634775000)) + """T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633911000,300,1634775000)) |-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) |----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(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~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) |----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(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~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~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(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~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) |----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(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 + |------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 validatePlan(execPlan, expected) } @@ -1070,15 +1068,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(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~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) |---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(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~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~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(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~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) |---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(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 + |----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"},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,false,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin validatePlan(execPlan, expected) } @@ -1100,10 +1098,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(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~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) |---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(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 + |----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"},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,false,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin validatePlan(execPlan, expected) } @@ -1119,10 +1117,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(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~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) |--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(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 + |---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 validatePlan(execPlan, expected) } @@ -1138,10 +1136,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(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~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) |--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(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 + |---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]),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,false,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin validatePlan(execPlan, expected) } @@ -1154,16 +1152,16 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS plannerParams = PlannerParams(processMultiPartition = true))) val expected = - """T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1634775000,0,1634775000)) + """T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1634771400,60,1634775000)) |-E~MultiPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on InProcessPlanDispatcher |--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(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~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) |---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(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 + |-----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])),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,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin validatePlan(execPlan, expected) } @@ -1177,16 +1175,16 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS val expected = """T~PeriodicSamplesMapper(start=1634775000000, step=0, end=1634775000000, window=Some(3600000), functionId=Some(MaxOverTime), rawSource=false, offsetMs=None) - |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1634775000,0,1634775000)) + |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1634771400,60,1634775000)) |--E~MultiPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on InProcessPlanDispatcher |---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(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~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) |----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(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 + |------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])),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,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin validatePlan(execPlan, expected) } @@ -1199,7 +1197,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS plannerParams = PlannerParams(processMultiPartition = true))) val expected = - """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""" + """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""" validatePlan(execPlan, expected) } @@ -1218,10 +1216,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(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~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) |---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(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 + |----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]),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,false,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin validatePlan(execPlan, expected) } @@ -1241,10 +1239,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(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~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) |----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(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 + |-----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]),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,false,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin validatePlan(execPlan, expected) } @@ -1275,43 +1273,43 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS plannerParams = PlannerParams(processMultiPartition = true))) val expected = - """E~BinaryJoinExec(binaryOp=MUL, on=List(), ignoring=List()) on InProcessPlanDispatcher(filodb.core.query.QueryConfig@5eabff6b) + """E~BinaryJoinExec(binaryOp=MUL, on=List(), ignoring=List()) on InProcessPlanDispatcher |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634777330)) - |--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) + |--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) |-----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#542289610],raw) + |-------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) |-----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#542289610],raw) - |----E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#542289610],downsample) + |-------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) |-----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#542289610],downsample) + |-------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) |-----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#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) + |-------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 |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634777330)) - |--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) + |--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) |-----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(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#542289610],raw) + |-------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) |-----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(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) + |-------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) |-----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(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#542289610],downsample) + |-------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) |-----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(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 + |-------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 validatePlan(execPlan, expected) } @@ -1562,7 +1560,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), 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,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(filodb.core.query.QueryConfig@570ba13)" validatePlan(execPlan, expectedPlan) } @@ -2490,7 +2488,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, 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))""" + 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""" validatePlan(execPlan2, expectedPlan2) // Case 3: top k with regex, the resolved regex should be in local and remote partitions and use PromQLRemoteExec, should fail @@ -2517,8 +2515,8 @@ 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) - val gRpcRemoteRootPlanner = new ShardKeyRegexPlanner(dataset, grpcRemoteMultiPartitionPlanner, shardKeyMatcherFn, queryConfig) + "localPartition", dataset, queryConfig, shardKeyMatcher = shardKeyMatcherFn) + val gRpcRemoteRootPlanner = new ShardKeyRegexPlanner(dataset, grpcRemoteMultiPartitionPlanner, shardKeyMatcherFn, gRpcRemotePartitionLocationProvider, queryConfig) val query4 = """topk(2, test{_ws_ = "demo", _ns_ =~ ".*Ns", instance = "Inst-1"})""" val lp4 = Parser.queryRangeToLogicalPlan(query4, TimeStepParams(startSeconds, step, endSeconds), Antlr) @@ -2529,23 +2527,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(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 + |-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 validatePlan(execPlan4, expectedPlan4) @@ -2559,38 +2557,23 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS plannerParams = PlannerParams(processMultiPartition = true))) val expectedPlan5 = - """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 + """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 validatePlan(execPlan5, expectedPlan5) // Case 6: top k with regex, the resolved regex should all be two remote partition, both PromQLRemoteExec should fail @@ -2639,8 +2622,8 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS } val twoGrpcRemoteMpPlanner = new MultiPartitionPlanner(rwoRemoteGrpcPartitionLocationProvider, singlePartitionPlanner, - "localPartition", dataset, queryConfig) - val twoGrpcRemoteShardKeyRegexPlanner = new ShardKeyRegexPlanner(dataset, twoGrpcRemoteMpPlanner, twoRemoteShardKeyMatcherFn, queryConfig) + "localPartition", dataset, queryConfig, shardKeyMatcher = twoRemoteShardKeyMatcherFn) + val twoGrpcRemoteShardKeyRegexPlanner = new ShardKeyRegexPlanner(dataset, twoGrpcRemoteMpPlanner, twoRemoteShardKeyMatcherFn, rwoRemoteGrpcPartitionLocationProvider, queryConfig) val query8 = """topk(2, test{_ws_ = "demo", _ns_ =~ "remoteNs.*", instance = "Inst-1"})""" val lp8 = @@ -2653,9 +2636,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(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 + |-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 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 @@ -2672,9 +2655,9 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS timeRange: TimeRange): List[PartitionAssignment] = ??? } - val oneRemoteMpPlanner = new MultiPartitionPlanner(singleRemotePartitionLocationProvider, singlePartitionPlanner, - "localPartition", dataset, queryConfig) - val oneRemoteShardKeyRegexPlanner = new ShardKeyRegexPlanner(dataset, oneRemoteMpPlanner, twoRemoteShardKeyMatcherFn, queryConfig) + val twoRemoteMpPlanner = new MultiPartitionPlanner(twoRemotePartitionLocationProvider, twoRemotePlanners.spp, + "localPartition", dataset, queryConfig, shardKeyMatcher = twoRemoteShardKeyMatcherFn) + val oneRemoteShardKeyRegexPlanner = new ShardKeyRegexPlanner(dataset, twoRemoteMpPlanner, twoRemoteShardKeyMatcherFn, twoRemotePartitionLocationProvider, queryConfig) val query9 = """topk(2, test{_ws_ = "demo", _ns_ =~ "remoteNs.*", instance = "Inst-1"})""" val lp9 = Parser.queryRangeToLogicalPlan(query9, TimeStepParams(startSeconds, step, endSeconds), Antlr) @@ -2694,11 +2677,12 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS // Case 10: MPP configured not to use grpc but partition assignment has grpc endpoint configured - val twoGrpcRemoteMpPlannerNoGrpc = new MultiPartitionPlanner(rwoRemoteGrpcPartitionLocationProvider, singlePartitionPlanner, + val twoGrpcRemoteMpPlannerNoGrpc = new MultiPartitionPlanner(rwoRemoteGrpcPartitionLocationProvider, twoRemotePlanners.spp, "localPartition", dataset, - queryConfig.copy(grpcPartitionsDenyList = Set("remotepartition1"))) + queryConfig.copy(grpcPartitionsDenyList = Set("remotepartition1")), + shardKeyMatcher = twoRemoteShardKeyMatcherFn) val twoGrpcRemoteShardKeyRegexPlannerNoGrpc = - new ShardKeyRegexPlanner(dataset, twoGrpcRemoteMpPlannerNoGrpc, twoRemoteShardKeyMatcherFn, queryConfig) + new ShardKeyRegexPlanner(dataset, twoGrpcRemoteMpPlannerNoGrpc, twoRemoteShardKeyMatcherFn, rwoRemoteGrpcPartitionLocationProvider, queryConfig) val query10 = """sum(test{_ws_ = "demo", _ns_ =~ "remoteNs.*", instance = "Inst-1"})""" val lp10 = @@ -2714,9 +2698,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(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 + |-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 validatePlan(execPlan10, expectedPlan10) } @@ -3477,7 +3461,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS } val multiPartitionPlanner = new MultiPartitionPlanner(partitionLocationProvider, singlePartitionPlanner, "local", dataset, queryConfig) - val shardKeyRegexPlanner = new ShardKeyRegexPlanner(dataset, multiPartitionPlanner, shardKeyMatcherFunc, queryConfig, targetSchemaProvider) + val shardKeyRegexPlanner = new ShardKeyRegexPlanner(dataset, multiPartitionPlanner, shardKeyMatcherFunc, partitionLocationProvider, queryConfig, targetSchemaProvider) val tschema = FunctionalTargetSchemaProvider(tschemaProviderFunc) diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlannerSpec.scala index 3af295d6de..d06f7a965e 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlannerSpec.scala @@ -3,22 +3,21 @@ 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 +import filodb.core.{MetricsTestData, SpreadChange} import filodb.core.metadata.Schemas import filodb.prometheus.ast.TimeStepParams 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.core.query.Filter.{Equals, EqualsRegex} 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 { @@ -39,13 +38,16 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture private val localMapper = new ShardMapper(32) for {i <- 0 until 32} localMapper.registerNode(Seq(i), node) - - val localPlanner = new SingleClusterPlanner(dataset, schemas, localMapper, earliestRetainedTimestampFn = 0, - queryConfig, "raw") + 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))) + } def partitions(timeRange: TimeRange): List[PartitionAssignment] = List(PartitionAssignment("remote", "remote-url", TimeRange(timeRange.startMs, timeRange.endMs))) - val partitionLocationProvider = new PartitionLocationProvider { + val mppPartitionLocationProvider = new PartitionLocationProvider { override def getPartitions(routingKey: Map[String, String], timeRange: TimeRange): List[PartitionAssignment] = { if (routingKey.equals(Map("_ns_" -> "App-1", "_ws_" -> "demo"))) List(PartitionAssignment("remote", "remote-url", TimeRange(timeRange.startMs, timeRange.endMs))) @@ -56,10 +58,14 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture override def getMetadataPartitions(nonMetricShardKeyFilters: Seq[ColumnFilter], timeRange: TimeRange): List[PartitionAssignment] = partitions(timeRange) } + val simplePartitionLocationProvider = new PartitionLocationProvider { + override def getPartitions(routingKey: Map[String, String], timeRange: TimeRange): List[PartitionAssignment] = + List(PartitionAssignment(s"part-${routingKey("_ns_")}", "http://dummy-url.com", timeRange)) + override def getMetadataPartitions(nonMetricShardKeyFilters: Seq[ColumnFilter], timeRange: TimeRange): List[PartitionAssignment] = + 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( - partitionLocationProvider, localPlanner, "local", dataset, c - ) + val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq( Seq( @@ -72,48 +78,54 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture ) ) } - val skrp = new ShardKeyRegexPlanner(dataset, mpp, shardKeyMatcherFn, queryConfig) + + 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") { val lp = Parser.queryToLogicalPlan("test{_ws_ = \"demo\", _ns_ =~ \"App.*\", instance = \"Inst-1\" }", 1000, 1000) val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-2"))))} - val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) - 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.isInstanceOf[LocalPartitionDistConcatExec] shouldEqual(true) + execPlan.children(0).isInstanceOf[MultiSchemaPartitionsExec] + execPlan.children(1).asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", EqualsRegex("App.*"))) shouldEqual(true) + execPlan.children(0).asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", EqualsRegex("App.*"))) shouldEqual(true) } - it("should generate Exec plan for implicit ws query") { - val lp = Parser.queryToLogicalPlan("test{_ns_ =~ \"App.*\", instance = \"Inst-1\" }", 1000, 1000) - val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), - ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), - ColumnFilter("_ns_", Equals("App-2"))))} - val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) - val execPlan = engine.materialize(lp, QueryContext(origQueryParams = PromQlQueryParams( - """test{_ns_ =~ "App.*", instance = "Inst-1" }""", 100, 1, 1000))) - 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) - } +// // TODO: is this still necessary? +// it("should generate Exec plan for implicit ws query") { +// val lp = Parser.queryToLogicalPlan("test{_ns_ =~ \"App.*\", instance = \"Inst-1\" }", 1000, 1000) +// val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), +// ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), +// ColumnFilter("_ns_", Equals("App-2"))))} +// val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) +// val execPlan = engine.materialize(lp, QueryContext(origQueryParams = PromQlQueryParams( +// """test{_ns_ =~ "App.*", instance = "Inst-1" }""", 100, 1, 1000))) +// execPlan.isInstanceOf[LocalPartitionDistConcatExec] shouldEqual(true) +// execPlan.children(0).isInstanceOf[MultiSchemaPartitionsExec] +// execPlan.children(1).asInstanceOf[MultiSchemaPartitionsExec].filters. +// contains(ColumnFilter("_ns_", EqualsRegex("App.*"))) shouldEqual(true) +// execPlan.children(0).asInstanceOf[MultiSchemaPartitionsExec].filters. +// contains(ColumnFilter("_ns_", EqualsRegex("App.*"))) shouldEqual(true) +// execPlan.children(0).asInstanceOf[MultiSchemaPartitionsExec].filters. +// contains(ColumnFilter("_ws_", Equals("demo"))) shouldEqual(true) +// execPlan.children(1).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 engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val nonMetricShardColumns = dataset.options.nonMetricShardColumns val implicitLp = Parser.queryToLogicalPlan("test{_ns_ =~ \"App.*\", instance = \"Inst-1\" }", 1000, 1000) @@ -128,20 +140,24 @@ 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=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 + """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"},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,false,true,false,false,true), 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 ) - val engine = new ShardKeyRegexPlanner(dataset, mpp, shardKeyMatcherFn, queryConfig) + val engine = new ShardKeyRegexPlanner(dataset, mpp, shardKeyMatcherFn, mppPartitionLocationProvider, queryConfig) val execPlan = engine.materialize( lp, QueryContext(origQueryParams = promQlQueryParams, plannerParams = PlannerParams(processMultiPartition = true)) @@ -159,14 +175,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 engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) - 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.isInstanceOf[LocalPartitionDistConcatExec] shouldEqual(true) + execPlan.children(0).isInstanceOf[MultiSchemaPartitionsExec] + execPlan.children(1).asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", EqualsRegex("App.*"))) shouldEqual(true) + execPlan.children(0).asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", EqualsRegex("App.*"))) shouldEqual(true) } it("should generate Exec plan for subquery with windowing with aggregate") { @@ -177,15 +193,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 engine = new ShardKeyRegexPlanner( dataset, localPlanner, shardKeyMatcherFn, queryConfig) + val engine = new ShardKeyRegexPlanner( dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = PromQlQueryParams("sum(heap_usage)", 100, 1, 1000))) - 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) + execPlan.isInstanceOf[LocalPartitionReduceAggregateExec] shouldEqual(true) + execPlan.children(0).isInstanceOf[MultiSchemaPartitionsExec] + execPlan.children(1).asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", EqualsRegex("App.*"))) shouldEqual(true) + execPlan.children(0).asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", EqualsRegex("App.*"))) shouldEqual(true) } it("should generate Exec plan for top level subquery") { @@ -193,10 +209,14 @@ 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=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) + |---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=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 + |---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"},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,false,true,false,false,true), queryEndpoint=remote-url, requestTimeoutMs=60000) on InProcessPlanDispatcher""".stripMargin val lp = Parser.queryToLogicalPlan( """test{_ws_ = "demo", _ns_ =~ "App.*", instance = "Inst-1" }[5m:1m]""", 1000, 1000 @@ -213,21 +233,21 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-2"))))} - val engine = new ShardKeyRegexPlanner( dataset, localPlanner, shardKeyMatcherFn, queryConfig) + val engine = new ShardKeyRegexPlanner( dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = PromQlQueryParams("sum(heap_usage)", 100, 1, 1000))) - 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) + execPlan.isInstanceOf[LocalPartitionReduceAggregateExec] shouldEqual(true) + execPlan.children(0).isInstanceOf[MultiSchemaPartitionsExec] + execPlan.children(1).asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", EqualsRegex("App.*"))) shouldEqual(true) + execPlan.children(0).asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", EqualsRegex("App.*"))) 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 engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) execPlan.isInstanceOf[TimeScalarGeneratorExec] shouldEqual(true) } @@ -238,23 +258,22 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-2"))))} - val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) + 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))) - execPlan.isInstanceOf[MultiPartitionDistConcatExec] shouldEqual(true) + execPlan.isInstanceOf[LocalPartitionDistConcatExec] shouldEqual(true) execPlan.rangeVectorTransformers(0).isInstanceOf[ScalarOperationMapper] shouldEqual true - execPlan.children(0).children.head.isInstanceOf[MultiSchemaPartitionsExec] - + execPlan.children(0).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) + execPlan.children(1).queryContext.origQueryParams.asInstanceOf[PromQlQueryParams].promQl shouldEqual + "test{_ws_=\"demo\",_ns_=~\"App.*\",instance=\"Inst-1\"}" + execPlan.children(0).queryContext.origQueryParams.asInstanceOf[PromQlQueryParams].promQl shouldEqual + "test{_ws_=\"demo\",_ns_=~\"App.*\",instance=\"Inst-1\"}" + execPlan.children(0).rangeVectorTransformers(0).isInstanceOf[PeriodicSamplesMapper] shouldEqual true + execPlan.children(1).asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", EqualsRegex("App.*"))) shouldEqual(true) + execPlan.children(0).asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", EqualsRegex("App.*"))) shouldEqual(true) } it("should generate Exec plan for Binary join without regex") { @@ -263,7 +282,7 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-2"))))} - val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) + 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))) execPlan.isInstanceOf[BinaryJoinExec] shouldEqual(true) @@ -277,7 +296,7 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-2"))))} - val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) execPlan.isInstanceOf[PartKeysDistConcatExec] shouldEqual (true) } @@ -288,32 +307,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 engine = new ShardKeyRegexPlanner( dataset, localPlanner, shardKeyMatcherFn, queryConfig) + val engine = new ShardKeyRegexPlanner( dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) - execPlan.isInstanceOf[MultiPartitionReduceAggregateExec] shouldEqual(true) - execPlan.asInstanceOf[MultiPartitionReduceAggregateExec].rangeVectorTransformers(0). + execPlan.isInstanceOf[LocalPartitionReduceAggregateExec] shouldEqual(true) + execPlan.asInstanceOf[LocalPartitionReduceAggregateExec].rangeVectorTransformers(0). isInstanceOf[AggregatePresenter] shouldEqual true - execPlan.asInstanceOf[MultiPartitionReduceAggregateExec].rangeVectorTransformers(1). + execPlan.asInstanceOf[LocalPartitionReduceAggregateExec].rangeVectorTransformers(1). isInstanceOf[InstantVectorFunctionMapper] shouldEqual true - execPlan.asInstanceOf[MultiPartitionReduceAggregateExec].rangeVectorTransformers(1). + execPlan.asInstanceOf[LocalPartitionReduceAggregateExec].rangeVectorTransformers(1). asInstanceOf[InstantVectorFunctionMapper].function shouldEqual HistogramQuantile - execPlan.children(0).children.head.isInstanceOf[MultiSchemaPartitionsExec] + execPlan.children(0).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) + execPlan.children(0).rangeVectorTransformers.length shouldEqual 2 + execPlan.children(0).rangeVectorTransformers(0).isInstanceOf[PeriodicSamplesMapper] shouldEqual true + execPlan.children(0).rangeVectorTransformers(1).isInstanceOf[AggregateMapReduce] shouldEqual true + execPlan.children(1).asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", EqualsRegex("App.*"))) shouldEqual(true) + execPlan.children(0).asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", EqualsRegex("App.*"))) 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"})""" + execPlan.children(1).queryContext.origQueryParams.asInstanceOf[PromQlQueryParams].promQl shouldEqual + """sum(test{_ws_="demo",_ns_=~"App.*"})""" + execPlan.children(0).queryContext.origQueryParams.asInstanceOf[PromQlQueryParams].promQl shouldEqual + """sum(test{_ws_="demo",_ns_=~"App.*"})""" } it("should generate Exec plan for exp for Aggregate query") { @@ -322,39 +341,39 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-2"))))} - val engine = new ShardKeyRegexPlanner( dataset, localPlanner, shardKeyMatcherFn, queryConfig) + val engine = new ShardKeyRegexPlanner( dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) - execPlan.isInstanceOf[MultiPartitionReduceAggregateExec] shouldEqual(true) - execPlan.asInstanceOf[MultiPartitionReduceAggregateExec].rangeVectorTransformers(0). + execPlan.isInstanceOf[LocalPartitionReduceAggregateExec] shouldEqual(true) + execPlan.asInstanceOf[LocalPartitionReduceAggregateExec].rangeVectorTransformers(0). isInstanceOf[AggregatePresenter] shouldEqual true - execPlan.asInstanceOf[MultiPartitionReduceAggregateExec].rangeVectorTransformers(1). + execPlan.asInstanceOf[LocalPartitionReduceAggregateExec].rangeVectorTransformers(1). isInstanceOf[InstantVectorFunctionMapper] shouldEqual true - execPlan.asInstanceOf[MultiPartitionReduceAggregateExec].rangeVectorTransformers(1). + execPlan.asInstanceOf[LocalPartitionReduceAggregateExec].rangeVectorTransformers(1). asInstanceOf[InstantVectorFunctionMapper].function shouldEqual Exp - execPlan.children(0).children.head.isInstanceOf[MultiSchemaPartitionsExec] + execPlan.children(0).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) + execPlan.children(0).rangeVectorTransformers.length shouldEqual 2 + execPlan.children(0).rangeVectorTransformers(0).isInstanceOf[PeriodicSamplesMapper] shouldEqual true + execPlan.children(0).rangeVectorTransformers(1).isInstanceOf[AggregateMapReduce] shouldEqual true + execPlan.children(1).asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", EqualsRegex("App.*"))) shouldEqual(true) + execPlan.children(0).asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", EqualsRegex("App.*"))) 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"})""" + execPlan.children(1).queryContext.origQueryParams.asInstanceOf[PromQlQueryParams].promQl shouldEqual + """sum(test{_ws_="demo",_ns_=~"App.*"})""" + execPlan.children(0).queryContext.origQueryParams.asInstanceOf[PromQlQueryParams].promQl shouldEqual + """sum(test{_ws_="demo",_ns_=~"App.*"})""" } 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 engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) execPlan.isInstanceOf[LocalPartitionDistConcatExec] shouldEqual(true) execPlan.children(0).isInstanceOf[MultiSchemaPartitionsExec] @@ -369,7 +388,7 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-2"))))} - val engine = new ShardKeyRegexPlanner( dataset, localPlanner, shardKeyMatcherFn, queryConfig) + val engine = new ShardKeyRegexPlanner( dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) execPlan.isInstanceOf[TimeScalarGeneratorExec] shouldEqual(true) execPlan.rangeVectorTransformers.head.isInstanceOf[ScalarOperationMapper] shouldEqual true @@ -378,22 +397,22 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture 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 + asInstanceOf[ExecPlanFuncArgs].execPlan.isInstanceOf[LocalPartitionDistConcatExec] 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"}""" + multiPartitionExec.children(1).queryContext.origQueryParams + .asInstanceOf[PromQlQueryParams].promQl shouldEqual """test{_ws_="demo",_ns_=~"App.*"}""" + multiPartitionExec.children(0).queryContext.origQueryParams + .asInstanceOf[PromQlQueryParams].promQl shouldEqual """test{_ws_="demo",_ns_=~"App.*"}""" } it ("should generate Exec plan for Metadata Label values query") { val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => Seq.empty - val engine = new ShardKeyRegexPlanner( dataset, localPlanner, shardKeyMatcherFn, queryConfig) + val engine = new ShardKeyRegexPlanner( dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val lp = Parser.labelValuesQueryToLogicalPlan(Seq("""__metric__"""), Some("""_ws_="demo", _ns_=~".*" """), TimeStepParams(1000, 20, 5000) ) @@ -408,7 +427,7 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture it ("should generate ExecPlan for TsCardinalities") { val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => Nil - val engine = new ShardKeyRegexPlanner( dataset, localPlanner, shardKeyMatcherFn, queryConfig) + val engine = new ShardKeyRegexPlanner( dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val lp = TsCardinalities(Seq("ws_foo", "ns_bar"), 3) val promQlQueryParams = PromQlQueryParams( "", 1000, 20, 5000, Some("/api/v1/metering/cardinality/timeseries")) @@ -425,23 +444,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 engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) execPlan.isInstanceOf[BinaryJoinExec] shouldEqual(true) - execPlan.children(0).isInstanceOf[MultiPartitionDistConcatExec] shouldEqual(true) - val lhs = execPlan.children(0).asInstanceOf[MultiPartitionDistConcatExec] - lhs.children.length shouldEqual 2 - lhs.children(0).children.head.isInstanceOf[MultiSchemaPartitionsExec] shouldEqual true - lhs.children(1).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. - contains(ColumnFilter("_ns_", Equals("App-1"))) shouldEqual(true) - lhs.children(0).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. - contains(ColumnFilter("_ns_", Equals("App-2"))) shouldEqual(true) - val rhs = execPlan.children(1).asInstanceOf[MultiPartitionDistConcatExec] - rhs.children(0).children.head.isInstanceOf[MultiSchemaPartitionsExec] shouldEqual true - rhs.children(1).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. - contains(ColumnFilter("_ns_", Equals("App-1"))) shouldEqual(true) - rhs.children(0).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. - contains(ColumnFilter("_ns_", Equals("App-2"))) shouldEqual(true) + execPlan.children(0).isInstanceOf[LocalPartitionDistConcatExec] shouldEqual(true) + val lhs = execPlan.children(0).asInstanceOf[LocalPartitionDistConcatExec] + lhs.children.length shouldEqual 4 + lhs.children(0).isInstanceOf[MultiSchemaPartitionsExec] shouldEqual true + lhs.children(1).asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", EqualsRegex("App.*"))) shouldEqual(true) + lhs.children(0).asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", EqualsRegex("App.*"))) shouldEqual(true) + val rhs = execPlan.children(1).asInstanceOf[LocalPartitionDistConcatExec] + rhs.children(0).isInstanceOf[MultiSchemaPartitionsExec] shouldEqual true + rhs.children(1).asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", EqualsRegex("App.*"))) shouldEqual(true) + rhs.children(0).asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", EqualsRegex("App.*"))) shouldEqual(true) } @@ -454,23 +473,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 engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) execPlan.isInstanceOf[BinaryJoinExec] shouldEqual(true) - execPlan.children(0).isInstanceOf[MultiPartitionDistConcatExec] shouldEqual(true) - val lhs = execPlan.children(0).asInstanceOf[MultiPartitionDistConcatExec] - lhs.children.length shouldEqual 2 - lhs.children(0).children.head.isInstanceOf[MultiSchemaPartitionsExec] shouldEqual true - lhs.children(1).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. - contains(ColumnFilter("_ns_", Equals("App-1"))) shouldEqual(true) - lhs.children(0).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. - contains(ColumnFilter("_ns_", Equals("App-2"))) shouldEqual(true) - val rhs = execPlan.children(1).asInstanceOf[MultiPartitionDistConcatExec] - rhs.children(0).children.head.isInstanceOf[MultiSchemaPartitionsExec] shouldEqual true - rhs.children(1).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. - contains(ColumnFilter("_ns_", Equals("App-1"))) shouldEqual(true) - rhs.children(0).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. - contains(ColumnFilter("_ns_", Equals("App-2"))) shouldEqual(true) + execPlan.children(0).isInstanceOf[LocalPartitionDistConcatExec] shouldEqual(true) + val lhs = execPlan.children(0).asInstanceOf[LocalPartitionDistConcatExec] + lhs.children.length shouldEqual 4 + lhs.children(0).isInstanceOf[MultiSchemaPartitionsExec] shouldEqual true + lhs.children(1).asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", EqualsRegex("App.*"))) shouldEqual(true) + lhs.children(0).asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", EqualsRegex("App.*"))) shouldEqual(true) + val rhs = execPlan.children(1).asInstanceOf[LocalPartitionDistConcatExec] + rhs.children(0).isInstanceOf[MultiSchemaPartitionsExec] shouldEqual true + rhs.children(1).asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", EqualsRegex("App.*"))) shouldEqual(true) + rhs.children(0).asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", EqualsRegex("App.*"))) shouldEqual(true) } it("should generate Exec plan for Binary join with regex only on one side") { @@ -482,19 +501,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 engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) execPlan.isInstanceOf[BinaryJoinExec] shouldEqual (true) execPlan.children(0).isInstanceOf[LocalPartitionDistConcatExec] shouldEqual (true) val lhs = execPlan.children(0).asInstanceOf[LocalPartitionDistConcatExec] lhs.children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. contains(ColumnFilter("_ns_", Equals("App-0"))) shouldEqual (true) - val rhs = execPlan.children(1).asInstanceOf[MultiPartitionDistConcatExec] - rhs.children(0).children.head.isInstanceOf[MultiSchemaPartitionsExec] shouldEqual true - rhs.children(1).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. - contains(ColumnFilter("_ns_", Equals("App-1"))) shouldEqual (true) - rhs.children(0).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. - contains(ColumnFilter("_ns_", Equals("App-2"))) shouldEqual (true) + val rhs = execPlan.children(1).asInstanceOf[LocalPartitionDistConcatExec] + rhs.children(0).isInstanceOf[MultiSchemaPartitionsExec] shouldEqual true + rhs.children(1).asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", EqualsRegex("App.*"))) shouldEqual (true) + rhs.children(0).asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", EqualsRegex("App.*"))) shouldEqual (true) } it("should generate Exec plan for topk query with single matching value for regex") { @@ -504,7 +523,7 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture Seq(Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-1")))) } - val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) execPlan.isInstanceOf[LocalPartitionReduceAggregateExec] shouldEqual (true) } @@ -518,9 +537,9 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-2")))) } - val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) - execPlan.isInstanceOf[MultiPartitionReduceAggregateExec] shouldEqual true + execPlan.isInstanceOf[LocalPartitionReduceAggregateExec] shouldEqual true } @@ -531,7 +550,7 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture Seq(Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-1")))) } - val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) execPlan.isInstanceOf[LocalPartitionReduceAggregateExec] shouldEqual (true) execPlan.asInstanceOf[LocalPartitionReduceAggregateExec].rangeVectorTransformers(0). @@ -544,7 +563,7 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture execPlan.children.head.isInstanceOf[MultiSchemaPartitionsExec] execPlan.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams].promQl shouldEqual - """histogram_quantile(0.2,sum(test{_ws_="demo",_ns_="App-1"}))""" + """histogram_quantile(0.2,sum(test{_ws_="demo",_ns_=~"App-1"}))""" execPlan.queryContext.plannerParams.skipAggregatePresent shouldEqual (false) } @@ -557,11 +576,11 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture Seq(Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App")))) } - val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) execPlan.isInstanceOf[BinaryJoinExec] shouldEqual (true) execPlan.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams].promQl shouldEqual - ("""(test1{_ws_="demo",_ns_="App"} + test2{_ws_="demo",_ns_="App"})""") + ("""(test1{_ws_="demo",_ns_=~"App"} + test2{_ws_="demo",_ns_=~"App"})""") } it("should preserve brackets in Binary join with regex query") { @@ -572,11 +591,11 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture Seq(Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App")))) } - val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) 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"})))""") + ("""(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") { @@ -591,27 +610,22 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture ) ) } - val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) 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.isInstanceOf[LocalPartitionReduceAggregateExec] shouldEqual true + mpExec.asInstanceOf[LocalPartitionReduceAggregateExec].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") + case children: Seq[ExecPlan] if children.size == 4 => + children.map(_.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams].promQl).toSet shouldEqual Set("""count(test1{_ws_="demo",_ns_=~"App-.*"}) by (foo)""") + children.head.isInstanceOf[MultiSchemaPartitionsExec] shouldEqual true + case _ => fail("Expected four children") } } @@ -625,13 +639,13 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture ColumnFilter("_ns_", Equals("App1"))) ) } - val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) // 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))""" + """sum(count(test1{_ws_="demo",_ns_=~"App-.*"}) by (foo))""" } it("should handle multiple binary joins appropriately, case 1") { @@ -647,7 +661,7 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture ColumnFilter("_ns_", Equals("App1"))) ) } - val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) + 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))) @@ -670,7 +684,7 @@ 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 engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) execPlan.isInstanceOf[BinaryJoinExec] shouldEqual true val inJoin = execPlan.asInstanceOf[BinaryJoinExec] @@ -698,19 +712,14 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture ) } - val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) - 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.isInstanceOf[LocalPartitionDistConcatExec] 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") + case children: Seq[ExecPlan] if children.size == 4 => + children.map(_.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams].promQl).toSet shouldEqual Set("""test1{_ws_="demo",_ns_=~"App-1"}""") + children.head.isInstanceOf[MultiSchemaPartitionsExec] shouldEqual true + case _ => fail("Expected four children") } execPlan.rangeVectorTransformers.size shouldEqual 1 val transformer = execPlan.rangeVectorTransformers.head @@ -733,23 +742,18 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture ) ) } - val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) - execPlan.isInstanceOf[MultiPartitionDistConcatExec] shouldEqual (true) - // Since the dispatcher is ActorDispatcher, its materialized by the underlying queryPlanner - execPlan.dispatcher.isInstanceOf[InProcessPlanDispatcher] shouldEqual (true) + execPlan.isInstanceOf[LocalPartitionDistConcatExec] 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") + case children: Seq[ExecPlan] if children.size == 4 => + children.map(_.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams].promQl).toSet shouldEqual Set("""test1{_ws_="demo",_ns_=~"App-*"}""") + children.head.isInstanceOf[MultiSchemaPartitionsExec] shouldEqual true + case _ => fail("Expected four children") } } @@ -758,7 +762,7 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture 1000, 1000) val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => Seq(shardColumnFilters) - val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) + 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))) execPlan.isInstanceOf[LocalPartitionDistConcatExec] shouldEqual (true) @@ -779,7 +783,7 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture 1000, 1000) val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => Seq(shardColumnFilters) - val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) + 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))) execPlan.isInstanceOf[LocalPartitionReduceAggregateExec] shouldEqual (true) @@ -808,29 +812,23 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture ) } - val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) - execPlan.isInstanceOf[MultiPartitionDistConcatExec] shouldEqual (true) - execPlan.dispatcher.isInstanceOf[InProcessPlanDispatcher] shouldEqual (true) - execPlan.children.size shouldEqual 2 + execPlan.isInstanceOf[LocalPartitionDistConcatExec] shouldEqual (true) + execPlan.children.size shouldEqual 4 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.isInstanceOf[MultiSchemaPartitionsExec] 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") + case children: Seq[ExecPlan] if children.size == 4 => + children.map(_.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams].promQl).toSet shouldEqual Set("""foo{_ws_="Demo",_ns_=~"App*"}""") + children.head.isInstanceOf[MultiSchemaPartitionsExec] shouldEqual true + case _ => fail("Expected four children") } } it("should generate the appropriate plan with sort") { @@ -845,29 +843,24 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture ) } - val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) - execPlan.isInstanceOf[MultiPartitionDistConcatExec] shouldEqual (true) - execPlan.dispatcher.isInstanceOf[InProcessPlanDispatcher] shouldEqual (true) - execPlan.children.size shouldEqual 2 + execPlan.isInstanceOf[LocalPartitionDistConcatExec] shouldEqual (true) + execPlan.children.size shouldEqual 4 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.isInstanceOf[MultiSchemaPartitionsExec] 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") + case children: Seq[ExecPlan] if children.size == 4 => + children.map(_.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams].promQl).toSet shouldEqual Set("""foo{_ws_="Demo",_ns_=~"App*"}""") + children.head.isInstanceOf[MultiSchemaPartitionsExec] shouldEqual true + case _ => fail("Expected four children") } } @@ -884,14 +877,13 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture ) } - val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) + 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)) - execPlan.isInstanceOf[MultiPartitionReduceAggregateExec] shouldEqual (true) - execPlan.dispatcher.isInstanceOf[InProcessPlanDispatcher] shouldEqual (true) + execPlan.isInstanceOf[LocalPartitionReduceAggregateExec] shouldEqual (true) execPlan.asInstanceOf[ReduceAggregateExec].aggrOp shouldEqual Sum - execPlan.children.size shouldEqual 2 + execPlan.children.size shouldEqual 1 execPlan.rangeVectorTransformers.find(_.isInstanceOf[AggregatePresenter]) match { case Some(x: AggregatePresenter) => x.aggrOp shouldEqual Sum case _ => fail("Expected to see an AggregatePresenter") @@ -899,19 +891,15 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture execPlan.children.foreach( x => { x.isInstanceOf[LocalPartitionReduceAggregateExec] shouldEqual true x.dispatcher.isInstanceOf[ActorPlanDispatcher] shouldEqual (true) - x.rangeVectorTransformers.isEmpty shouldEqual true + x.rangeVectorTransformers.isEmpty shouldEqual false 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") + execPlan.children.head.children match { + case children: Seq[ExecPlan] if children.size == 4 => + children.map(_.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams].promQl).toSet shouldEqual Set("""sum(absent(foo{_ws_="Demo",_ns_=~"App*"}))""") + children.head.isInstanceOf[MultiSchemaPartitionsExec] shouldEqual true + case _ => fail("Expected four children") } } @@ -928,11 +916,10 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture ) } - val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) 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 { @@ -940,20 +927,16 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture case _ => fail("Expected to see an AbsentFunctionMapper") } val child = execPlan.children.head - child.isInstanceOf[MultiPartitionDistConcatExec] shouldEqual true - child.dispatcher.isInstanceOf[InProcessPlanDispatcher] shouldEqual (true) + child.isInstanceOf[LocalPartitionDistConcatExec] 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") + execPlan.children.head.children match { + case children: Seq[ExecPlan] if children.size == 4 => + children.map(_.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams].promQl).toSet shouldEqual Set("""foo{_ws_="Demo",_ns_=~"App*"}""") + children.head.isInstanceOf[MultiSchemaPartitionsExec] shouldEqual true + case _ => fail("Expected four children") } } @@ -966,7 +949,7 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture ColumnFilter("_ns_", Equals("App-1")))) } - val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) + 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))) @@ -985,7 +968,7 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture 1000, 1000) val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => Seq(shardColumnFilters) - val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) + 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))) @@ -1016,14 +999,13 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture ) } - val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) // Entre Plan should be materialized on the wrapped - execPlan.isInstanceOf[MultiPartitionReduceAggregateExec] shouldEqual (true) - execPlan.dispatcher.isInstanceOf[InProcessPlanDispatcher] shouldEqual (true) + execPlan.isInstanceOf[LocalPartitionReduceAggregateExec] shouldEqual (true) execPlan.asInstanceOf[ReduceAggregateExec].aggrOp shouldEqual Sum - execPlan.children.size shouldEqual 2 + execPlan.children.size shouldEqual 4 execPlan.rangeVectorTransformers.contains(VectorFunctionMapper()) shouldEqual true execPlan.rangeVectorTransformers.find(_.isInstanceOf[ScalarFunctionMapper]) match { case Some(x: ScalarFunctionMapper) => @@ -1031,134 +1013,132 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture } 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") { - { - // absent of sum_over_time: _ws_ in query - val lp = Parser.queryToLogicalPlan( - "absent(sum_over_time(test{_ws_ = \"demo\", _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, 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, 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( - "absent(test{_ws_ = \"demo\", _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, 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, 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( - "absent_over_time(test{_ws_ = \"demo\", _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, 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, queryConfig) - val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) - execPlan.asInstanceOf[LocalPartitionReduceAggregateExec].rangeVectorTransformers.head - .asInstanceOf[AbsentFunctionMapper].columnFilter.size shouldEqual 4 // _ws_, _ns_, __name__ & instance + case children: Seq[ExecPlan] if children.size == 4 => + children.map(_.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams].promQl).toSet shouldEqual Set("""sum(foo{_ws_="demo",_ns_=~"App*"})""") + children.head.isInstanceOf[MultiSchemaPartitionsExec] shouldEqual true + case _ => fail("Expected four children") } } - 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) - val queryExpectedPairs = Seq( - // inst func with scalar() arg: _ws_ in query - (s"""clamp_max(test{_ws_="demo",_ns_="App-1"}, scalar(sc_test{_ws_="demo",_ns_="App-1"}))""", - """T~InstantVectorFunctionMapper(function=ClampMax) - |-FA1~ - |-T~ScalarFunctionMapper(function=Scalar, funcParams=List()) - |--E~LocalPartitionDistConcatExec() 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=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/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=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/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 engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) - - queryExpectedPairs.foreach{ case (query, expected) => - val lp = Parser.queryToLogicalPlan(query, 1000, 1000) - val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) - execPlan.isInstanceOf[DistConcatExec] shouldEqual true - validatePlan(execPlan.children.head, expected) - } - } + // TODO: is this still needed? +// it("should materialize absent function mapper correctly with implicit WS") { +// { +// // absent of sum_over_time: _ws_ in query +// val lp = Parser.queryToLogicalPlan( +// "absent(sum_over_time(test{_ws_ = \"demo\", _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 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( +// "absent(test{_ws_ = \"demo\", _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( +// "absent_over_time(test{_ws_ = \"demo\", _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: is this still needed? +// 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) +// val queryExpectedPairs = Seq( +// // inst func with scalar() arg: _ws_ in query +// (s"""clamp_max(test{_ws_="demo",_ns_="App-1"}, scalar(sc_test{_ws_="demo",_ns_="App-1"}))""", +// """T~InstantVectorFunctionMapper(function=ClampMax) +// |-FA1~ +// |-T~ScalarFunctionMapper(function=Scalar, funcParams=List()) +// |--E~LocalPartitionDistConcatExec() 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=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/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=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/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 engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) +// +// queryExpectedPairs.foreach{ case (query, expected) => +// val lp = Parser.queryToLogicalPlan(query, 1000, 1000) +// val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) +// execPlan.isInstanceOf[DistConcatExec] shouldEqual true +// validatePlan(execPlan.children.head, expected) +// } +// } } diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/SingleClusterPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/SingleClusterPlannerSpec.scala index 4d830061f8..b0b12dfa0e 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/SingleClusterPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/SingleClusterPlannerSpec.scala @@ -24,6 +24,7 @@ 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 { @@ -40,11 +41,19 @@ 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") + queryConfig, "raw", shardKeyMatcher = regexPipeShardKeyMatcher) /* This is the PromQL @@ -373,7 +382,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.children.size) + println(execPlan) execPlan.rangeVectorTransformers.head.isInstanceOf[StitchRvsMapper] shouldEqual true } @@ -416,7 +425,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 = true) + ), qContext, start, end, useTargetSchemaForShards = (filts) => true) } should contain theSameElementsAs Seq(13, 20, 13, 20) } @@ -1210,6 +1219,7 @@ 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/query/src/test/scala/filodb/query/PlanValidationSpec.scala b/query/src/test/scala/filodb/query/PlanValidationSpec.scala index 93785b4752..c51a898531 100644 --- a/query/src/test/scala/filodb/query/PlanValidationSpec.scala +++ b/query/src/test/scala/filodb/query/PlanValidationSpec.scala @@ -1,11 +1,13 @@ package filodb.query +import com.typesafe.scalalogging.StrictLogging import org.scalatest.matchers.should.Matchers import filodb.query.exec.ExecPlan + import scala.collection.mutable -trait PlanValidationSpec extends Matchers { +trait PlanValidationSpec extends Matchers with StrictLogging { private def getIndent(line: String): Int = { line.prefixLength(c => c == '-') @@ -20,7 +22,7 @@ trait PlanValidationSpec extends Matchers { /** * Prints the same tree for all logically-identical plans. */ - private def sortTree(planString: String): Unit = { + private def sortTree(planString: String): String = { // Returns the index to resume reading from "lines" and a sorted string tree. def helper(lines: Seq[String], index: Int): (Int, String) = { // Recursively build a list of child strings, then sort and concatenate. @@ -61,6 +63,16 @@ trait PlanValidationSpec extends Matchers { (denoisedPlan, denoisedExpected) } } + if (planString != expectedString) { + logger.error( + s"""======== PLAN VALIDATION FAILED ======== + |~~~~~~~~~~~~~~~ EXPECTED ~~~~~~~~~~~~~~~ + |$expectedString + |~~~~~~~~~~~~~~~~ ACTUAL ~~~~~~~~~~~~~~~~ + |$planString + |""".stripMargin + ) + } planString shouldEqual expectedString } From f526e7a5e2a21b7aa4cae0db73c1ab4d672c0867 Mon Sep 17 00:00:00 2001 From: Alex Theimer Date: Fri, 29 Sep 2023 01:22:36 -0700 Subject: [PATCH 03/27] auto-fix test code --- .../queryplanner/PlannerHierarchySpec.scala | 2807 +++++++++-------- 1 file changed, 1477 insertions(+), 1330 deletions(-) diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala index f6e734a918..809998904b 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala @@ -7,7 +7,7 @@ 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} +import filodb.coordinator.client.QueryCommands.FunctionalTargetSchemaProvider import filodb.core.{MetricsTestData, StaticTargetSchemaProvider, TargetSchemaChange} import filodb.core.metadata.{Dataset, Schemas} import filodb.core.query._ @@ -18,7 +18,6 @@ import filodb.prometheus.parse.Parser.Antlr import filodb.query.{BadQueryException, IntervalSelector, LabelCardinality, PlanValidationSpec, RawSeries} import filodb.query.exec._ - // scalastyle:off line.size.limit // scalastyle:off number.of.methods class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationSpec { @@ -442,272 +441,251 @@ 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""", - // 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. + """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), ("""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,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. + """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#-585347165],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#-585347165],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#-585347165],raw) + |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-585347165],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#-585347165],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#-585347165],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), ("""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)) - |-E~BinaryJoinExec(binaryOp=ADD, on=List(), 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)) - |---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#1132741402],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#1132741402],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#1132741402],raw) - |----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1132741402],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#1132741402],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#1132741402],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)) - |--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#1132741402],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#1132741402],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#1132741402],raw) - |----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1132741402],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#1132741402],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#1132741402],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), - // Aggregate with namespace regex selector. + """T~ScalarOperationMapper(operator=MUL, scalarOnLhs=true) + |-FA1~StaticFuncArgs(5.0,RangeParams(1633913330,300,1634777330)) + |-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#-585347165],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#-585347165],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#-585347165],raw) + |----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-585347165],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#-585347165],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#-585347165],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))) + |--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#-585347165],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#-585347165],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#-585347165],raw) + |----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-585347165],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#-585347165],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#-585347165],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), ("""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,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. + """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#-585347165],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#-585347165],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#-585347165],raw) + |----E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-585347165],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#-585347165],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#-585347165],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), ("""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,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 + """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#-585347165],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#-585347165],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#-585347165],raw) + |----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-585347165],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#-585347165],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#-585347165],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), ("""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,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 + """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#-585347165],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#-585347165],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#-585347165],raw) + |-----E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-585347165],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#-585347165],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#-585347165],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#-585347165],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#-585347165],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#-585347165],raw) + |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-585347165],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#-585347165],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#-585347165],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), ("""foo{_ws_ = "demo", _ns_ =~ ".*Ns", instance = "Inst-1" } > vector(5)""", - """E~BinaryJoinExec(binaryOp=GTR, on=List(), 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)) - |--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#1893185297],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#1893185297],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#1893185297],raw) - |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1893185297],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#1893185297],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#1893185297],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)) - |-T~VectorFunctionMapper(funcParams=List()) - |--E~ScalarFixedDoubleExec(params = RangeParams(1633913330,300,1634777330), value = 5.0) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), - // join multiple adjacent scalars - // NOTE: this can be optimized by pushing down the scalar operation. + """E~BinaryJoinExec(binaryOp=GTR, 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#-585347165],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#-585347165],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#-585347165],raw) + |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-585347165],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#-585347165],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#-585347165],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))) + |-T~VectorFunctionMapper(funcParams=List()) + |--E~ScalarFixedDoubleExec(params = RangeParams(1633913330,300,1634777330), value = 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)))""".stripMargin), ("""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,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. + """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#-585347165],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#-585347165],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#-585347165],raw) + |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-585347165],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#-585347165],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#-585347165],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), ("""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,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) + """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#-585347165],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#-585347165],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#-585347165],raw) + |----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-585347165],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#-585347165],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#-585347165],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), ("""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,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 + """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#-585347165],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#-585347165],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#-585347165],raw) + |----E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-585347165],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#-585347165],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#-585347165],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), ("""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,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 + """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#-585347165],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#-585347165],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#-585347165],raw) + |----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-585347165],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#-585347165],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#-585347165],downsample) + |---E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~".*Ns",instance="Inst-1"},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,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), ("""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,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 + """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#-585347165],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#-585347165],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#-585347165],raw) + |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-585347165],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#-585347165],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#-585347165],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), ("""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,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), + """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#-585347165],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#-585347165],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#-585347165],raw) + |----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-585347165],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#-585347165],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#-585347165],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) ) for ((query, rootPlan) <- queryExpectedPairs) { val queryParams = PromQlQueryParams(query, startSeconds, step, endSeconds) @@ -720,49 +698,34 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS 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,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. + """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), ("""foo{_ws_ = "demo", _ns_ =~ ".*remoteNs", instance = "Inst-1" } + 5""", - // 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. + """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), ("""5 * (foo{_ws_ = "demo", _ns_ =~ ".*remoteNs", instance = "Inst-1" } + foo{_ws_ = "demo", _ns_ =~ ".*remoteNs", instance = "Inst-1" })""", - """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. + """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), ("""5 / sum(foo{_ws_ = "demo", _ns_ =~ ".*remoteNs", instance = "Inst-1" })""", - """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. + """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), ("""sgn(foo{_ws_ = "demo", _ns_ =~ ".*remoteNs", instance = "Inst-1" }) == 5""", - """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 + """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), ("""foo{_ws_ = "demo", _ns_ =~ ".*remoteNs", instance = "Inst-1" } - scalar(sum(foo{_ws_ = "demo", _ns_ =~ ".*remoteNs", instance = "Inst-1" }))""", - """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 + """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), ("""foo{_ws_ = "demo", _ns_ =~ ".*remoteNs", instance = "Inst-1" } > vector(5)""", - """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 + """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), ("""5 * 5 * foo{_ws_ = "demo", _ns_ =~ ".*remoteNs", instance = "Inst-1" }""", - """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 + """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), ("""5 * foo{_ws_ = "demo", _ns_ =~ ".*remoteNs", instance = "Inst-1" } * 5""", - """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) + """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), ("""sum(rate(foo{_ws_ = "demo", _ns_ =~ ".*remoteNs", instance = "Inst-1" }[5m])) > 5""", - """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 + """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), ("""rate(foo{_ws_ = "demo", _ns_ =~ ".*remoteNs", instance = "Inst-1" }[1h:5m]) - 5""", - """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 + """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), ("""5 / count_over_time(foo{_ws_ = "demo", _ns_ =~ ".*remoteNs", instance = "Inst-1" }[20m])""", - """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 + """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), ("""5 <= topk(2, foo{_ws_ = "demo", _ns_ =~ ".*remoteNs", instance = "Inst-1" })""", - """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 + """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), ("""histogram_quantile(0.9, foo{_ws_ = "demo", _ns_ =~ ".*remoteNs", instance = "Inst-1" }) - 5""", - """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), + """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) ) for ((query, otherPlan) <- queryExpectedPairs) { val queryParams = PromQlQueryParams(query, startSeconds, step, endSeconds) @@ -775,117 +738,97 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS 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""", - // 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. + """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), ("""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)) - |-E~BinaryJoinExec(binaryOp=ADD, on=List(), 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)) - |---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)) - |--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), - // Aggregate with namespace regex selector. + """T~ScalarOperationMapper(operator=MUL, scalarOnLhs=true) + |-FA1~StaticFuncArgs(5.0,RangeParams(1633913330,300,1634777330)) + |-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~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))) + |--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), ("""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,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. + """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), ("""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,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 + """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), ("""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,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) + """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), ("""foo{_ws_ = "demo", _ns_ =~ "remoteNs.*", instance = "Inst-1" } > vector(5)""", - """E~BinaryJoinExec(binaryOp=GTR, on=List(), 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)) - |--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)) - |-T~VectorFunctionMapper(funcParams=List()) - |--E~ScalarFixedDoubleExec(params = RangeParams(1633913330,300,1634777330), value = 5.0) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), - // join multiple adjacent scalars - // NOTE: this can be optimized by pushing down the scalar operation. + """E~BinaryJoinExec(binaryOp=GTR, 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~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))) + |-T~VectorFunctionMapper(funcParams=List()) + |--E~ScalarFixedDoubleExec(params = RangeParams(1633913330,300,1634777330), value = 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)))""".stripMargin), ("""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,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. + """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), ("""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,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) + """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), ("""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,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 + """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), ("""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,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 + """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"},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,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"},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,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), ("""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,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 + """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), ("""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,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), + """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) ) for ((query, otherPlan) <- queryExpectedPairs) { val queryParams = PromQlQueryParams(query, startSeconds, step, endSeconds) @@ -1960,158 +1903,174 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS } val tests = Seq( - // aggregate - Test("""sum(test{job="app"} offset 10m)""", + Test(query = """sum(test{job="app"} offset 10m)""", + lookbackSec = 300, offsetSec = 600, - 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))""", + 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),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))""", lookbackSec = 1200, offsetSec = 600, - 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, + 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)),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, offsetSec = 600, - 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]))""", + 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)),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]))""", lookbackSec = 1200, - 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 = 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]))),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, offsetSec = 600, - 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))""", + 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),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))""", lookbackSec = 1200, offsetSec = 600, - 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, + 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)),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, offsetSec = 600, - 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]))""", + 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)),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]))""", lookbackSec = 1200, - 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])""", + 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]))),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"}),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"})),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"})),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])""", lookbackSec = 900, - 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]))""", + 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])),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])""", lookbackSec = 1200, - 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 = 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])),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]))),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]))),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),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, offsetSec = 600, - 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))""", + 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)),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))""", lookbackSec = 1200, offsetSec = 600, - 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, + 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))),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, offsetSec = 600, - 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]))""", + 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))),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]))""", lookbackSec = 1200, - 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 = 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])))),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, offsetSec = 600, - 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))""", + 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),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))""", lookbackSec = 1200, offsetSec = 600, - 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, + 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)),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, offsetSec = 600, - 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]))""", + 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)),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]))""", lookbackSec = 1200, - 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 = 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]))),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, offsetSec = 600, - 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))""", + 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),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))""", lookbackSec = 1200, offsetSec = 600, - 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, + 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)),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, offsetSec = 600, - 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]))""", + 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)),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]))""", lookbackSec = 1200, - 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), + 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]))),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) ) val partitionLocationProvider = new PartitionLocationProvider { override def getPartitions(routingKey: Map[String, String], @@ -2153,8 +2112,6 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS 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 @@ -2182,162 +2139,174 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS } val tests = Seq( - // aggregate - Test("""sum(test{job="app"} offset 10m)""", + Test(query = """sum(test{job="app"} offset 10m)""", + lookbackSec = 300, offsetSec = 600, - 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))""", + 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),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))""", lookbackSec = 1200, offsetSec = 600, - 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, + 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)),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, offsetSec = 600, - 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]))""", + 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)),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]))""", lookbackSec = 1200, - 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 = 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]))),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, offsetSec = 600, - 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))""", + 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),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))""", lookbackSec = 1200, offsetSec = 600, - 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, + 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)),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, offsetSec = 600, - 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]))""", + 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)),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]))""", lookbackSec = 1200, - 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])""", + 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]))),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"}),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"})),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"})),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])""", lookbackSec = 900, - 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]))""", + 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])),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])""", lookbackSec = 1200, - 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 = 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])),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]))),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]))),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),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, offsetSec = 600, - 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))""", + 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)),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))""", lookbackSec = 1200, offsetSec = 600, - 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, + 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))),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, offsetSec = 600, - 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]))""", + 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))),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]))""", lookbackSec = 1200, - 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 = 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])))),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, offsetSec = 600, - 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))""", + 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),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))""", lookbackSec = 1200, offsetSec = 600, - 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, + 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)),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, offsetSec = 600, - 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]))""", + 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)),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]))""", lookbackSec = 1200, - 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 = 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]))),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, offsetSec = 600, - 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))""", + 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),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))""", lookbackSec = 1200, offsetSec = 600, - 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, + 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)),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, offsetSec = 600, - 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]))""", + 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)),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]))""", lookbackSec = 1200, - 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), + 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]))),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) ) val partitionLocationProvider = new PartitionLocationProvider { override def getPartitions(routingKey: Map[String, String], @@ -2379,12 +2348,9 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS 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) } } @@ -2734,693 +2700,606 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS case class Test(query: String, expected: String, tschemaEnabled: Set[String] = Set()) val tests = Seq( - // no namespaces with target-schemas enabled - Test(query=s"""sum(foo{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bar"}) by ($TSCHEMA_LABEL)""", - tschemaEnabled=Set(), - expected="""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,true,false,true,Set(),Some(plannerSelector))) - |--E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,false,false,true,Set(),None)) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#37899380],raw) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#37899380],raw) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#37899380],raw) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#37899380],downsample) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#37899380],downsample) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#37899380],downsample) - |--E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,false,false,true,Set(),None)) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#37899380],raw) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#37899380],raw) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#37899380],raw) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#37899380],downsample) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#37899380],downsample) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#37899380],downsample)""".stripMargin), - // one namespace has a target-schema assigned - Test(query=s"""sum(foo{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bar"}) by ($TSCHEMA_LABEL)""", - tschemaEnabled=Set("local1"), - expected="""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,true,false,true,Set(),Some(plannerSelector))) - |--E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,false,false,true,Set(),None)) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1028628326],raw) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1028628326],raw) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1028628326],raw) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1028628326],downsample) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1028628326],downsample) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1028628326],downsample) - |--E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,false,false,true,Set(),None)) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1028628326],raw) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1028628326],raw) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1028628326],downsample) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1028628326],downsample)""".stripMargin), - // both namespaces are assigned target-schemas; should concat results - Test(query=s"""sum(foo{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bar"}) by ($TSCHEMA_LABEL)""", - tschemaEnabled=Set("local1", "local2"), - expected="""E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |-E~StitchRvsExec() on InProcessPlanDispatcher - |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-E~StitchRvsExec() on InProcessPlanDispatcher - |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw)""".stripMargin), - // remove "by" clause; should not concatenate - Test(query=s"""sum(foo{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bar"})""", - tschemaEnabled=Set("local1", "local2"), - expected="""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,true,false,true,Set(),Some(plannerSelector))) - |--E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,false,false,true,Set(),None)) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#888263698],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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#888263698],raw) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#888263698],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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#888263698],downsample) - |--E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,false,false,true,Set(),None)) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#888263698],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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#888263698],raw) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#888263698],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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#888263698],downsample)""".stripMargin), - // "by" clause is missing a target-schema label - Test(query=s"""sum(foo{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bar"}) by (notATargetSchemaLabel)""", - tschemaEnabled=Set("local1", "local2"), - expected="""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,true,false,true,Set(),Some(plannerSelector))) - |--E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,false,false,true,Set(),None)) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-367138549],raw) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(notATargetSchemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-367138549],raw) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-367138549],downsample) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(notATargetSchemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-367138549],downsample) - |--E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,false,false,true,Set(),None)) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-367138549],raw) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(notATargetSchemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-367138549],raw) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-367138549],downsample) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(notATargetSchemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-367138549],downsample)""".stripMargin), - // remote partition, no target-schemas assigned - Test(query=s"""sum(foo{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"}) by ($TSCHEMA_LABEL)""", - tschemaEnabled=Set(), - expected="""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,true,false,true,Set(),Some(plannerSelector))) - |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{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(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,true,false,true,Set(),Some(plannerSelector))) - |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{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(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,true,false,true,Set(),Some(plannerSelector)))""".stripMargin), - // remote partition, one ns has a target-schema assigned - Test(query=s"""sum(foo{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"}) by ($TSCHEMA_LABEL)""", - tschemaEnabled=Set("oneRemote1"), - expected="""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,true,false,true,Set(),Some(plannerSelector))) - |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{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(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,true,false,true,Set(),Some(plannerSelector))) - |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{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(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,true,false,true,Set(),Some(plannerSelector)))""".stripMargin), - // remote partition, both ns have a target-schema assigned - Test(query=s"""sum(foo{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"}) by ($TSCHEMA_LABEL)""", - tschemaEnabled=Set("oneRemote1", "oneRemote2"), - expected="""E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |-E~PromQlRemoteExec(PromQlQueryParams(sum(foo{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(sum(foo{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), - // two remote partitions, both ns have a target-schema assigned - Test(query=s"""sum(foo{_ws_="demo",_ns_=~"$TWO_REMOTE",$TSCHEMA_LABEL="bar"}) by ($TSCHEMA_LABEL)""", - tschemaEnabled=Set("twoRemote1", "twoRemote2"), - expected="""E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |-E~PromQlRemoteExec(PromQlQueryParams(sum(foo{tschemaLabel="bar",_ws_="demo",_ns_="twoRemote1"}) 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=twoRemote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher - |-E~PromQlRemoteExec(PromQlQueryParams(sum(foo{tschemaLabel="bar",_ws_="demo",_ns_="twoRemote2"}) 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=twoRemote2-url, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin), - // remote partition, both ns have a target-schema assigned, but "by" clause is missing - Test(query=s"""sum(foo{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"})""", - tschemaEnabled=Set("oneRemote1", "oneRemote2"), - expected="""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,true,false,true,Set(),Some(plannerSelector))) - |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{tschemaLabel="bar",_ws_="demo",_ns_="oneRemote1"}),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(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,true,false,true,Set(),Some(plannerSelector))) - |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{tschemaLabel="bar",_ws_="demo",_ns_="oneRemote2"}),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(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,true,false,true,Set(),Some(plannerSelector)))""".stripMargin), - // remote/local partitions, no ns has a tschema assigned - Test(query=s"""sum(foo{_ws_="demo",_ns_=~"$BOTH",$TSCHEMA_LABEL="bar"}) by ($TSCHEMA_LABEL)""", - tschemaEnabled=Set(), - expected="""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,true,false,true,Set(),Some(plannerSelector))) - |--E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,false,false,true,Set(),None)) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-996691824],raw) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-996691824],raw) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-996691824],raw) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-996691824],downsample) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-996691824],downsample) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-996691824],downsample) - |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{tschemaLabel="bar",_ws_="demo",_ns_="remote"}) 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(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,true,false,true,Set(),Some(plannerSelector)))""".stripMargin), - // remote/local partitions, local ns has a tschema assigned - Test(query=s"""sum(foo{_ws_="demo",_ns_=~"$BOTH",$TSCHEMA_LABEL="bar"}) by ($TSCHEMA_LABEL)""", - tschemaEnabled=Set("local"), - expected="""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,true,false,true,Set(),Some(plannerSelector))) - |--E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,false,false,true,Set(),None)) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1673714297],raw) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1673714297],raw) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1673714297],downsample) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1673714297],downsample) - |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{tschemaLabel="bar",_ws_="demo",_ns_="remote"}) 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(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,true,false,true,Set(),Some(plannerSelector)))""".stripMargin), - // remote/local partitions, remote ns has a tschema assigned - Test(query=s"""sum(foo{_ws_="demo",_ns_=~"$BOTH",$TSCHEMA_LABEL="bar"}) by ($TSCHEMA_LABEL)""", - tschemaEnabled=Set("remote"), - expected="""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,true,false,true,Set(),Some(plannerSelector))) - |--E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,false,false,true,Set(),None)) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1678078999],raw) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1678078999],raw) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1678078999],raw) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1678078999],downsample) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1678078999],downsample) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1678078999],downsample) - |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{tschemaLabel="bar",_ws_="demo",_ns_="remote"}) 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(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,true,false,true,Set(),Some(plannerSelector)))""".stripMargin), - // one remote partition, and all ns are assigned a tschema, but tschema label missing from selector - Test(query=s"""sum(foo{_ws_="demo",_ns_=~"$ONE_REMOTE"}) by ($TSCHEMA_LABEL)""", - tschemaEnabled=Set("oneRemote1", "oneRemote2"), - expected="""E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |-E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_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(sum(foo{_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), - // one remote partition, and all ns are assigned a tschema, but tschema label missing from selector and "by" clause - Test(query=s"""sum(foo{_ws_="demo",_ns_=~"$ONE_REMOTE"}) by (notALabel)""", - tschemaEnabled=Set("oneRemote1", "oneRemote2"), - expected="""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,true,false,true,Set(),Some(plannerSelector))) - |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_="oneRemote1"}) by (notALabel),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(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,true,false,true,Set(),Some(plannerSelector))) - |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_="oneRemote2"}) by (notALabel),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(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,true,false,true,Set(),Some(plannerSelector)))""".stripMargin), - // local partition, and all ns are assigned a tschema, but tschema label missing from selector - Test(query=s"""sum(foo{_ws_="demo",_ns_=~"$LOCAL"}) by ($TSCHEMA_LABEL)""", - tschemaEnabled=Set("local1", "local2"), - expected="""E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |-E~StitchRvsExec() on InProcessPlanDispatcher - |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |----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(local1)), 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(tschemaLabel)) - |----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(local1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |----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(local1)), 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(tschemaLabel)) - |----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(local1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-E~StitchRvsExec() on InProcessPlanDispatcher - |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |----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(local2)), 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(tschemaLabel)) - |----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(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |----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(local2)), 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(tschemaLabel)) - |----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(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw)""".stripMargin), - // one remote partition, and all ns are assigned a tschema; nested aggregation with all target-schema labels present in each "by" clause - Test(query=s"""sum(sum(foo{_ws_="demo",_ns_=~"$ONE_REMOTE"}) by ($TSCHEMA_LABEL)) by ($TSCHEMA_LABEL)""", - tschemaEnabled=Set("oneRemote1", "oneRemote2"), - expected="""E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |-E~PromQlRemoteExec(PromQlQueryParams(sum(sum(foo{_ws_="demo",_ns_="oneRemote1"}) by (tschemaLabel)) 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(sum(sum(foo{_ws_="demo",_ns_="oneRemote2"}) by (tschemaLabel)) 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), - // one remote partition, and all ns are assigned a tschema; nested aggregation with target-schema label missing from outer "by" clause - Test(query=s"""sum(sum(foo{_ws_="demo",_ns_=~"$ONE_REMOTE"}) by ($TSCHEMA_LABEL))""", - tschemaEnabled=Set("oneRemote1", "oneRemote2"), - expected="""T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634777330)) - |-E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on InProcessPlanDispatcher - |--T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |---E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |----E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_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(sum(foo{_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), - // one remote partition, and all ns are assigned a tschema; nested aggregation with target-schema label missing from inner "by" clause - Test(query=s"""sum(sum(foo{_ws_="demo",_ns_=~"$ONE_REMOTE"})) by ($TSCHEMA_LABEL)""", - tschemaEnabled=Set("oneRemote1", "oneRemote2"), - expected="""T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634777330)) - |-E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on InProcessPlanDispatcher - |--T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |---T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634777330)) - |----E~MultiPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on InProcessPlanDispatcher - |-----E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_="oneRemote1"}),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(sum(foo{_ws_="demo",_ns_="oneRemote2"}),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), - - // ------- begin binary-join tests -------- - - // no namespaces with target-schemas enabled - Test(query=s"""foo{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) baz{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bat"}""", - tschemaEnabled=Set(), - expected="""E~BinaryJoinExec(binaryOp=ADD, on=List(tschemaLabel), ignoring=List()) on InProcessPlanDispatcher - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |--E~StitchRvsExec() on InProcessPlanDispatcher - |---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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), 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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |---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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), 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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--E~StitchRvsExec() on InProcessPlanDispatcher - |---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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), 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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |---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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), 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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |--E~StitchRvsExec() on InProcessPlanDispatcher - |---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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(baz))), 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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |---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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(baz))), 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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--E~StitchRvsExec() on InProcessPlanDispatcher - |---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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(baz))), 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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |---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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(baz))), 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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw)""".stripMargin), - // one namespace has a target-schema assigned - Test(query=s"""foo{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) baz{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bat"}""", - tschemaEnabled=Set("local1"), - expected="""E~BinaryJoinExec(binaryOp=ADD, on=List(tschemaLabel), ignoring=List()) on InProcessPlanDispatcher - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |--E~StitchRvsExec() on InProcessPlanDispatcher - |---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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), 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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |---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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), 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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--E~StitchRvsExec() on InProcessPlanDispatcher - |---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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |---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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |--E~StitchRvsExec() on InProcessPlanDispatcher - |---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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(baz))), 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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |---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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(baz))), 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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--E~StitchRvsExec() on InProcessPlanDispatcher - |---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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |---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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw)""".stripMargin), - // both namespaces are assigned target-schemas; should concat results - Test(query=s"""foo{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) baz{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bat"}""", - tschemaEnabled=Set("local1", "local2"), - expected="""E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |-E~StitchRvsExec() on InProcessPlanDispatcher - |--E~BinaryJoinExec(binaryOp=ADD, on=List(tschemaLabel), ignoring=List()) 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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), 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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |--E~BinaryJoinExec(binaryOp=ADD, on=List(tschemaLabel), ignoring=List()) 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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), 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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-E~StitchRvsExec() on InProcessPlanDispatcher - |--E~BinaryJoinExec(binaryOp=ADD, on=List(tschemaLabel), ignoring=List()) 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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(baz))), 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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |--E~BinaryJoinExec(binaryOp=ADD, on=List(tschemaLabel), ignoring=List()) 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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(baz))), 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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw)""".stripMargin), - // remove "on" clause; should not concatenate - Test(query=s"""foo{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bar"} + baz{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bat"}""", - tschemaEnabled=Set("local1", "local2"), - expected="""E~BinaryJoinExec(binaryOp=ADD, on=List(), ignoring=List()) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,true,false,true,Set(),Some(plannerSelector))) - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,true,false,true,Set(),Some(plannerSelector))) - |--E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,false,false,true,Set(),None)) - |---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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#2094049380],raw) - |---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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#2094049380],downsample) - |--E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,false,false,true,Set(),None)) - |---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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#2094049380],raw) - |---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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#2094049380],downsample) - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,true,false,true,Set(),Some(plannerSelector))) - |--E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,false,false,true,Set(),None)) - |---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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#2094049380],raw) - |---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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#2094049380],downsample) - |--E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,false,false,true,Set(),None)) - |---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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#2094049380],raw) - |---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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#2094049380],downsample)""".stripMargin), - // "on" clause is missing a target-schema label - Test(query=s"""foo{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bar"} + on (notATargetSchemaLabel) baz{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bat"}""", - tschemaEnabled=Set("local1", "local2"), - expected="""E~BinaryJoinExec(binaryOp=ADD, on=List(notATargetSchemaLabel), ignoring=List()) on InProcessPlanDispatcher - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |--E~StitchRvsExec() on InProcessPlanDispatcher - |---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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |---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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--E~StitchRvsExec() on InProcessPlanDispatcher - |---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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |---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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |--E~StitchRvsExec() on InProcessPlanDispatcher - |---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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |---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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--E~StitchRvsExec() on InProcessPlanDispatcher - |---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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |---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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw)""".stripMargin), - // remote partition, no target-schemas assigned - Test(query=s"""foo{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) baz{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bat"}""", - tschemaEnabled=Set(), - expected="""E~BinaryJoinExec(binaryOp=ADD, on=List(tschemaLabel), ignoring=List()) on InProcessPlanDispatcher - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |--E~PromQlRemoteExec(PromQlQueryParams(baz{tschemaLabel="bat",_ws_="demo",_ns_="oneRemote1"},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(baz{tschemaLabel="bat",_ws_="demo",_ns_="oneRemote2"},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~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |--E~PromQlRemoteExec(PromQlQueryParams(foo{tschemaLabel="bar",_ws_="demo",_ns_="oneRemote1"},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"},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), - // remote partition, one ns has a target-schema assigned - Test(query=s"""foo{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) baz{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bat"}""", - tschemaEnabled=Set("oneRemote1"), - expected="""E~BinaryJoinExec(binaryOp=ADD, on=List(tschemaLabel), ignoring=List()) on InProcessPlanDispatcher - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |--E~PromQlRemoteExec(PromQlQueryParams(baz{tschemaLabel="bat",_ws_="demo",_ns_="oneRemote1"},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(baz{tschemaLabel="bat",_ws_="demo",_ns_="oneRemote2"},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~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |--E~PromQlRemoteExec(PromQlQueryParams(foo{tschemaLabel="bar",_ws_="demo",_ns_="oneRemote1"},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"},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), - // remote partition, both ns have a target-schema assigned - Test(query=s"""foo{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) baz{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bat"}""", - tschemaEnabled=Set("oneRemote1", "oneRemote2"), - expected="""E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |-E~PromQlRemoteExec(PromQlQueryParams((foo{tschemaLabel="bar",_ws_="demo",_ns_="oneRemote1"} + on(tschemaLabel) baz{tschemaLabel="bat",_ws_="demo",_ns_="oneRemote1"}),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) baz{tschemaLabel="bat",_ws_="demo",_ns_="oneRemote2"}),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), - // two remote partitions, both ns have a target-schema assigned - Test(query=s"""foo{_ws_="demo",_ns_=~"$TWO_REMOTE",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) baz{_ws_="demo",_ns_=~"$TWO_REMOTE",$TSCHEMA_LABEL="bat"}""", - tschemaEnabled=Set("twoRemote1", "twoRemote2"), - expected="""E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |-E~PromQlRemoteExec(PromQlQueryParams((foo{tschemaLabel="bar",_ws_="demo",_ns_="twoRemote1"} + on(tschemaLabel) baz{tschemaLabel="bat",_ws_="demo",_ns_="twoRemote1"}),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=twoRemote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher - |-E~PromQlRemoteExec(PromQlQueryParams((foo{tschemaLabel="bar",_ws_="demo",_ns_="twoRemote2"} + on(tschemaLabel) baz{tschemaLabel="bat",_ws_="demo",_ns_="twoRemote2"}),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=twoRemote2-url, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin), - // remote partition, both ns have a target-schema assigned, but "on" clause is missing - Test(query=s"""foo{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"} + baz{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bat"}""", - tschemaEnabled=Set("oneRemote1", "oneRemote2"), - expected="""E~BinaryJoinExec(binaryOp=ADD, on=List(), ignoring=List()) on InProcessPlanDispatcher - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |--E~PromQlRemoteExec(PromQlQueryParams(baz{tschemaLabel="bat",_ws_="demo",_ns_="oneRemote1"},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(baz{tschemaLabel="bat",_ws_="demo",_ns_="oneRemote2"},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~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |--E~PromQlRemoteExec(PromQlQueryParams(foo{tschemaLabel="bar",_ws_="demo",_ns_="oneRemote1"},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"},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), - // remote/local partitions, no ns has a tschema assigned - Test(query=s"""foo{_ws_="demo",_ns_=~"$BOTH",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) baz{_ws_="demo",_ns_=~"$BOTH",$TSCHEMA_LABEL="bat"}""", - tschemaEnabled=Set(), - expected="""E~BinaryJoinExec(binaryOp=ADD, on=List(tschemaLabel), ignoring=List()) on InProcessPlanDispatcher - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |--E~PromQlRemoteExec(PromQlQueryParams(baz{tschemaLabel="bat",_ws_="demo",_ns_="remote"},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~StitchRvsExec() on InProcessPlanDispatcher - |---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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(baz))), 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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |---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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(baz))), 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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |--E~PromQlRemoteExec(PromQlQueryParams(foo{tschemaLabel="bar",_ws_="demo",_ns_="remote"},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~StitchRvsExec() on InProcessPlanDispatcher - |---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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(foo))), 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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |---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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(foo))), 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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw)""".stripMargin), - // remote/local partitions, local ns has a tschema assigned - Test(query=s"""foo{_ws_="demo",_ns_=~"$BOTH",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) baz{_ws_="demo",_ns_=~"$BOTH",$TSCHEMA_LABEL="bat"}""", - tschemaEnabled=Set("local"), - expected="""E~BinaryJoinExec(binaryOp=ADD, on=List(tschemaLabel), ignoring=List()) on InProcessPlanDispatcher - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |--E~PromQlRemoteExec(PromQlQueryParams(baz{tschemaLabel="bat",_ws_="demo",_ns_="remote"},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~StitchRvsExec() on InProcessPlanDispatcher - |---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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |---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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |--E~PromQlRemoteExec(PromQlQueryParams(foo{tschemaLabel="bar",_ws_="demo",_ns_="remote"},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~StitchRvsExec() on InProcessPlanDispatcher - |---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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |---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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw)""".stripMargin), - // remote/local partitions, remote ns has a tschema assigned - Test(query=s"""foo{_ws_="demo",_ns_=~"$BOTH",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) baz{_ws_="demo",_ns_=~"$BOTH",$TSCHEMA_LABEL="bat"}""", - tschemaEnabled=Set("remote"), - expected="""E~BinaryJoinExec(binaryOp=ADD, on=List(tschemaLabel), ignoring=List()) on InProcessPlanDispatcher - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |--E~PromQlRemoteExec(PromQlQueryParams(baz{tschemaLabel="bat",_ws_="demo",_ns_="remote"},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~StitchRvsExec() on InProcessPlanDispatcher - |---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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(baz))), 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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |---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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(baz))), 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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |--E~PromQlRemoteExec(PromQlQueryParams(foo{tschemaLabel="bar",_ws_="demo",_ns_="remote"},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~StitchRvsExec() on InProcessPlanDispatcher - |---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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(foo))), 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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |---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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(foo))), 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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw)""".stripMargin), - // one remote partition, and all ns are assigned a tschema, but tschema label missing from selector - Test(query=s"""foo{_ws_="demo",_ns_=~"$ONE_REMOTE"} + on ($TSCHEMA_LABEL) baz{_ws_="demo",_ns_=~"$ONE_REMOTE"}""", - tschemaEnabled=Set("oneRemote1", "oneRemote2"), - expected="""E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |-E~PromQlRemoteExec(PromQlQueryParams((foo{_ws_="demo",_ns_="oneRemote1"} + on(tschemaLabel) baz{_ws_="demo",_ns_="oneRemote1"}),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{_ws_="demo",_ns_="oneRemote2"} + on(tschemaLabel) baz{_ws_="demo",_ns_="oneRemote2"}),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), - // one remote partition, and all ns are assigned a tschema, but tschema label missing from selector and "by" clause - Test(query=s"""foo{_ws_="demo",_ns_=~"$ONE_REMOTE"} + on (notATargetSchemaLabel) baz{_ws_="demo",_ns_=~"$ONE_REMOTE"}""", - tschemaEnabled=Set("oneRemote1", "oneRemote2"), - expected="""E~BinaryJoinExec(binaryOp=ADD, on=List(notATargetSchemaLabel), ignoring=List()) on InProcessPlanDispatcher - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |--E~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_="oneRemote1"},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(baz{_ws_="demo",_ns_="oneRemote2"},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~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_="oneRemote1"},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{_ws_="demo",_ns_="oneRemote2"},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), - // local partition, and all ns are assigned a tschema, but tschema label missing from selector - Test(query=s"""foo{_ws_="demo",_ns_=~"$LOCAL"} + on ($TSCHEMA_LABEL) baz{_ws_="demo",_ns_=~"$LOCAL"}""", - tschemaEnabled=Set("local1", "local2"), - expected="""E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |-E~StitchRvsExec() on InProcessPlanDispatcher - |--E~BinaryJoinExec(binaryOp=ADD, on=List(tschemaLabel), ignoring=List()) 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_,Equals(local1)), ColumnFilter(_metric_,Equals(baz))), 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=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), 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_,Equals(local1)), ColumnFilter(_metric_,Equals(baz))), 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_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |--E~BinaryJoinExec(binaryOp=ADD, on=List(tschemaLabel), ignoring=List()) 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_,Equals(local1)), ColumnFilter(_metric_,Equals(baz))), 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=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), 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_,Equals(local1)), ColumnFilter(_metric_,Equals(baz))), 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_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-E~StitchRvsExec() on InProcessPlanDispatcher - |--E~BinaryJoinExec(binaryOp=ADD, on=List(tschemaLabel), ignoring=List()) 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_,Equals(local2)), ColumnFilter(_metric_,Equals(baz))), 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=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), 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_,Equals(local2)), ColumnFilter(_metric_,Equals(baz))), 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_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |--E~BinaryJoinExec(binaryOp=ADD, on=List(tschemaLabel), ignoring=List()) 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_,Equals(local2)), ColumnFilter(_metric_,Equals(baz))), 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=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), 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_,Equals(local2)), ColumnFilter(_metric_,Equals(baz))), 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_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw)""".stripMargin), - // one remote partition, and all ns are assigned a tschema; nested join - Test(query=s"""foo{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) (baz{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"} - on($TSCHEMA_LABEL) bak{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"})""", - tschemaEnabled=Set("oneRemote1", "oneRemote2"), - expected="""E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |-E~PromQlRemoteExec(PromQlQueryParams((foo{tschemaLabel="bar",_ws_="demo",_ns_="oneRemote1"} + on(tschemaLabel) (baz{tschemaLabel="bar",_ws_="demo",_ns_="oneRemote1"} - on(tschemaLabel) bak{tschemaLabel="bar",_ws_="demo",_ns_="oneRemote1"})),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) (baz{tschemaLabel="bar",_ws_="demo",_ns_="oneRemote2"} - on(tschemaLabel) bak{tschemaLabel="bar",_ws_="demo",_ns_="oneRemote2"})),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), - // one remote partition, and all ns are assigned a tschema; nested aggregation with target-schema label missing from outer "by" clause - Test(query=s"""foo{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"} + on (notATargetSchemaLabel) (baz{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"} - on($TSCHEMA_LABEL) bak{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"})""", - tschemaEnabled=Set("oneRemote1", "oneRemote2"), - expected="""E~BinaryJoinExec(binaryOp=ADD, on=List(notATargetSchemaLabel), ignoring=List()) on InProcessPlanDispatcher - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |--E~PromQlRemoteExec(PromQlQueryParams((baz{tschemaLabel="bar",_ws_="demo",_ns_="oneRemote1"} - on(tschemaLabel) bak{tschemaLabel="bar",_ws_="demo",_ns_="oneRemote1"}),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((baz{tschemaLabel="bar",_ws_="demo",_ns_="oneRemote2"} - on(tschemaLabel) bak{tschemaLabel="bar",_ws_="demo",_ns_="oneRemote2"}),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~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |--E~PromQlRemoteExec(PromQlQueryParams(foo{tschemaLabel="bar",_ws_="demo",_ns_="oneRemote1"},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"},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), - // one remote partition, and all ns are assigned a tschema; nested aggregation with target-schema label missing from inner "by" clause - Test(query=s"""foo{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) (baz{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"} - on (notATargetSchemaLabel) bak{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"})""", - tschemaEnabled=Set("oneRemote1", "oneRemote2"), - expected="""E~BinaryJoinExec(binaryOp=ADD, on=List(tschemaLabel), ignoring=List()) on InProcessPlanDispatcher - |-E~BinaryJoinExec(binaryOp=SUB, on=List(notATargetSchemaLabel), ignoring=List()) on InProcessPlanDispatcher - |--E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |---E~PromQlRemoteExec(PromQlQueryParams(bak{tschemaLabel="bar",_ws_="demo",_ns_="oneRemote1"},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(bak{tschemaLabel="bar",_ws_="demo",_ns_="oneRemote2"},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~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |---E~PromQlRemoteExec(PromQlQueryParams(baz{tschemaLabel="bar",_ws_="demo",_ns_="oneRemote1"},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(baz{tschemaLabel="bar",_ws_="demo",_ns_="oneRemote2"},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~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |--E~PromQlRemoteExec(PromQlQueryParams(foo{tschemaLabel="bar",_ws_="demo",_ns_="oneRemote1"},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"},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), - - // ----------------- begin compound agg/join tests --------------------- - - // one remote partition, and all ns are assigned a tschema; nested join - Test(query=s"""sum(foo{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) baz{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"}) by ($TSCHEMA_LABEL)""", - tschemaEnabled=Set("oneRemote1", "oneRemote2"), - expected="""E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |-E~PromQlRemoteExec(PromQlQueryParams(sum((foo{tschemaLabel="bar",_ws_="demo",_ns_="oneRemote1"} + on(tschemaLabel) 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(sum((foo{tschemaLabel="bar",_ws_="demo",_ns_="oneRemote2"} + on(tschemaLabel) 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), - - // one remote partition, and all ns are assigned a tschema; nested aggregation - Test(query=s"""foo{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) sum(baz{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"}) by ($TSCHEMA_LABEL)""", - tschemaEnabled=Set("oneRemote1", "oneRemote2"), - expected="""E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |-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) + Test(query = """sum(foo{_ws_="demo",_ns_=~"local.*",tschemaLabel="bar"}) by (tschemaLabel)""", + tschemaEnabled = Set(), + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192))) + |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1634173130,300,1634777330)) + |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],raw) + |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |----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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],raw) + |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |----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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],raw) + |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634172830)) + |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],downsample) + |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |----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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],downsample) + |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |----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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],downsample)""".stripMargin), + Test(query = """sum(foo{_ws_="demo",_ns_=~"local.*",tschemaLabel="bar"}) by (tschemaLabel)""", + tschemaEnabled = Set("local1"), + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192))) + |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1634173130,300,1634777330)) + |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],raw) + |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |----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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],raw) + |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |----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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],raw) + |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634172830)) + |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],downsample) + |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |----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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],downsample) + |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |----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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],downsample)""".stripMargin), + Test(query = """sum(foo{_ws_="demo",_ns_=~"local.*",tschemaLabel="bar"}) by (tschemaLabel)""", + tschemaEnabled = Set("local1", "local2"), + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192))) + |-E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],raw) + |--T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1634173130,300,1634777330)) + |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],raw) + |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |-----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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,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~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1634173130,300,1634777330)) + |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],raw) + |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |-----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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) + |-E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],downsample) + |--T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634172830)) + |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],downsample) + |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |-----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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,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~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634172830)) + |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],downsample) + |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |-----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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,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(foo{_ws_="demo",_ns_=~"local.*",tschemaLabel="bar"})""", + tschemaEnabled = Set("local1", "local2"), + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192))) + |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1634173130,300,1634777330)) + |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],raw) + |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634172830)) + |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],downsample)""".stripMargin), + Test(query = """sum(foo{_ws_="demo",_ns_=~"local.*",tschemaLabel="bar"}) by (notATargetSchemaLabel)""", + tschemaEnabled = Set("local1", "local2"), + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192))) + |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1634173130,300,1634777330)) + |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],raw) + |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(notATargetSchemaLabel)) + |----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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],raw) + |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(notATargetSchemaLabel)) + |----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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],raw) + |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634172830)) + |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],downsample) + |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(notATargetSchemaLabel)) + |----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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],downsample) + |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(notATargetSchemaLabel)) + |----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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],downsample)""".stripMargin), + Test(query = """sum(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"}) by (tschemaLabel)""", + tschemaEnabled = Set(), + expected = """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_=~"oneRemote.*",tschemaLabel="bar"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"}) by (tschemaLabel)""", + tschemaEnabled = Set("oneRemote1"), + expected = """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_=~"oneRemote.*",tschemaLabel="bar"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"}) by (tschemaLabel)""", + tschemaEnabled = Set("oneRemote1", "oneRemote2"), + expected = """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_=~"oneRemote.*",tschemaLabel="bar"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(foo{_ws_="demo",_ns_=~"twoRemote.*",tschemaLabel="bar"}) by (tschemaLabel)""", + tschemaEnabled = Set("twoRemote1", "twoRemote2"), + expected = """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_=~"twoRemote.*",tschemaLabel="bar"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=twoRemote1-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(foo{_ws_="demo",_ns_=~"twoRemote.*",tschemaLabel="bar"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=twoRemote2-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(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"})""", + tschemaEnabled = Set("oneRemote1", "oneRemote2"), + expected = """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_=~"oneRemote.*",tschemaLabel="bar"}),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"}),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(foo{_ws_="demo",_ns_=~"both.*",tschemaLabel="bar"}) by (tschemaLabel)""", + tschemaEnabled = Set(), + expected = """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#1790138487],raw) + |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |-----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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],raw) + |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |-----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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],raw) + |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],downsample) + |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |-----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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],downsample) + |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |-----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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],downsample) + |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"both.*",tschemaLabel="bar"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(foo{_ws_="demo",_ns_=~"both.*",tschemaLabel="bar"}) by (tschemaLabel)""", + tschemaEnabled = Set("local"), + expected = """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#1790138487],raw) + |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |-----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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],raw) + |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |-----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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],raw) + |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],downsample) + |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |-----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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],downsample) + |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |-----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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],downsample) + |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"both.*",tschemaLabel="bar"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(foo{_ws_="demo",_ns_=~"both.*",tschemaLabel="bar"}) by (tschemaLabel)""", + tschemaEnabled = Set("remote"), + expected = """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#1790138487],raw) + |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |-----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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],raw) + |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |-----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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],raw) + |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],downsample) + |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |-----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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],downsample) + |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |-----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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],downsample) + |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"both.*",tschemaLabel="bar"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(foo{_ws_="demo",_ns_=~"oneRemote.*"}) by (tschemaLabel)""", + tschemaEnabled = Set("oneRemote1", "oneRemote2"), + expected = """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_=~"oneRemote.*"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(foo{_ws_="demo",_ns_=~"oneRemote.*"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(foo{_ws_="demo",_ns_=~"oneRemote.*"}) by (notALabel)""", + tschemaEnabled = Set("oneRemote1", "oneRemote2"), + expected = """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_=~"oneRemote.*"}) by (notALabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(foo{_ws_="demo",_ns_=~"oneRemote.*"}) by (notALabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(foo{_ws_="demo",_ns_=~"local.*"}) by (tschemaLabel)""", + tschemaEnabled = Set("local1", "local2"), + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192))) + |-E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],raw) + |--T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1634173130,300,1634777330)) + |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],raw) + |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |-----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(local.*)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,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~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1634173130,300,1634777330)) + |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],raw) + |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |-----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(local.*)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) + |-E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],downsample) + |--T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634172830)) + |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],downsample) + |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |-----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(local.*)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,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~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634172830)) + |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],downsample) + |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |-----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(local.*)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,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(sum(foo{_ws_="demo",_ns_=~"oneRemote.*"}) by (tschemaLabel)) by (tschemaLabel)""", + tschemaEnabled = Set("oneRemote1", "oneRemote2"), + expected = """T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634777330)) + |-E~LocalPartitionReduceAggregateExec(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))) + |--T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |---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_=~"oneRemote.*"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(foo{_ws_="demo",_ns_=~"oneRemote.*"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(sum(foo{_ws_="demo",_ns_=~"oneRemote.*"}) by (tschemaLabel))""", + tschemaEnabled = Set("oneRemote1", "oneRemote2"), + expected = """T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634777330)) + |-E~LocalPartitionReduceAggregateExec(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))) + |--T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=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_=~"oneRemote.*"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(foo{_ws_="demo",_ns_=~"oneRemote.*"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(sum(foo{_ws_="demo",_ns_=~"oneRemote.*"})) by (tschemaLabel)""", + tschemaEnabled = Set("oneRemote1", "oneRemote2"), + expected = """T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634777330)) + |-E~LocalPartitionReduceAggregateExec(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))) + |--T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |---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_=~"oneRemote.*"}),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(foo{_ws_="demo",_ns_=~"oneRemote.*"}),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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 = """foo{_ws_="demo",_ns_=~"local.*",tschemaLabel="bar"} + on (tschemaLabel) baz{_ws_="demo",_ns_=~"local.*",tschemaLabel="bat"}""", + tschemaEnabled = Set(), + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192))) + |-E~BinaryJoinExec(binaryOp=ADD, on=List(tschemaLabel), ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(local.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(local.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],raw) + |-E~BinaryJoinExec(binaryOp=ADD, on=List(tschemaLabel), ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(local.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(local.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],downsample)""".stripMargin), + Test(query = """foo{_ws_="demo",_ns_=~"local.*",tschemaLabel="bar"} + on (tschemaLabel) baz{_ws_="demo",_ns_=~"local.*",tschemaLabel="bat"}""", + tschemaEnabled = Set("local1"), + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192))) + |-E~BinaryJoinExec(binaryOp=ADD, on=List(tschemaLabel), ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(local.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(local.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],raw) + |-E~BinaryJoinExec(binaryOp=ADD, on=List(tschemaLabel), ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(local.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(local.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],downsample)""".stripMargin), + Test(query = """foo{_ws_="demo",_ns_=~"local.*",tschemaLabel="bar"} + on (tschemaLabel) baz{_ws_="demo",_ns_=~"local.*",tschemaLabel="bat"}""", + tschemaEnabled = Set("local1", "local2"), + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192))) + |-E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],raw) + |--E~BinaryJoinExec(binaryOp=ADD, on=List(tschemaLabel), ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,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~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(local.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) + |--E~BinaryJoinExec(binaryOp=ADD, on=List(tschemaLabel), ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,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~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(local.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) + |-E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],downsample) + |--E~BinaryJoinExec(binaryOp=ADD, on=List(tschemaLabel), ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,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~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(local.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) + |--E~BinaryJoinExec(binaryOp=ADD, on=List(tschemaLabel), ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,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~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(local.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,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 = """foo{_ws_="demo",_ns_=~"local.*",tschemaLabel="bar"} + baz{_ws_="demo",_ns_=~"local.*",tschemaLabel="bat"}""", + tschemaEnabled = Set("local1", "local2"), + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192))) + |-E~BinaryJoinExec(binaryOp=ADD, on=List(), ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(local.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(local.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],raw) + |-E~BinaryJoinExec(binaryOp=ADD, on=List(), ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(local.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(local.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],downsample)""".stripMargin), + Test(query = """foo{_ws_="demo",_ns_=~"local.*",tschemaLabel="bar"} + on (notATargetSchemaLabel) baz{_ws_="demo",_ns_=~"local.*",tschemaLabel="bat"}""", + tschemaEnabled = Set("local1", "local2"), + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192))) + |-E~BinaryJoinExec(binaryOp=ADD, on=List(notATargetSchemaLabel), ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(local.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(local.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],raw) + |-E~BinaryJoinExec(binaryOp=ADD, on=List(notATargetSchemaLabel), ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(local.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(local.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],downsample)""".stripMargin), + Test(query = """foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"} + on (tschemaLabel) baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bat"}""", + tschemaEnabled = Set(), + expected = """E~BinaryJoinExec(binaryOp=ADD, on=List(tschemaLabel), 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~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bat"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bat"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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 = """foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"} + on (tschemaLabel) baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bat"}""", + tschemaEnabled = Set("oneRemote1"), + expected = """E~BinaryJoinExec(binaryOp=ADD, on=List(tschemaLabel), 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~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bat"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bat"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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 = """foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"} + on (tschemaLabel) baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bat"}""", + tschemaEnabled = Set("oneRemote1", "oneRemote2"), + expected = """E~BinaryJoinExec(binaryOp=ADD, on=List(tschemaLabel), 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~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bat"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bat"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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 = """foo{_ws_="demo",_ns_=~"twoRemote.*",tschemaLabel="bar"} + on (tschemaLabel) baz{_ws_="demo",_ns_=~"twoRemote.*",tschemaLabel="bat"}""", + tschemaEnabled = Set("twoRemote1", "twoRemote2"), + expected = """E~BinaryJoinExec(binaryOp=ADD, on=List(tschemaLabel), 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~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"twoRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=twoRemote1-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(foo{_ws_="demo",_ns_=~"twoRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=twoRemote2-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~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"twoRemote.*",tschemaLabel="bat"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=twoRemote1-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(baz{_ws_="demo",_ns_=~"twoRemote.*",tschemaLabel="bat"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=twoRemote2-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 = """foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"} + baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bat"}""", + tschemaEnabled = Set("oneRemote1", "oneRemote2"), + expected = """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~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bat"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bat"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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 = """foo{_ws_="demo",_ns_=~"both.*",tschemaLabel="bar"} + on (tschemaLabel) baz{_ws_="demo",_ns_=~"both.*",tschemaLabel="bat"}""", + tschemaEnabled = Set(), + expected = """E~BinaryJoinExec(binaryOp=ADD, on=List(tschemaLabel), 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#1790138487],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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],raw) + |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],downsample) + |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"both.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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#1790138487],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(both.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(both.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],raw) + |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(both.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(both.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],downsample) + |--E~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"both.*",tschemaLabel="bat"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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 = """foo{_ws_="demo",_ns_=~"both.*",tschemaLabel="bar"} + on (tschemaLabel) baz{_ws_="demo",_ns_=~"both.*",tschemaLabel="bat"}""", + tschemaEnabled = Set("local"), + expected = """E~BinaryJoinExec(binaryOp=ADD, on=List(tschemaLabel), 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#1790138487],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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],raw) + |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],downsample) + |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"both.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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#1790138487],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(both.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(both.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],raw) + |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(both.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(both.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],downsample) + |--E~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"both.*",tschemaLabel="bat"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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 = """foo{_ws_="demo",_ns_=~"both.*",tschemaLabel="bar"} + on (tschemaLabel) baz{_ws_="demo",_ns_=~"both.*",tschemaLabel="bat"}""", + tschemaEnabled = Set("remote"), + expected = """E~BinaryJoinExec(binaryOp=ADD, on=List(tschemaLabel), 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#1790138487],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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],raw) + |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],downsample) + |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"both.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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#1790138487],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(both.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(both.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],raw) + |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(both.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(both.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],downsample) + |--E~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"both.*",tschemaLabel="bat"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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 = """foo{_ws_="demo",_ns_=~"oneRemote.*"} + on (tschemaLabel) baz{_ws_="demo",_ns_=~"oneRemote.*"}""", + tschemaEnabled = Set("oneRemote1", "oneRemote2"), + expected = """E~BinaryJoinExec(binaryOp=ADD, on=List(tschemaLabel), 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~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"oneRemote.*"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(foo{_ws_="demo",_ns_=~"oneRemote.*"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"oneRemote.*"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(baz{_ws_="demo",_ns_=~"oneRemote.*"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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 = """foo{_ws_="demo",_ns_=~"oneRemote.*"} + on (notATargetSchemaLabel) baz{_ws_="demo",_ns_=~"oneRemote.*"}""", + tschemaEnabled = Set("oneRemote1", "oneRemote2"), + expected = """E~BinaryJoinExec(binaryOp=ADD, on=List(notATargetSchemaLabel), 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~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"oneRemote.*"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(foo{_ws_="demo",_ns_=~"oneRemote.*"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"oneRemote.*"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(baz{_ws_="demo",_ns_=~"oneRemote.*"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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 = """foo{_ws_="demo",_ns_=~"local.*"} + on (tschemaLabel) baz{_ws_="demo",_ns_=~"local.*"}""", + tschemaEnabled = Set("local1", "local2"), + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192))) + |-E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],raw) + |--E~BinaryJoinExec(binaryOp=ADD, on=List(tschemaLabel), ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(local.*)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,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~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(local.*)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) + |--E~BinaryJoinExec(binaryOp=ADD, on=List(tschemaLabel), ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(local.*)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,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~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(local.*)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) + |-E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],downsample) + |--E~BinaryJoinExec(binaryOp=ADD, on=List(tschemaLabel), ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(local.*)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,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~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(local.*)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536))) + |--E~BinaryJoinExec(binaryOp=ADD, on=List(tschemaLabel), ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],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(local.*)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,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~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(local.*)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,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 = """foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"} + on (tschemaLabel) (baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"} - on(tschemaLabel) bak{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"})""", + tschemaEnabled = Set("oneRemote1", "oneRemote2"), + expected = """E~BinaryJoinExec(binaryOp=ADD, on=List(tschemaLabel), 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~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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~BinaryJoinExec(binaryOp=SUB, on=List(tschemaLabel), 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~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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~PromQlRemoteExec(PromQlQueryParams(bak{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(bak{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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 = """foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"} + on (notATargetSchemaLabel) (baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"} - on(tschemaLabel) bak{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"})""", + tschemaEnabled = Set("oneRemote1", "oneRemote2"), + expected = """E~BinaryJoinExec(binaryOp=ADD, on=List(notATargetSchemaLabel), 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~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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~BinaryJoinExec(binaryOp=SUB, on=List(tschemaLabel), 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~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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~PromQlRemoteExec(PromQlQueryParams(bak{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(bak{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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 = """foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"} + on (tschemaLabel) (baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"} - on (notATargetSchemaLabel) bak{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"})""", + tschemaEnabled = Set("oneRemote1", "oneRemote2"), + expected = """E~BinaryJoinExec(binaryOp=ADD, on=List(tschemaLabel), 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~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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~BinaryJoinExec(binaryOp=SUB, on=List(notATargetSchemaLabel), 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~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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~PromQlRemoteExec(PromQlQueryParams(bak{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(bak{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"} + on (tschemaLabel) baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"}) by (tschemaLabel)""", + tschemaEnabled = Set("oneRemote1", "oneRemote2"), + expected = """T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634777330)) + |-E~LocalPartitionReduceAggregateExec(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))) + |--T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |---E~BinaryJoinExec(binaryOp=ADD, on=List(tschemaLabel), 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~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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 = """foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"} + on (tschemaLabel) sum(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"}) by (tschemaLabel)""", + tschemaEnabled = Set("oneRemote1", "oneRemote2"), + expected = """E~BinaryJoinExec(binaryOp=ADD, on=List(tschemaLabel), 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~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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))) + |-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(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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) ) for (test <- tests) { def tschemaProviderFunc(filters: Seq[ColumnFilter]): Seq[TargetSchemaChange] = { @@ -3460,7 +3339,8 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS } } - val multiPartitionPlanner = new MultiPartitionPlanner(partitionLocationProvider, singlePartitionPlanner, "local", dataset, queryConfig) + val spp = getPlanners(2, dataset, shardKeyMatcherFunc).spp + val multiPartitionPlanner = new MultiPartitionPlanner(partitionLocationProvider, spp, "local", dataset, queryConfig, shardKeyMatcher = shardKeyMatcherFunc) val shardKeyRegexPlanner = new ShardKeyRegexPlanner(dataset, multiPartitionPlanner, shardKeyMatcherFunc, partitionLocationProvider, queryConfig, targetSchemaProvider) val tschema = FunctionalTargetSchemaProvider(tschemaProviderFunc) @@ -3503,4 +3383,271 @@ 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) + } } From 2b62db839b2d300d714c45d924926f92a58ec7ca Mon Sep 17 00:00:00 2001 From: Alex Theimer Date: Fri, 29 Sep 2023 01:32:18 -0700 Subject: [PATCH 04/27] add tests --- .../queryplanner/PlannerHierarchySpec.scala | 178 +++++++++--------- 1 file changed, 86 insertions(+), 92 deletions(-) diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala index 809998904b..ddc7f16013 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 -import filodb.core.{MetricsTestData, StaticTargetSchemaProvider, TargetSchemaChange} +import filodb.coordinator.client.QueryCommands.{FunctionalTargetSchemaProvider, StaticSpreadProvider} +import filodb.core.{MetricsTestData, SpreadChange, StaticTargetSchemaProvider, TargetSchemaChange} import filodb.core.metadata.{Dataset, Schemas} import filodb.core.query._ import filodb.core.query.Filter.{Equals, EqualsRegex} @@ -3414,75 +3414,72 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS // 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) + """E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher + |-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(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~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/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(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) + |----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/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(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~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/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(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), + |----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/testActor],downsample) + |-E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="dummy",_ns_=~"dummy.*",bar="hello"},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=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~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) |----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) + |------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/testActor],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) + |------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/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=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) + |------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/testActor],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), + |------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/testActor],downsample) + |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="dummy",_ns_=~"dummy.*",bar="hello"}),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=dummy-endpoint, requestTimeoutMs=10000) on InProcessPlanDispatcher""".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) + """E~BinaryJoinExec(binaryOp=ADD, on=List(), ignoring=List()) on InProcessPlanDispatcher + |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher + |--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(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~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/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(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) + |-----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/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(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~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/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(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) + |-----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/testActor],downsample) + |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="dummy",_ns_=~"dummy.*",bar="hello"},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=dummy-endpoint, requestTimeoutMs=10000) on InProcessPlanDispatcher + |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher + |--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(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~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/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(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) + |-----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/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(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~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/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(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) + |-----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/testActor],downsample) + |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="dummy",_ns_=~"dummy.*",bar="goodbye"},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=dummy-endpoint, requestTimeoutMs=10000) on InProcessPlanDispatcher""".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") { @@ -3524,93 +3521,90 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS // 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) + """E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher + |-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=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) + |----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/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(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~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/testActor],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) + |----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/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=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) + |----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/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(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~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/testActor],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), + |----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/testActor],downsample) + |-E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="dummy",_ns_=~"dummy.*",bar="hello"},1633913330,300,1634777330,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""".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~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) |----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) + |------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/testActor],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) + |------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/testActor],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) + |------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/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=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) + |------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/testActor],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) + |------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/testActor],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), + |------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/testActor],downsample) + |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="dummy",_ns_=~"dummy.*",bar="hello"}),1633913330,300,1634777330,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""".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) + """E~BinaryJoinExec(binaryOp=ADD, on=List(), ignoring=List()) on InProcessPlanDispatcher + |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher + |--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=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) + |-----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/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(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~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/testActor],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) + |-----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/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=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) + |-----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/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(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~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/testActor],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) + |-----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/testActor],downsample) + |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="dummy",_ns_=~"dummy.*",bar="hello"},1633913330,300,1634777330,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 + |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher + |--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=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) + |-----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/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(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~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/testActor],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) + |-----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/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=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) + |-----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/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(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~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/testActor],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) + |-----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/testActor],downsample) + |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="dummy",_ns_=~"dummy.*",bar="goodbye"},1633913330,300,1634777330,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""".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") { From 7a6ff9e8c8b254b3f1488678acaaafee0985ead5 Mon Sep 17 00:00:00 2001 From: Alex Theimer Date: Fri, 29 Sep 2023 01:34:53 -0700 Subject: [PATCH 05/27] scalastyle --- .../queryplanner/MultiPartitionPlanner.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala index a0f83b925e..cedad639e2 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala @@ -2,10 +2,13 @@ package filodb.coordinator.queryplanner import java.util.concurrent.ConcurrentHashMap + import scala.collection.concurrent.{Map => ConcurrentMap} import scala.jdk.CollectionConverters._ + import com.typesafe.scalalogging.StrictLogging import io.grpc.ManagedChannel + import filodb.coordinator.queryplanner.LogicalPlanUtils._ import filodb.core.metadata.{Dataset, DatasetOptions, Schemas} import filodb.core.query.{ColumnFilter, PromQlQueryParams, QueryConfig, QueryContext, RangeParams, RvRange} From 14d685c5f6168390e18cb5afc049f3e0a9091121 Mon Sep 17 00:00:00 2001 From: Alex Theimer Date: Wed, 4 Oct 2023 18:52:12 -0700 Subject: [PATCH 06/27] incorporate PR feedback --- .../queryplanner/MultiPartitionPlanner.scala | 26 ++++++++++++------- 1 file changed, 17 insertions(+), 9 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala index cedad639e2..9705b7a65d 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala @@ -98,12 +98,7 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider } else logicalPlan match { case mqp: MetadataQueryPlan => materializeMetadataQueryPlan(mqp, qContext).plans.head case lp: TsCardinalities => materializeTsCardinalities(lp, qContext).plans.head - case _ => - val result = walkLogicalPlanTree(logicalPlan, qContext) - if (result.plans.size > 1) { - val dispatcher = PlannerUtil.pickDispatcher(result.plans) - MultiPartitionDistConcatExec(qContext, dispatcher, result.plans) - } else result.plans.head + case _ => walkLogicalPlanTree(logicalPlan, qContext).plans.head } } @@ -430,7 +425,7 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider val qParams = qContext.origQueryParams.asInstanceOf[PromQlQueryParams] // Create one plan per RawSeries/shard-key pair, then resolve its partitions. // If any resides on more than one partition, the leaf is "split". - val hasMultiPartitionLeaves = + val hasSplitPartitionLeaves = LogicalPlan.findLeafLogicalPlans(logicalPlan) .filter(_.isInstanceOf[RawSeries]) .flatMap { rs => @@ -439,7 +434,7 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider filters.map(rs.replaceFilters) } .exists(getPartitions(_, qParams).size > 1) - if (hasMultiPartitionLeaves) { + if (hasSplitPartitionLeaves) { materializeSplitLeafPlan(logicalPlan, qContext) } else { logicalPlan match { case agg: Aggregate => materializeAggregate(agg, qContext) @@ -453,7 +448,20 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider case ps: PeriodicSeries => materializeNonSplitPeriodicAndRawSeries(ps, qContext) case rcm: RawChunkMeta => materializeNonSplitPeriodicAndRawSeries(rcm, qContext) case rs: RawSeries => materializeNonSplitPeriodicAndRawSeries(rs, qContext) - case x => throw new IllegalArgumentException(s"unhandled type: ${x.getClass}") + case _: ApplyInstantFunctionRaw | + _: ApplyLimitFunction | + _: ApplyMiscellaneousFunction | + _: ApplySortFunction | + _: LabelCardinality | + _: LabelNames | + _: LabelValues | + _: ScalarBinaryOperation | + _: ScalarFixedDoublePlan | + _: ScalarTimeBasedPlan | + _: SeriesKeysByFilters | + _: TopLevelSubquery | + _: TsCardinalities | + _: VectorPlan => throw new IllegalArgumentException(s"unhandled type: ${logicalPlan.getClass}") }} } From 34c6583e04d9416e323c64cb67f3236ff09408d6 Mon Sep 17 00:00:00 2001 From: Alex Theimer Date: Wed, 4 Oct 2023 21:01:30 -0700 Subject: [PATCH 07/27] remove unnecessary tests --- .../ShardKeyRegexPlannerSpec.scala | 143 ------------------ 1 file changed, 143 deletions(-) diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlannerSpec.scala index d06f7a965e..6890e32fb3 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlannerSpec.scala @@ -100,27 +100,6 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture contains(ColumnFilter("_ns_", EqualsRegex("App.*"))) shouldEqual(true) } -// // TODO: is this still necessary? -// it("should generate Exec plan for implicit ws query") { -// val lp = Parser.queryToLogicalPlan("test{_ns_ =~ \"App.*\", instance = \"Inst-1\" }", 1000, 1000) -// val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), -// ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), -// ColumnFilter("_ns_", Equals("App-2"))))} -// val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) -// val execPlan = engine.materialize(lp, QueryContext(origQueryParams = PromQlQueryParams( -// """test{_ns_ =~ "App.*", instance = "Inst-1" }""", 100, 1, 1000))) -// execPlan.isInstanceOf[LocalPartitionDistConcatExec] shouldEqual(true) -// execPlan.children(0).isInstanceOf[MultiSchemaPartitionsExec] -// execPlan.children(1).asInstanceOf[MultiSchemaPartitionsExec].filters. -// contains(ColumnFilter("_ns_", EqualsRegex("App.*"))) shouldEqual(true) -// execPlan.children(0).asInstanceOf[MultiSchemaPartitionsExec].filters. -// contains(ColumnFilter("_ns_", EqualsRegex("App.*"))) shouldEqual(true) -// execPlan.children(0).asInstanceOf[MultiSchemaPartitionsExec].filters. -// contains(ColumnFilter("_ws_", Equals("demo"))) shouldEqual(true) -// execPlan.children(1).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")), @@ -1019,126 +998,4 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture case _ => fail("Expected four children") } } - - // TODO: is this still needed? -// it("should materialize absent function mapper correctly with implicit WS") { -// { -// // absent of sum_over_time: _ws_ in query -// val lp = Parser.queryToLogicalPlan( -// "absent(sum_over_time(test{_ws_ = \"demo\", _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 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( -// "absent(test{_ws_ = \"demo\", _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( -// "absent_over_time(test{_ws_ = \"demo\", _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: is this still needed? -// 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) -// val queryExpectedPairs = Seq( -// // inst func with scalar() arg: _ws_ in query -// (s"""clamp_max(test{_ws_="demo",_ns_="App-1"}, scalar(sc_test{_ws_="demo",_ns_="App-1"}))""", -// """T~InstantVectorFunctionMapper(function=ClampMax) -// |-FA1~ -// |-T~ScalarFunctionMapper(function=Scalar, funcParams=List()) -// |--E~LocalPartitionDistConcatExec() 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=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/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=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/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 engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, simplePartitionLocationProvider, queryConfig) -// -// queryExpectedPairs.foreach{ case (query, expected) => -// val lp = Parser.queryToLogicalPlan(query, 1000, 1000) -// val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) -// execPlan.isInstanceOf[DistConcatExec] shouldEqual true -// validatePlan(execPlan.children.head, expected) -// } -// } } From acd153562c4f73880917bb968bf294a92bacf819 Mon Sep 17 00:00:00 2001 From: Alex Theimer Date: Wed, 4 Oct 2023 22:08:57 -0700 Subject: [PATCH 08/27] use stitch instead of concat --- .../queryplanner/MultiPartitionPlanner.scala | 4 +- .../queryplanner/PlannerHierarchySpec.scala | 140 +++++++++--------- .../ShardKeyRegexPlannerSpec.scala | 8 +- 3 files changed, 76 insertions(+), 76 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala index 9705b7a65d..aae6092d3f 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala @@ -277,8 +277,8 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider if (execPlans.size == 1) execPlans.head else { // TODO: Do we pass in QueryContext in LogicalPlan's helper rvRangeForPlan? - MultiPartitionDistConcatExec( - qContext, inProcessPlanDispatcher, execPlans.sortWith((x, _) => !x.isInstanceOf[PromQlRemoteExec])) + StitchRvsExec(qContext, inProcessPlanDispatcher, rvRangeFromPlan(logicalPlan), + execPlans.sortWith((x, _) => !x.isInstanceOf[PromQlRemoteExec])) } // Sort to move RemoteExec in end as it does not have schema } diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala index ddc7f16013..fc65c1911f 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala @@ -317,7 +317,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |-------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) |------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~StitchRvsExec() on InProcessPlanDispatcher |-----E~PromQlRemoteExec(PromQlQueryParams(bar{_ws_="demo",_ns_=~"remoteNs.*",instance=~".*"},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 |-----E~PromQlRemoteExec(PromQlQueryParams(bar{_ws_="demo",_ns_=~"remoteNs.*",instance=~".*"},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""".stripMargin validatePlan(execPlan, expected) @@ -446,7 +446,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS ("""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,Some(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#-585347165],raw) |----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) @@ -463,7 +463,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS """T~ScalarOperationMapper(operator=MUL, scalarOnLhs=true) |-FA1~StaticFuncArgs(5.0,RangeParams(1633913330,300,1634777330)) |-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,Some(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#-585347165],raw) |-----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) @@ -476,7 +476,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |-----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#-585347165],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))) - |--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,Some(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#-585347165],raw) |-----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) @@ -514,7 +514,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS """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,Some(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#-585347165],raw) |-----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) @@ -549,7 +549,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |-------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#-585347165],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,Some(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#-585347165],raw) |----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) @@ -564,7 +564,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |--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), ("""foo{_ws_ = "demo", _ns_ =~ ".*Ns", instance = "Inst-1" } > vector(5)""", """E~BinaryJoinExec(binaryOp=GTR, 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,Some(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#-585347165],raw) |----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) @@ -583,7 +583,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS """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,Some(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#-585347165],raw) |----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) @@ -601,7 +601,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |-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,Some(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#-585347165],raw) |-----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) @@ -639,7 +639,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS """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,Some(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#-585347165],raw) |-----T~PeriodicSamplesMapper(start=1634172900000, step=300000, end=1634777100000, window=None, functionId=None, rawSource=true, offsetMs=None) @@ -655,7 +655,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS ("""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,Some(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#-585347165],raw) |----T~PeriodicSamplesMapper(start=1634174030000, step=300000, end=1634777330000, window=Some(1200000), functionId=Some(CountOverTime), rawSource=true, offsetMs=None) @@ -673,7 +673,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |-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,Some(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#-585347165],raw) |-----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) @@ -741,17 +741,17 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS ("""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~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(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), ("""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)) |-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,Some(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))) - |--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,Some(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), ("""5 / sum(foo{_ws_ = "demo", _ns_ =~ "remoteNs.*", instance = "Inst-1" })""", @@ -765,7 +765,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS """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,Some(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), ("""foo{_ws_ = "demo", _ns_ =~ "remoteNs.*", instance = "Inst-1" } - scalar(sum(foo{_ws_ = "demo", _ns_ =~ "remoteNs.*", instance = "Inst-1" }))""", @@ -776,12 +776,12 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |---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~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(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), ("""foo{_ws_ = "demo", _ns_ =~ "remoteNs.*", instance = "Inst-1" } > vector(5)""", """E~BinaryJoinExec(binaryOp=GTR, 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,Some(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))) |-T~VectorFunctionMapper(funcParams=List()) @@ -790,7 +790,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS """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,Some(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), ("""5 * foo{_ws_ = "demo", _ns_ =~ "remoteNs.*", instance = "Inst-1" } * 5""", @@ -798,7 +798,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |-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,Some(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), ("""sum(rate(foo{_ws_ = "demo", _ns_ =~ "remoteNs.*", instance = "Inst-1" }[5m])) > 5""", @@ -812,13 +812,13 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS """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,Some(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"},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,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"},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,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), ("""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~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{_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), ("""histogram_quantile(0.9, foo{_ws_ = "demo", _ns_ =~ "remoteNs.*", instance = "Inst-1" }) - 5""", @@ -826,7 +826,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |-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,Some(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) ) @@ -1007,7 +1007,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS plannerParams = PlannerParams(processMultiPartition = true))) val expected = - """E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher + """E~StitchRvsExec() on InProcessPlanDispatcher |-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) @@ -1026,7 +1026,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS // TODO subquery does not push the aggregation down; though this case is NOT a normal case for subquery usage - // if we fix materializeOthers() creates MultiPartitionDistConcatExec + // if we fix materializeOthers() creates StitchRvsExec // this is because of Line bla it("should push down aggregation in generated plan for SubqueryWithWindowing spanning multiple partitions with namespace regex") { val query ="""sum_over_time(foo{_ws_ = "demo", _ns_ =~ ".*Ns", instance = "Inst-1" }[5d:300s])""" @@ -1038,7 +1038,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS val expected = """T~PeriodicSamplesMapper(start=1634775000000, step=0, end=1634775000000, window=Some(432000000), functionId=Some(SumOverTime), rawSource=false, offsetMs=None) - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher + |-E~StitchRvsExec() 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) @@ -1057,7 +1057,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS plannerParams = PlannerParams(processMultiPartition = true))) val expected = - """E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher + """E~StitchRvsExec() 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) @@ -1076,7 +1076,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS plannerParams = PlannerParams(processMultiPartition = true))) val expected = - """E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher + """E~StitchRvsExec() 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) @@ -1145,7 +1145,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS } // TODO max_over_time should have been pushed down for subqueries but ShardKeyRegexPlanner - // in materializeOthers() creates MultiPartitionDistConcatExec + // in materializeOthers() creates StitchRvsExec it("verification of common subquery cases: aggregation of smoothed aggregation function") { val query ="""max_over_time(rate(foo{_ws_ = "demo", _ns_ =~ ".*Ns", instance = "Inst-1" }[10m])[1h:1m])""" val endSecs = 1634775000L @@ -1156,7 +1156,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS val expected = """T~PeriodicSamplesMapper(start=1634775000000, step=0, end=1634775000000, window=Some(3600000), functionId=Some(MaxOverTime), rawSource=false, offsetMs=None) - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher + |-E~StitchRvsExec() 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) @@ -1167,7 +1167,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS } // TODO max_over_time should have been pushed down for subqueries but ShardKeyRegexPlanner - // in materializeOthers() creates MultiPartitionDistConcatExec + // in materializeOthers() creates StitchRvsExec it("verification of common subquery cases: nested subquery") { val query ="""max_over_time(deriv(rate(foo{_ws_ = "demo", _ns_ =~ ".*Ns", instance = "Inst-1" }[1m])[5m:1m])[1h:1m])""" val endSecs = 1634775000L @@ -1179,7 +1179,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS val expected = """T~PeriodicSamplesMapper(start=1634775000000, step=0, end=1634775000000, window=Some(3600000), functionId=Some(MaxOverTime), rawSource=false, offsetMs=None) |-T~PeriodicSamplesMapper(start=1634771400000, step=60000, end=1634775000000, window=Some(300000), functionId=Some(Deriv), rawSource=false, offsetMs=None) - |--E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher + |--E~StitchRvsExec() 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) @@ -3067,52 +3067,52 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS Test(query = """foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"} + on (tschemaLabel) baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bat"}""", tschemaEnabled = Set(), expected = """E~BinaryJoinExec(binaryOp=ADD, on=List(tschemaLabel), 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,Some(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_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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,Some(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(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bat"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bat"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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 = """foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"} + on (tschemaLabel) baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bat"}""", tschemaEnabled = Set("oneRemote1"), expected = """E~BinaryJoinExec(binaryOp=ADD, on=List(tschemaLabel), 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,Some(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_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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,Some(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(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bat"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bat"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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 = """foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"} + on (tschemaLabel) baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bat"}""", tschemaEnabled = Set("oneRemote1", "oneRemote2"), expected = """E~BinaryJoinExec(binaryOp=ADD, on=List(tschemaLabel), 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,Some(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_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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,Some(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(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bat"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bat"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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 = """foo{_ws_="demo",_ns_=~"twoRemote.*",tschemaLabel="bar"} + on (tschemaLabel) baz{_ws_="demo",_ns_=~"twoRemote.*",tschemaLabel="bat"}""", tschemaEnabled = Set("twoRemote1", "twoRemote2"), expected = """E~BinaryJoinExec(binaryOp=ADD, on=List(tschemaLabel), 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,Some(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_=~"twoRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=twoRemote1-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(foo{_ws_="demo",_ns_=~"twoRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=twoRemote2-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,Some(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(baz{_ws_="demo",_ns_=~"twoRemote.*",tschemaLabel="bat"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=twoRemote1-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(baz{_ws_="demo",_ns_=~"twoRemote.*",tschemaLabel="bat"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=twoRemote2-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 = """foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"} + baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bat"}""", tschemaEnabled = Set("oneRemote1", "oneRemote2"), expected = """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,Some(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_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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,Some(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(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bat"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bat"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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 = """foo{_ws_="demo",_ns_=~"both.*",tschemaLabel="bar"} + on (tschemaLabel) baz{_ws_="demo",_ns_=~"both.*",tschemaLabel="bat"}""", tschemaEnabled = Set(), expected = """E~BinaryJoinExec(binaryOp=ADD, on=List(tschemaLabel), 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,Some(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#1790138487],raw) |----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) @@ -3125,7 +3125,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |----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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],downsample) |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"both.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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,Some(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#1790138487],raw) |----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) @@ -3141,7 +3141,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS Test(query = """foo{_ws_="demo",_ns_=~"both.*",tschemaLabel="bar"} + on (tschemaLabel) baz{_ws_="demo",_ns_=~"both.*",tschemaLabel="bat"}""", tschemaEnabled = Set("local"), expected = """E~BinaryJoinExec(binaryOp=ADD, on=List(tschemaLabel), 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,Some(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#1790138487],raw) |----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) @@ -3154,7 +3154,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |----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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],downsample) |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"both.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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,Some(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#1790138487],raw) |----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) @@ -3170,7 +3170,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS Test(query = """foo{_ws_="demo",_ns_=~"both.*",tschemaLabel="bar"} + on (tschemaLabel) baz{_ws_="demo",_ns_=~"both.*",tschemaLabel="bat"}""", tschemaEnabled = Set("remote"), expected = """E~BinaryJoinExec(binaryOp=ADD, on=List(tschemaLabel), 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,Some(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#1790138487],raw) |----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) @@ -3183,7 +3183,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |----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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1790138487],downsample) |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"both.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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,Some(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#1790138487],raw) |----T~PeriodicSamplesMapper(start=1634173130000, step=300000, end=1634777330000, window=None, functionId=None, rawSource=true, offsetMs=None) @@ -3199,19 +3199,19 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS Test(query = """foo{_ws_="demo",_ns_=~"oneRemote.*"} + on (tschemaLabel) baz{_ws_="demo",_ns_=~"oneRemote.*"}""", tschemaEnabled = Set("oneRemote1", "oneRemote2"), expected = """E~BinaryJoinExec(binaryOp=ADD, on=List(tschemaLabel), 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,Some(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_=~"oneRemote.*"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(foo{_ws_="demo",_ns_=~"oneRemote.*"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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,Some(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(baz{_ws_="demo",_ns_=~"oneRemote.*"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(baz{_ws_="demo",_ns_=~"oneRemote.*"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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 = """foo{_ws_="demo",_ns_=~"oneRemote.*"} + on (notATargetSchemaLabel) baz{_ws_="demo",_ns_=~"oneRemote.*"}""", tschemaEnabled = Set("oneRemote1", "oneRemote2"), expected = """E~BinaryJoinExec(binaryOp=ADD, on=List(notATargetSchemaLabel), 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,Some(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_=~"oneRemote.*"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(foo{_ws_="demo",_ns_=~"oneRemote.*"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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,Some(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(baz{_ws_="demo",_ns_=~"oneRemote.*"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(baz{_ws_="demo",_ns_=~"oneRemote.*"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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 = """foo{_ws_="demo",_ns_=~"local.*"} + on (tschemaLabel) baz{_ws_="demo",_ns_=~"local.*"}""", @@ -3242,40 +3242,40 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS Test(query = """foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"} + on (tschemaLabel) (baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"} - on(tschemaLabel) bak{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"})""", tschemaEnabled = Set("oneRemote1", "oneRemote2"), expected = """E~BinaryJoinExec(binaryOp=ADD, on=List(tschemaLabel), 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,Some(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_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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~BinaryJoinExec(binaryOp=SUB, on=List(tschemaLabel), 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,Some(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(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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,Some(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(bak{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(bak{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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 = """foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"} + on (notATargetSchemaLabel) (baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"} - on(tschemaLabel) bak{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"})""", tschemaEnabled = Set("oneRemote1", "oneRemote2"), expected = """E~BinaryJoinExec(binaryOp=ADD, on=List(notATargetSchemaLabel), 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,Some(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_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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~BinaryJoinExec(binaryOp=SUB, on=List(tschemaLabel), 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,Some(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(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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,Some(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(bak{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(bak{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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 = """foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"} + on (tschemaLabel) (baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"} - on (notATargetSchemaLabel) bak{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"})""", tschemaEnabled = Set("oneRemote1", "oneRemote2"), expected = """E~BinaryJoinExec(binaryOp=ADD, on=List(tschemaLabel), 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,Some(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_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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~BinaryJoinExec(binaryOp=SUB, on=List(notATargetSchemaLabel), 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,Some(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(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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,Some(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(bak{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(bak{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"} + on (tschemaLabel) baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"}) by (tschemaLabel)""", @@ -3284,16 +3284,16 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |-E~LocalPartitionReduceAggregateExec(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))) |--T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) |---E~BinaryJoinExec(binaryOp=ADD, on=List(tschemaLabel), 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,Some(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_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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,Some(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(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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 = """foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"} + on (tschemaLabel) sum(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"}) by (tschemaLabel)""", tschemaEnabled = Set("oneRemote1", "oneRemote2"), expected = """E~BinaryJoinExec(binaryOp=ADD, on=List(tschemaLabel), 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,Some(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_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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))) |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634777330)) @@ -3414,7 +3414,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS // 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 + """E~StitchRvsExec() on InProcessPlanDispatcher |-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) @@ -3448,7 +3448,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="dummy",_ns_=~"dummy.*",bar="hello"}),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=dummy-endpoint, requestTimeoutMs=10000) on InProcessPlanDispatcher""".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 - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher + |-E~StitchRvsExec() on InProcessPlanDispatcher |--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) @@ -3461,7 +3461,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |----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/testActor],downsample) |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="dummy",_ns_=~"dummy.*",bar="hello"},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=dummy-endpoint, requestTimeoutMs=10000) on InProcessPlanDispatcher - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher + |-E~StitchRvsExec() on InProcessPlanDispatcher |--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) @@ -3521,7 +3521,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS // 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 + """E~StitchRvsExec() on InProcessPlanDispatcher |-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) @@ -3565,7 +3565,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="dummy",_ns_=~"dummy.*",bar="hello"}),1633913330,300,1634777330,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""".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 - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher + |-E~StitchRvsExec() on InProcessPlanDispatcher |--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) @@ -3582,7 +3582,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |----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/testActor],downsample) |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="dummy",_ns_=~"dummy.*",bar="hello"},1633913330,300,1634777330,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 - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher + |-E~StitchRvsExec() on InProcessPlanDispatcher |--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) diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlannerSpec.scala index 6890e32fb3..c8789b34d8 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlannerSpec.scala @@ -120,7 +120,7 @@ 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~StitchRvsExec() 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) @@ -141,7 +141,7 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture lp, QueryContext(origQueryParams = promQlQueryParams, plannerParams = PlannerParams(processMultiPartition = true)) ) - execPlan.isInstanceOf[MultiPartitionDistConcatExec] shouldEqual(true) + execPlan.isInstanceOf[StitchRvsExec] shouldEqual(true) execPlan.children(0).children.head.isInstanceOf[MultiSchemaPartitionsExec] validatePlan(execPlan, expected) } @@ -185,7 +185,7 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture it("should generate Exec plan for top level subquery") { val expected = - """E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher + """E~StitchRvsExec() 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) @@ -201,7 +201,7 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture 1000, 1000 ) val execPlan = skrp.materialize(lp, QueryContext(origQueryParams = promQlQueryParams, plannerParams = PlannerParams(processMultiPartition = true))) - execPlan.isInstanceOf[MultiPartitionDistConcatExec] shouldEqual(true) + execPlan.isInstanceOf[StitchRvsExec] shouldEqual(true) execPlan.children(0).children.head.isInstanceOf[MultiSchemaPartitionsExec] validatePlan(execPlan, expected) } From 9a002eff96c1a4b9090721f36beaeba6a1204b20 Mon Sep 17 00:00:00 2001 From: Alex Theimer Date: Wed, 4 Oct 2023 22:11:56 -0700 Subject: [PATCH 09/27] remove outdated test --- .../MultiPartitionPlannerSpec.scala | 58 ------------------- 1 file changed, 58 deletions(-) diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala index b5d623a648..52762d1823 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala @@ -941,64 +941,6 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers with PlanValida remoteExec.queryEndpoint shouldEqual "remote-url2" } -// // TODO: what is this the purpose of this test? The lookback is 300s, and the assignment -// // durations are-- in order-- 300s and 299s,999ms. -// it ("should generate second Exec with start and end time equal to query end time when query duration is less" + -// "than or equal to lookback ") { -// -// val startSeconds = 1594309980L -// val endSeconds = 1594310280L -// val localPartitionStartMs: Long = 1594309980001L -// val step = 15L -// -// val partitionLocationProvider = new PartitionLocationProvider { -// override def getPartitions(routingKey: Map[String, String], timeRange: TimeRange): List[PartitionAssignment] = { -// if (routingKey.equals(Map("job" -> "app"))) List( -// PartitionAssignment("remote1", "remote-url", TimeRange(startSeconds * 1000 - lookbackMs, -// localPartitionStartMs - 1)), PartitionAssignment("remote2", "remote-url", -// TimeRange(localPartitionStartMs, endSeconds * 1000))) -// else Nil -// } -// -// override def getMetadataPartitions(nonMetricShardKeyFilters: Seq[ColumnFilter], timeRange: TimeRange): List[PartitionAssignment] = List( -// PartitionAssignment("remote1", "remote-url", TimeRange(startSeconds * 1000 - lookbackMs, -// localPartitionStartMs - 1)), PartitionAssignment("remote2", "remote-url", -// TimeRange(localPartitionStartMs, endSeconds * 1000))) -// -// } -// val engine = new MultiPartitionPlanner(partitionLocationProvider, localPlanner, "local", dataset, queryConfig) -// val lp = Parser.queryRangeToLogicalPlan("test{job = \"app\"}", TimeStepParams(startSeconds, step, endSeconds)) -// -// val promQlQueryParams = PromQlQueryParams("test{job = \"app\"}", startSeconds, step, endSeconds) -// -// val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams, plannerParams = -// PlannerParams(processMultiPartition = true))) -// val stitchRvsExec = execPlan.asInstanceOf[StitchRvsExec] -// stitchRvsExec.children.size shouldEqual (2) -// stitchRvsExec.children(0).isInstanceOf[PromQlRemoteExec] shouldEqual (true) -// stitchRvsExec.children(1).isInstanceOf[PromQlRemoteExec] shouldEqual (true) -// -// -// val remoteExec = stitchRvsExec.children(0).asInstanceOf[PromQlRemoteExec] -// val queryParams = remoteExec.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams] -// queryParams.startSecs shouldEqual startSeconds -// queryParams.endSecs shouldEqual (localPartitionStartMs - 1) / 1000 -// queryParams.stepSecs shouldEqual step -// remoteExec.queryContext.plannerParams.processFailure shouldEqual true -// remoteExec.queryContext.plannerParams.processMultiPartition shouldEqual false -// remoteExec.queryEndpoint shouldEqual "remote-url" -// -// val remoteExec2 = stitchRvsExec.children(1).asInstanceOf[PromQlRemoteExec] -// val queryParams2 = remoteExec2.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams] -// queryParams2.startSecs shouldEqual endSeconds -// queryParams2.endSecs shouldEqual endSeconds -// queryParams2.stepSecs shouldEqual step -// remoteExec2.queryContext.plannerParams.processFailure shouldEqual true -// remoteExec2.queryContext.plannerParams.processMultiPartition shouldEqual false -// remoteExec2.queryEndpoint shouldEqual "remote-url" -// -// } - it ("should generate Exec plan for Metadata Label values query") { def partitions(timeRange: TimeRange): List[PartitionAssignment] = List(PartitionAssignment("remote", "remote-url", From 81cd1d60fd65baee41003025793b5fdd47ac12a7 Mon Sep 17 00:00:00 2001 From: Alex Theimer Date: Mon, 9 Oct 2023 11:58:44 -0700 Subject: [PATCH 10/27] add test comment --- .../queryplanner/MultiPartitionPlannerSpec.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala index 52762d1823..af4d148461 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala @@ -899,7 +899,10 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers with PlanValida } - it ("should generate at most one PromQlRemoteExec plan for non-shard-key-regex instant queries") { + it ("should generate at most one PromQlRemoteExec plan for split non-shard-key-regex instant queries") { + // This is an instant query, so only one timestamp is evaluated. There is enough space on the second partition + // (remote2) to fit the query's 100s lookback. We should expect only one RemoteExec to be materialized, + // and it should be materialized for remote2. val startSeconds = 1000 val endSeconds = 1000 val localPartitionStartSec = 850 From a32eebcaa99d19a434e55a9d95d6313bc393ce45 Mon Sep 17 00:00:00 2001 From: Alex Theimer Date: Mon, 9 Oct 2023 21:49:44 -0700 Subject: [PATCH 11/27] add EmptyResultExec test --- .../MultiPartitionPlannerSpec.scala | 35 +++++++++++++++++++ 1 file changed, 35 insertions(+) diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala index af4d148461..f692f30503 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala @@ -899,6 +899,41 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers with PlanValida } + it("should generate an empty plan when partition-split shard-key data is sourced from multiple " + + "partitions for all timesteps") { + // This is an instant query, so only one timestamp is evaluated. There is not enough space on the second partition + // (remote2) to fit the query's 100s lookback. We should expect an empty plan to be materialized. + val startSeconds = 1000 + val endSeconds = 1000 + val localPartitionStartSec = 950 + val lookbackMs = 100000 + val step = 1 + + val partitionLocationProvider = new PartitionLocationProvider { + override def getPartitions(routingKey: Map[String, String], timeRange: TimeRange): List[PartitionAssignment] = { + if (routingKey.equals(Map("job" -> "app"))) + List(PartitionAssignment("remote1", "remote-url1", TimeRange(startSeconds * 1000 - lookbackMs, + localPartitionStartSec * 1000 - 1)), PartitionAssignment("remote2", "remote-url2", + TimeRange(localPartitionStartSec * 1000, endSeconds * 1000))) + else Nil + } + + override def getMetadataPartitions(nonMetricShardKeyFilters: Seq[ColumnFilter], timeRange: TimeRange): List[PartitionAssignment] = + List(PartitionAssignment("remote", "remote-url", TimeRange(startSeconds * 1000 - lookbackMs, + localPartitionStartSec * 1000 - 1)), PartitionAssignment("local", "local-url", + TimeRange(localPartitionStartSec * 1000, endSeconds * 1000))) + } + val engine = new MultiPartitionPlanner(partitionLocationProvider, localPlanner, "local", dataset, queryConfig) + val lp = Parser.queryRangeToLogicalPlan("test{job = \"app\"}[100s]", TimeStepParams(startSeconds, step, endSeconds)) + + val promQlQueryParams = PromQlQueryParams("test{job = \"app\"}[100s]", startSeconds, step, endSeconds) + + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams, + plannerParams = PlannerParams(processMultiPartition = true))) + + execPlan.isInstanceOf[EmptyResultExec] shouldEqual (true) + } + it ("should generate at most one PromQlRemoteExec plan for split non-shard-key-regex instant queries") { // This is an instant query, so only one timestamp is evaluated. There is enough space on the second partition // (remote2) to fit the query's 100s lookback. We should expect only one RemoteExec to be materialized, From f0168890c940fc0cb5fab51d1fe4bf5ee9cf9143 Mon Sep 17 00:00:00 2001 From: Alex Theimer Date: Mon, 9 Oct 2023 22:50:17 -0700 Subject: [PATCH 12/27] update QueryUtils and add test Co-authored-by: Amol Nayak --- .../queryplanner/QueryUtils.scala | 27 +++------- .../queryplanner/QueryUtilsSpec.scala | 51 +++++++++++++++++++ 2 files changed, 59 insertions(+), 19 deletions(-) create mode 100644 coordinator/src/test/scala/filodb.coordinator/queryplanner/QueryUtilsSpec.scala diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/QueryUtils.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/QueryUtils.scala index 425eeabb00..10617733b7 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/QueryUtils.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/QueryUtils.scala @@ -1,7 +1,5 @@ package filodb.coordinator.queryplanner -import scala.collection.mutable - /** * Storage for miscellaneous utility functions. */ @@ -14,22 +12,13 @@ object QueryUtils { * @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 + def combinations[T](choices: Seq[Seq[T]]): Seq[Seq[T]] = choices match { + case Nil => Nil + case head +: Nil => head.map(Seq(_)) + case head +: tail => + val tailCombos = combinations(tail) + head.flatMap(h => { + tailCombos.map(Seq(h) ++ _) + }) } } diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/QueryUtilsSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/QueryUtilsSpec.scala new file mode 100644 index 0000000000..7af4d300bb --- /dev/null +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/QueryUtilsSpec.scala @@ -0,0 +1,51 @@ +package filodb.coordinator.queryplanner + +import org.scalatest.funspec.AnyFunSpec +import org.scalatest.matchers.should.Matchers + +class QueryUtilsSpec extends AnyFunSpec with Matchers { + it("should correctly calculate combinations") { + // Pairs of input choice lists and output combinations. + val testPairs = Seq( + (Seq(), Seq()), + ( + Seq(Seq(1)), + Seq(Seq(1)) + ), + ( + Seq(Seq(1, 2)), + Seq( + Seq(1), + Seq(2)) + ), + ( + Seq( + Seq(1), + Seq(2)), + Seq(Seq(1, 2)) + ), + ( + Seq( + Seq(1), + Seq(2, 3)), + Seq( + Seq(1, 2), + Seq(1, 3)) + ), + ( + Seq( + Seq(1, 2), + Seq(3, 4)), + Seq( + Seq(1, 3), + Seq(1, 4), + Seq(2, 3), + Seq(2, 4)) + ) + ) + for ((choices, expected) <- testPairs) { + val res = QueryUtils.combinations(choices) + res.toSet shouldEqual expected.toSet + } + } +} From c5f2a2997e825e6301eadf01f12ead801b02f572 Mon Sep 17 00:00:00 2001 From: Alex Theimer Date: Tue, 9 Jan 2024 09:46:48 -0800 Subject: [PATCH 13/27] resolve remaining build conflicts --- .../queryplanner/MultiPartitionPlanner.scala | 9 ++++++++- .../queryplanner/MultiPartitionPlannerSpec.scala | 1 - 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala index 2d87a15333..f8413e0207 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala @@ -227,7 +227,14 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider case lp: ApplyLimitFunction => super.materializeLimitFunction(qContext, lp) case lp: TsCardinalities => materializeTsCardinalities(lp, qContext) case lp: TopLevelSubquery => super.materializeTopLevelSubquery(qContext, lp) - case _: PeriodicSeriesWithWindowing | + case _: BinaryJoin | + _: ApplyInstantFunction | + _: Aggregate | + _: ScalarVectorBinaryOperation | + _: ScalarVaryingDoublePlan | + _: ApplyAbsentFunction | + _: SubqueryWithWindowing | + _: PeriodicSeriesWithWindowing | _: PeriodicSeries | _: RawChunkMeta => materializePlanHandleSplitLeaf(logicalPlan, qContext) case raw: RawSeries => diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala index f645861a53..7f756ac3e9 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala @@ -959,7 +959,6 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers with PlanValida // remoteExec2.queryEndpoint shouldEqual "remote-url" // // } ->>>>>>> develop it ("should generate Exec plan for Metadata Label values query") { def partitions(timeRange: TimeRange): List[PartitionAssignment] = From 6dd2e7a43ebfd1e5454114b7546e975cbfa6af47 Mon Sep 17 00:00:00 2001 From: Alex Theimer Date: Tue, 9 Jan 2024 09:55:14 -0800 Subject: [PATCH 14/27] resolve MultiPartitionPlannerSpec --- .../MultiPartitionPlannerSpec.scala | 162 +++++++++--------- 1 file changed, 81 insertions(+), 81 deletions(-) diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala index 7f756ac3e9..8c91d4469e 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala @@ -101,13 +101,13 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers with PlanValida val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams, plannerParams = PlannerParams(processMultiPartition = true))) // Notice how the cap between 2999 and 3300 is filled by pulling raw data from both partitions by enabling supportRawExport - val expectedPlanTree = s"""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),true,1800000 milliseconds)) - |-E~PromQlRemoteExec(PromQlQueryParams(test{job = "app"},1000,100,2999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,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=remote-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),true,1800000 milliseconds)) + val expectedPlanTree = s"""E~StitchRvsExec() on InProcessPlanDispatcher + |-E~PromQlRemoteExec(PromQlQueryParams(test{job="app"},1000,100,2999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,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=10000) on InProcessPlanDispatcher |-T~PeriodicSamplesMapper(start=2999000, step=100000, end=3299000, window=None, functionId=None, rawSource=false, offsetMs=None) - |--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),true,1800000 milliseconds)) - |---E~PromQlRemoteExec(PromQlQueryParams(test{job="app"}[300s],2999,1,2999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,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=remote-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),true,1800000 milliseconds)) - |---E~PromQlRemoteExec(PromQlQueryParams(test{job="app"}[299s],3299,1,3299,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,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=remote-url2, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),true,1800000 milliseconds)) - |-E~PromQlRemoteExec(PromQlQueryParams(test{job = "app"},3300,100,10000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,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=remote-url2, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),true,1800000 milliseconds))""".stripMargin + |--E~StitchRvsExec() on InProcessPlanDispatcher + |---E~PromQlRemoteExec(PromQlQueryParams(test{job="app"}[300s],2999,1,2999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,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=10000) on InProcessPlanDispatcher + |---E~PromQlRemoteExec(PromQlQueryParams(test{job="app"}[299s],3299,1,3299,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,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-url2, requestTimeoutMs=10000) on InProcessPlanDispatcher + |-E~PromQlRemoteExec(PromQlQueryParams(test{job="app"},3300,100,10000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,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-url2, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin validatePlan(execPlan, expectedPlanTree) } @@ -305,9 +305,9 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers with PlanValida ) ) val expectedPlan = - """E~BinaryJoinExec(binaryOp=ADD, on=None, ignoring=List()) on InProcessPlanDispatcher(filodb.core.query.QueryConfig@1dad01fe) - |-E~PromQlRemoteExec(PromQlQueryParams(test{job="app1"},3000,100,12000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,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=remote-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(filodb.core.query.QueryConfig@1dad01fe) - |-E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1232106303],raw) + """E~BinaryJoinExec(binaryOp=ADD, on=None, ignoring=List()) on InProcessPlanDispatcher + |-E~PromQlRemoteExec(PromQlQueryParams(test{job="app1"},3000,100,12000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,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=10000) on InProcessPlanDispatcher + |-E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) |--T~PeriodicSamplesMapper(start=3000000, step=100000, end=12000000, window=None, functionId=None, rawSource=true, offsetMs=None) |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=15, chunkMethod=TimeRangeChunkScan(2700000,12000000), filters=List(ColumnFilter(job,Equals(app2)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) |--T~PeriodicSamplesMapper(start=3000000, step=100000, end=12000000, window=None, functionId=None, rawSource=true, offsetMs=None) @@ -354,9 +354,9 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers with PlanValida ) execPlan.printTree() val expectedPlan = { - """E~BinaryJoinExec(binaryOp=ADD, on=None, ignoring=List()) on InProcessPlanDispatcher(filodb.core.query.QueryConfig@7c0f28f8) - |-E~PromQlRemoteExec(PromQlQueryParams(test{job="app1"},3000,100,12000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,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=remote-url1, requestTimeoutMs=10000) on InProcessPlanDispatcher(filodb.core.query.QueryConfig@7c0f28f8) - |-E~PromQlRemoteExec(PromQlQueryParams(test{job="app2"},3000,100,12000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,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=remote-url2, requestTimeoutMs=10000) on InProcessPlanDispatcher(filodb.core.query.QueryConfig@7c0f28f8)""".stripMargin + """E~BinaryJoinExec(binaryOp=ADD, on=None, ignoring=List()) on InProcessPlanDispatcher + |-E~PromQlRemoteExec(PromQlQueryParams(test{job="app1"},3000,100,12000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,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-url1, requestTimeoutMs=10000) on InProcessPlanDispatcher + |-E~PromQlRemoteExec(PromQlQueryParams(test{job="app2"},3000,100,12000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,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-url2, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin } validatePlan(execPlan, expectedPlan) } @@ -402,9 +402,9 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers with PlanValida """T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(3000,100,12000)) |-E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on InProcessPlanDispatcher |--T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |---E~BinaryJoinExec(binaryOp=ADD, on=None, ignoring=List()) on InProcessPlanDispatcher(filodb.core.query.QueryConfig@2a8a3ada) - |----E~PromQlRemoteExec(PromQlQueryParams(test{job="app1"},3000,100,12000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,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=remote-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(filodb.core.query.QueryConfig@2a8a3ada) - |----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1705462153],raw) + |---E~BinaryJoinExec(binaryOp=ADD, on=None, ignoring=List()) on InProcessPlanDispatcher + |----E~PromQlRemoteExec(PromQlQueryParams(test{job="app1"},3000,100,12000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,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=10000) on InProcessPlanDispatcher + |----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) |-----T~PeriodicSamplesMapper(start=3000000, step=100000, end=12000000, window=None, functionId=None, rawSource=true, offsetMs=None) |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=15, chunkMethod=TimeRangeChunkScan(2700000,12000000), filters=List(ColumnFilter(job,Equals(app2)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) |-----T~PeriodicSamplesMapper(start=3000000, step=100000, end=12000000, window=None, functionId=None, rawSource=true, offsetMs=None) @@ -512,9 +512,9 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers with PlanValida ) val expectedPlan = """T~PeriodicSamplesMapper(start=1200000, step=120000, end=1800000, window=Some(600000), functionId=Some(AvgOverTime), rawSource=false, offsetMs=None) - |-E~BinaryJoinExec(binaryOp=ADD, on=None, ignoring=List()) on InProcessPlanDispatcher(filodb.core.query.QueryConfig@4684240f) - |--E~PromQlRemoteExec(PromQlQueryParams(test{job="app1"},600,60,1800,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,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=remote-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(filodb.core.query.QueryConfig@4684240f) - |--E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1103787767],raw) + |-E~BinaryJoinExec(binaryOp=ADD, on=None, ignoring=List()) on InProcessPlanDispatcher + |--E~PromQlRemoteExec(PromQlQueryParams(test{job="app1"},600,60,1800,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,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=10000) on InProcessPlanDispatcher + |--E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) |---T~PeriodicSamplesMapper(start=600000, step=60000, end=1800000, window=None, functionId=None, rawSource=true, offsetMs=None) |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=15, chunkMethod=TimeRangeChunkScan(300000,1800000), filters=List(ColumnFilter(job,Equals(app2)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) |---T~PeriodicSamplesMapper(start=600000, step=60000, end=1800000, window=None, functionId=None, rawSource=true, offsetMs=None) @@ -553,7 +553,7 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers with PlanValida val expectedPlan = """T~PeriodicSamplesMapper(start=1200000, step=120000, end=1800000, window=Some(600000), functionId=Some(MinOverTime), rawSource=false, offsetMs=None) |-E~BinaryJoinExec(binaryOp=ADD, on=None, ignoring=List()) on InProcessPlanDispatcher - |--E~PromQlRemoteExec(PromQlQueryParams(sum_over_time(sum(test{job="app1"})[600s:60s]),600,60,1800,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,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=remote-url, requestTimeoutMs=10000) on InProcessPlanDispatcher + |--E~PromQlRemoteExec(PromQlQueryParams(sum_over_time(sum(test{job="app1"})[600s:60s]),600,60,1800,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,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=10000) on InProcessPlanDispatcher |--T~PeriodicSamplesMapper(start=600000, step=60000, end=1800000, window=Some(600000), functionId=Some(SumOverTime), rawSource=false, offsetMs=None) |---T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(0,60,1800)) |----E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) @@ -839,20 +839,20 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers with PlanValida val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams, plannerParams = PlannerParams(processMultiPartition = true))) // Even a three partition span works with stitch filling in the missing gaps - val expectedRawPlan = s"""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),true,1800000 milliseconds)) - |-E~PromQlRemoteExec(PromQlQueryParams(test{job = "app"},1000,100,3999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,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=remote-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),true,1800000 milliseconds)) + val expectedRawPlan = s"""E~StitchRvsExec() on InProcessPlanDispatcher + |-E~PromQlRemoteExec(PromQlQueryParams(test{job="app"},1000,100,3999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,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-url1, requestTimeoutMs=10000) on InProcessPlanDispatcher |-T~PeriodicSamplesMapper(start=3999000, step=100000, end=4299000, window=None, functionId=None, rawSource=false, offsetMs=None) - |--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),true,1800000 milliseconds)) - |---E~PromQlRemoteExec(PromQlQueryParams(test{job="app"}[300s],3999,1,3999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,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=remote-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),true,1800000 milliseconds)) - |---E~PromQlRemoteExec(PromQlQueryParams(test{job="app"}[299s],4299,1,4299,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,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=remote-url2, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),true,1800000 milliseconds)) - |---E~PromQlRemoteExec(PromQlQueryParams(test{job="app"}[-2701s],4299,1,4299,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,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=remote-url3, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),true,1800000 milliseconds)) - |-E~PromQlRemoteExec(PromQlQueryParams(test{job = "app"},4300,100,6999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,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=remote-url2, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),true,1800000 milliseconds)) + |--E~StitchRvsExec() on InProcessPlanDispatcher + |---E~PromQlRemoteExec(PromQlQueryParams(test{job="app"}[300s],3999,1,3999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,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-url1, requestTimeoutMs=10000) on InProcessPlanDispatcher + |---E~PromQlRemoteExec(PromQlQueryParams(test{job="app"}[299s],4299,1,4299,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,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-url2, requestTimeoutMs=10000) on InProcessPlanDispatcher + |---E~PromQlRemoteExec(PromQlQueryParams(test{job="app"}[-2701s],4299,1,4299,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,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-url3, requestTimeoutMs=10000) on InProcessPlanDispatcher + |-E~PromQlRemoteExec(PromQlQueryParams(test{job="app"},4300,100,6999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,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-url2, requestTimeoutMs=10000) on InProcessPlanDispatcher |-T~PeriodicSamplesMapper(start=6999000, step=100000, end=7299000, window=None, functionId=None, rawSource=false, offsetMs=None) - |--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),true,1800000 milliseconds)) - |---E~PromQlRemoteExec(PromQlQueryParams(test{job="app"}[-2699s],3999,1,3999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,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=remote-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),true,1800000 milliseconds)) - |---E~PromQlRemoteExec(PromQlQueryParams(test{job="app"}[300s],6999,1,6999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,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=remote-url2, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),true,1800000 milliseconds)) - |---E~PromQlRemoteExec(PromQlQueryParams(test{job="app"}[299s],7299,1,7299,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,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=remote-url3, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),true,1800000 milliseconds)) - |-E~PromQlRemoteExec(PromQlQueryParams(test{job = "app"},7300,100,10000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,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=remote-url3, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),true,1800000 milliseconds))""".stripMargin + |--E~StitchRvsExec() on InProcessPlanDispatcher + |---E~PromQlRemoteExec(PromQlQueryParams(test{job="app"}[-2699s],3999,1,3999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,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-url1, requestTimeoutMs=10000) on InProcessPlanDispatcher + |---E~PromQlRemoteExec(PromQlQueryParams(test{job="app"}[300s],6999,1,6999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,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-url2, requestTimeoutMs=10000) on InProcessPlanDispatcher + |---E~PromQlRemoteExec(PromQlQueryParams(test{job="app"}[299s],7299,1,7299,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,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-url3, requestTimeoutMs=10000) on InProcessPlanDispatcher + |-E~PromQlRemoteExec(PromQlQueryParams(test{job="app"},7300,100,10000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,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-url3, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin validatePlan(execPlan, expectedRawPlan) } @@ -888,9 +888,9 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers with PlanValida val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams, plannerParams = PlannerParams(processMultiPartition = true))) - val expectedPlanString = s"""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),true,1800000 milliseconds)) - |-E~PromQlRemoteExec(PromQlQueryParams(test{job="app"}[49s],949,1,949,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,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=remote-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),true,1800000 milliseconds)) - |-E~PromQlRemoteExec(PromQlQueryParams(test{job="app"}[50s],1000,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), queryEndpoint=remote-url2, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),true,1800000 milliseconds))""".stripMargin + val expectedPlanString = s"""E~StitchRvsExec() on InProcessPlanDispatcher + |-E~PromQlRemoteExec(PromQlQueryParams(test{job="app"}[49s],949,1,949,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,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-url1, requestTimeoutMs=10000) on InProcessPlanDispatcher + |-E~PromQlRemoteExec(PromQlQueryParams(test{job="app"}[50s],1000,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-url2, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin validatePlan(execPlan, expectedPlanString) } @@ -1458,8 +1458,8 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers with PlanValida // binary join for + should be done in process val expectedPlan = - """E~BinaryJoinExec(binaryOp=ADD, on=None, ignoring=List()) on InProcessPlanDispatcher(filodb.core.query.QueryConfig@298b64f7) - |-E~BinaryJoinExec(binaryOp=MUL, on=None, ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#450220728],raw) + """E~BinaryJoinExec(binaryOp=ADD, on=None, ignoring=List()) on InProcessPlanDispatcher + |-E~BinaryJoinExec(binaryOp=MUL, on=None, ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) |--T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1000,100,10000)) |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) @@ -1477,9 +1477,9 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers with PlanValida |-----T~PeriodicSamplesMapper(start=1000000, step=100000, end=10000000, window=None, functionId=None, rawSource=true, offsetMs=None) |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,10000000), filters=List(ColumnFilter(job,Equals(app1)), ColumnFilter(__name__,Equals(test2))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) |-T~InstantVectorFunctionMapper(function=Ln) - |--E~BinaryJoinExec(binaryOp=ADD, on=None, ignoring=List()) on InProcessPlanDispatcher(filodb.core.query.QueryConfig@298b64f7) - |---E~PromQlRemoteExec(PromQlQueryParams(sum(test3{job="app2"}),1000,100,10000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,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=remote-url-1, requestTimeoutMs=10000) on InProcessPlanDispatcher(filodb.core.query.QueryConfig@298b64f7) - |---E~PromQlRemoteExec(PromQlQueryParams(sum(test4{job="app3"}),1000,100,10000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,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=remote-url-2, requestTimeoutMs=10000) on InProcessPlanDispatcher(filodb.core.query.QueryConfig@298b64f7)""".stripMargin + |--E~BinaryJoinExec(binaryOp=ADD, on=None, ignoring=List()) on InProcessPlanDispatcher + |---E~PromQlRemoteExec(PromQlQueryParams(sum(test3{job="app2"}),1000,100,10000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,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-1, requestTimeoutMs=10000) on InProcessPlanDispatcher + |---E~PromQlRemoteExec(PromQlQueryParams(sum(test4{job="app3"}),1000,100,10000,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,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-2, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin validatePlan(execPlan, expectedPlan) @@ -1662,25 +1662,25 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers with PlanValida } val expectedPlanWithRemoteExport = - """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),true,1800000 milliseconds)) - |-E~PromQlRemoteExec(PromQlQueryParams(sum(rate(test{job = "app"}[10m])),1000,100,6999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,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=remote-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),true,1800000 milliseconds)) + """E~StitchRvsExec() on InProcessPlanDispatcher + |-E~PromQlRemoteExec(PromQlQueryParams(sum(rate(test{job="app"}[600s])),1000,100,6999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,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=10000) on InProcessPlanDispatcher |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(6999,100,7599)) - |--E~LocalPartitionReduceAggregateExec(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),true,1800000 milliseconds)) + |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on InProcessPlanDispatcher |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) |----T~PeriodicSamplesMapper(start=6999000, step=100000, end=7599000, window=Some(600000), functionId=Some(Rate), rawSource=false, offsetMs=None) - |-----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),true,1800000 milliseconds)) - |------E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#862629852],raw) - |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=3, chunkMethod=TimeRangeChunkScan(7000000,7599000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#862629852],raw) - |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=19, chunkMethod=TimeRangeChunkScan(7000000,7599000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#862629852],raw) - |------E~PromQlRemoteExec(PromQlQueryParams(test{job="app"}[600s],6999,1,6999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,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=remote-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),true,1800000 milliseconds)) - |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(7600,100,10000)) - |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#862629852],raw) + |-----E~StitchRvsExec() on InProcessPlanDispatcher + |------E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=3, chunkMethod=TimeRangeChunkScan(6400000,7599000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=19, chunkMethod=TimeRangeChunkScan(6400000,7599000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |------E~PromQlRemoteExec(PromQlQueryParams(test{job="app"}[600s],6999,1,6999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,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=10000) on InProcessPlanDispatcher + |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(2000,100,10000)) + |--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=7600000, step=100000, end=10000000, window=Some(600000), functionId=Some(Rate), rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=3, chunkMethod=TimeRangeChunkScan(7000000,10000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#862629852],raw) + |----T~PeriodicSamplesMapper(start=2000000, step=100000, end=10000000, window=Some(600000), functionId=Some(Rate), rawSource=true, offsetMs=None) + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=3, chunkMethod=TimeRangeChunkScan(1400000,10000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,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=7600000, step=100000, end=10000000, window=Some(600000), functionId=Some(Rate), rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=19, chunkMethod=TimeRangeChunkScan(7000000,10000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#862629852],raw)""".stripMargin + |----T~PeriodicSamplesMapper(start=2000000, step=100000, end=10000000, window=Some(600000), functionId=Some(Rate), rawSource=true, offsetMs=None) + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=19, chunkMethod=TimeRangeChunkScan(1400000,10000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw)""".stripMargin val engine = new MultiPartitionPlanner(partitionLocationProvider, localPlanner, "local", dataset, queryConfig.copy(supportRemoteRawExport = true)) @@ -1693,44 +1693,44 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers with PlanValida validatePlan(execPlan1, expectedPlanWithRemoteExport) val expectedPlanWithRemoteExec1 = - """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),true,1800000 milliseconds)) - |-E~PromQlRemoteExec(PromQlQueryParams(sum(rate(test{job = "app"}[10m])) + sum(rate(bar{job = "app"}[5m])),1000,100,6999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,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=remote-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),true,1800000 milliseconds)) - |-E~BinaryJoinExec(binaryOp=ADD, on=None, 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),true,1800000 milliseconds)) + """E~StitchRvsExec() on InProcessPlanDispatcher + |-E~PromQlRemoteExec(PromQlQueryParams((sum(rate(test{job="app"}[600s])) + sum(rate(bar{job="app"}[300s]))),1000,100,6999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,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=10000) on InProcessPlanDispatcher + |-E~BinaryJoinExec(binaryOp=ADD, on=None, ignoring=List()) on InProcessPlanDispatcher |--T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(6999,100,7599)) - |---E~LocalPartitionReduceAggregateExec(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),true,1800000 milliseconds)) + |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on InProcessPlanDispatcher |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) |-----T~PeriodicSamplesMapper(start=6999000, step=100000, end=7599000, window=Some(600000), functionId=Some(Rate), rawSource=false, offsetMs=None) - |------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),true,1800000 milliseconds)) - |-------E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#304042718],raw) - |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=3, chunkMethod=TimeRangeChunkScan(7000000,7599000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#304042718],raw) - |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=19, chunkMethod=TimeRangeChunkScan(7000000,7599000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#304042718],raw) - |-------E~PromQlRemoteExec(PromQlQueryParams(test{job="app"}[600s],6999,1,6999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,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=remote-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),true,1800000 milliseconds)) + |------E~StitchRvsExec() on InProcessPlanDispatcher + |-------E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=3, chunkMethod=TimeRangeChunkScan(6400000,7599000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=19, chunkMethod=TimeRangeChunkScan(6400000,7599000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |-------E~PromQlRemoteExec(PromQlQueryParams(test{job="app"}[600s],6999,1,6999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,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=10000) on InProcessPlanDispatcher |--T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(6999,100,7599)) - |---E~LocalPartitionReduceAggregateExec(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),true,1800000 milliseconds)) + |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on InProcessPlanDispatcher |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) |-----T~PeriodicSamplesMapper(start=6999000, step=100000, end=7599000, window=Some(300000), functionId=Some(Rate), rawSource=false, offsetMs=None) - |------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),true,1800000 milliseconds)) - |-------E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#304042718],raw) - |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=11, chunkMethod=TimeRangeChunkScan(7000000,7599000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#304042718],raw) - |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=27, chunkMethod=TimeRangeChunkScan(7000000,7599000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#304042718],raw) - |-------E~PromQlRemoteExec(PromQlQueryParams(bar{job="app"}[300s],6999,1,6999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,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=remote-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),true,1800000 milliseconds)) - |-E~BinaryJoinExec(binaryOp=ADD, on=None, ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#304042718],raw) - |--T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(7600,100,10000)) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#304042718],raw) + |------E~StitchRvsExec() on InProcessPlanDispatcher + |-------E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=11, chunkMethod=TimeRangeChunkScan(6400000,7599000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=27, chunkMethod=TimeRangeChunkScan(6400000,7599000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |-------E~PromQlRemoteExec(PromQlQueryParams(bar{job="app"}[300s],6999,1,6999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,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=10000) on InProcessPlanDispatcher + |-E~BinaryJoinExec(binaryOp=ADD, on=None, ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |--T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(2000,100,10000)) + |---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=7600000, step=100000, end=10000000, window=Some(600000), functionId=Some(Rate), rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=3, chunkMethod=TimeRangeChunkScan(7000000,10000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#304042718],raw) + |-----T~PeriodicSamplesMapper(start=2000000, step=100000, end=10000000, window=Some(600000), functionId=Some(Rate), rawSource=true, offsetMs=None) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=3, chunkMethod=TimeRangeChunkScan(1400000,10000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,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=7600000, step=100000, end=10000000, window=Some(600000), functionId=Some(Rate), rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=19, chunkMethod=TimeRangeChunkScan(7000000,10000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#304042718],raw) - |--T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(7600,100,10000)) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#304042718],raw) + |-----T~PeriodicSamplesMapper(start=2000000, step=100000, end=10000000, window=Some(600000), functionId=Some(Rate), rawSource=true, offsetMs=None) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=19, chunkMethod=TimeRangeChunkScan(1400000,10000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |--T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(2000,100,10000)) + |---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=7600000, step=100000, end=10000000, window=Some(300000), functionId=Some(Rate), rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=11, chunkMethod=TimeRangeChunkScan(7300000,10000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#304042718],raw) + |-----T~PeriodicSamplesMapper(start=2000000, step=100000, end=10000000, window=Some(300000), functionId=Some(Rate), rawSource=true, offsetMs=None) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=11, chunkMethod=TimeRangeChunkScan(1700000,10000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(bar))), 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=7600000, step=100000, end=10000000, window=Some(300000), functionId=Some(Rate), rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=27, chunkMethod=TimeRangeChunkScan(7300000,10000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#304042718],raw)""".stripMargin + |-----T~PeriodicSamplesMapper(start=2000000, step=100000, end=10000000, window=Some(300000), functionId=Some(Rate), rawSource=true, offsetMs=None) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=27, chunkMethod=TimeRangeChunkScan(1700000,10000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw)""".stripMargin val query2 = "sum(rate(test{job = \"app\"}[10m])) + sum(rate(bar{job = \"app\"}[5m]))" val lp2 = Parser.queryRangeToLogicalPlan(query2, TimeStepParams(2000, stepSecs, 10000)) From bc1d4dc10cec481de10a69dda04ef4150f4a8723 Mon Sep 17 00:00:00 2001 From: Alex Theimer Date: Tue, 9 Jan 2024 09:58:38 -0800 Subject: [PATCH 15/27] fix some PHS tests --- .../queryplanner/PlannerHierarchySpec.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala index bbd2013b8a..53ff837a92 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala @@ -1008,8 +1008,8 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS plannerParams = PlannerParams(processMultiPartition = true))) val expected = - """E~BinaryJoinExec(binaryOp=ADD, on=None, ignoring=List()) on InProcessPlanDispatcher(filodb.core.query.QueryConfig@20c812c8) - |-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@20c812c8) + """E~BinaryJoinExec(binaryOp=ADD, on=None, ignoring=List()) on InProcessPlanDispatcher + |-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 |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634777330)) |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testActor],recordingRules) |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) @@ -1036,9 +1036,9 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS """T~AggregatePresenter(aggrOp=Count, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634777330)) |-E~LocalPartitionReduceAggregateExec(aggrOp=Count, aggrParams=List()) on InProcessPlanDispatcher |--T~AggregateMapReduce(aggrOp=Count, aggrParams=List(), without=List(), by=List()) - |---E~BinaryJoinExec(binaryOp=ADD, on=None, ignoring=List()) on InProcessPlanDispatcher(filodb.core.query.QueryConfig@159424e2) - |----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,false,true,false,false), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(filodb.core.query.QueryConfig@159424e2) - |----E~StitchRvsExec() on InProcessPlanDispatcher(filodb.core.query.EmptyQueryConfig$@1f736d00) + |---E~BinaryJoinExec(binaryOp=ADD, on=None, ignoring=List()) on InProcessPlanDispatcher + |----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,false,true,false,false,true), queryEndpoint=remotePartition-url, requestTimeoutMs=10000) on InProcessPlanDispatcher + |----E~StitchRvsExec() on InProcessPlanDispatcher |-----T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1634173130,300,1634777330)) |------E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) |-------T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) From 2054a7ff87eefd6acfcf1100f4ec198de3368d2e Mon Sep 17 00:00:00 2001 From: Alex Theimer Date: Tue, 9 Jan 2024 10:08:41 -0800 Subject: [PATCH 16/27] DROP: printPairs --- .../queryplanner/PlannerHierarchySpec.scala | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala index 53ff837a92..ab9210b22a 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala @@ -18,6 +18,9 @@ import filodb.prometheus.parse.Parser.Antlr import filodb.query.{BadQueryException, IntervalSelector, LabelCardinality, PlanValidationSpec, RawSeries} import filodb.query.exec._ + +import scala.collection.mutable + // scalastyle:off line.size.limit // scalastyle:off number.of.methods class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationSpec { @@ -590,6 +593,18 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS validatePlan(execPlan, expected) } + class QueryExpectedPairs { + val pairs = new mutable.ArrayBuffer[(String, String)] + def addPair(query: String, plan: ExecPlan): Unit = { + pairs.append((query, plan.printTree().replaceAll("\n", "\n |"))) + } + def printPairs(): Unit = { + for ((query, expected) <- pairs) { + println(s"""(\"\"\"$query\"\"\",\n \"\"\"$expected\"\"\".stripMargin),""") + } + } + } + it("should not pushdown root scalar operation into a RemoteExec's promql when query spans multiple partitions") { val queryExpectedPairs = Seq( From bbb404656cde7481922ed7a52f8db9f0d367066c Mon Sep 17 00:00:00 2001 From: Alex Theimer Date: Tue, 9 Jan 2024 10:24:14 -0800 Subject: [PATCH 17/27] fix tests --- .../queryplanner/PlannerHierarchySpec.scala | 297 +++++++++--------- 1 file changed, 148 insertions(+), 149 deletions(-) diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala index ab9210b22a..7d0ff1c3c9 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala @@ -609,253 +609,252 @@ 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( ("""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.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),false,1800000 milliseconds,true))""".stripMargin), ("""foo{_ws_ = "demo", _ns_ =~ ".*Ns", instance = "Inst-1" } + 5""", """T~ScalarOperationMapper(operator=ADD, scalarOnLhs=false) |-FA1~StaticFuncArgs(5.0,RangeParams(1633913330,300,1634777330)) - |-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~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#-585347165],raw) + |-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),false,1800000 milliseconds,true)) + |--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),false,3 days,true)) + |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-797279913],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#-585347165],raw) + |-----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#-797279913],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#-585347165],raw) - |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-585347165],downsample) + |-----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#-797279913],raw) + |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-797279913],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#-585347165],downsample) + |-----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#-797279913],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#-585347165],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), + |-----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#-797279913],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),false,1800000 milliseconds,true))""".stripMargin), ("""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)) - |-E~BinaryJoinExec(binaryOp=ADD, 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)) - |---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#1132741402],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#1132741402],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#1132741402],raw) - |----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1132741402],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#1132741402],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#1132741402],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)) - |--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#1132741402],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#1132741402],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#1132741402],raw) - |----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1132741402],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#1132741402],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#1132741402],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), - // Aggregate with namespace regex selector. + """T~ScalarOperationMapper(operator=MUL, scalarOnLhs=true) + |-FA1~StaticFuncArgs(5.0,RangeParams(1633913330,300,1634777330)) + |-E~BinaryJoinExec(binaryOp=ADD, on=None, 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),false,1800000 milliseconds,true)) + |--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),false,1800000 milliseconds,true)) + |---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),false,3 days,true)) + |----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-797279913],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#-797279913],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#-797279913],raw) + |----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-797279913],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#-797279913],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#-797279913],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),false,1800000 milliseconds,true)) + |--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),false,1800000 milliseconds,true)) + |---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),false,3 days,true)) + |----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-797279913],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#-797279913],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#-797279913],raw) + |----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-797279913],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#-797279913],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#-797279913],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),false,1800000 milliseconds,true))""".stripMargin), ("""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#-585347165],raw) + |--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),false,1800000 milliseconds,true)) + |---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),false,3 days,true)) + |----E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-797279913],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#-585347165],raw) + |-------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#-797279913],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#-585347165],raw) - |----E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-585347165],downsample) + |-------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#-797279913],raw) + |----E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-797279913],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#-585347165],downsample) + |-------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#-797279913],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#-585347165],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), + |-------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#-797279913],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),false,1800000 milliseconds,true))""".stripMargin), ("""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~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~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#-585347165],raw) + |--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),false,1800000 milliseconds,true)) + |---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),false,3 days,true)) + |----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-797279913],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#-585347165],raw) + |------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#-797279913],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#-585347165],raw) - |----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-585347165],downsample) + |------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#-797279913],raw) + |----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-797279913],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#-585347165],downsample) + |------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#-797279913],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#-585347165],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), + |------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#-797279913],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),false,1800000 milliseconds,true))""".stripMargin), ("""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#-585347165],raw) + |---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),false,1800000 milliseconds,true)) + |----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),false,3 days,true)) + |-----E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-797279913],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#-585347165],raw) + |--------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#-797279913],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#-585347165],raw) - |-----E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-585347165],downsample) + |--------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#-797279913],raw) + |-----E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-797279913],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#-585347165],downsample) + |--------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#-797279913],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#-585347165],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~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~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#-585347165],raw) + |--------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#-797279913],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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |--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),false,3 days,true)) + |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-797279913],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#-585347165],raw) + |-----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#-797279913],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#-585347165],raw) - |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-585347165],downsample) + |-----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#-797279913],raw) + |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-797279913],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#-585347165],downsample) + |-----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#-797279913],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#-585347165],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), + |-----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#-797279913],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),false,1800000 milliseconds,true))""".stripMargin), ("""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)) - |--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#1893185297],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#1893185297],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#1893185297],raw) - |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1893185297],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#1893185297],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#1893185297],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)) - |-T~VectorFunctionMapper(funcParams=List()) - |--E~ScalarFixedDoubleExec(params = RangeParams(1633913330,300,1634777330), value = 5.0) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), - // join multiple adjacent scalars - // NOTE: this can be optimized by pushing down the scalar operation. + // join multiple adjacent scalars + // NOTE: this can be optimized by pushing down the scalar operation. + """E~BinaryJoinExec(binaryOp=GTR, on=None, 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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |--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),false,3 days,true)) + |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-797279913],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#-797279913],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#-797279913],raw) + |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-797279913],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#-797279913],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#-797279913],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),false,1800000 milliseconds,true)) + |-T~VectorFunctionMapper(funcParams=List()) + |--E~ScalarFixedDoubleExec(params = RangeParams(1633913330,300,1634777330), value = 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),false,1800000 milliseconds,true))""".stripMargin), ("""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~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~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#-585347165],raw) + |-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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |--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),false,3 days,true)) + |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-797279913],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#-585347165],raw) + |-----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#-797279913],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#-585347165],raw) - |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-585347165],downsample) + |-----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#-797279913],raw) + |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-797279913],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#-585347165],downsample) + |-----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#-797279913],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#-585347165],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), + |-----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#-797279913],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),false,1800000 milliseconds,true))""".stripMargin), ("""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~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~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#-585347165],raw) + |--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),false,1800000 milliseconds,true)) + |---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),false,3 days,true)) + |----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-797279913],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#-585347165],raw) + |------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#-797279913],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#-585347165],raw) - |----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-585347165],downsample) + |------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#-797279913],raw) + |----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-797279913],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#-585347165],downsample) + |------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#-797279913],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#-585347165],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), + |------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#-797279913],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),false,1800000 milliseconds,true))""".stripMargin), ("""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#-585347165],raw) + |--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),false,1800000 milliseconds,true)) + |---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),false,3 days,true)) + |----E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-797279913],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#-585347165],raw) + |-------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#-797279913],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#-585347165],raw) - |----E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-585347165],downsample) + |-------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#-797279913],raw) + |----E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-797279913],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#-585347165],downsample) + |-------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#-797279913],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#-585347165],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), + |-------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#-797279913],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),false,1800000 milliseconds,true))""".stripMargin), ("""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~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~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#-585347165],raw) + |--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),false,1800000 milliseconds,true)) + |---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),false,3 days,true)) + |----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-797279913],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#-585347165],raw) + |------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#-797279913],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#-585347165],raw) - |----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-585347165],downsample) + |------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#-797279913],raw) + |----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-797279913],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#-585347165],downsample) + |------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#-797279913],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#-585347165],downsample) - |---E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~".*Ns",instance="Inst-1"},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,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~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#-797279913],downsample) + |---E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~".*Ns",instance="Inst-1"},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,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),false,1800000 milliseconds,true))""".stripMargin), ("""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~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~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#-585347165],raw) + |-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),false,1800000 milliseconds,true)) + |--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),false,3 days,true)) + |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-797279913],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#-585347165],raw) + |-----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#-797279913],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#-585347165],raw) - |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-585347165],downsample) + |-----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#-797279913],raw) + |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-797279913],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#-585347165],downsample) + |-----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#-797279913],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#-585347165],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), + |-----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#-797279913],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),false,1800000 milliseconds,true))""".stripMargin), ("""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~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~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#-585347165],raw) + |--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),false,1800000 milliseconds,true)) + |---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),false,3 days,true)) + |----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-797279913],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#-585347165],raw) + |------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#-797279913],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#-585347165],raw) - |----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-585347165],downsample) + |------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#-797279913],raw) + |----E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-797279913],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#-585347165],downsample) + |------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#-797279913],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#-585347165],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) + |------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#-797279913],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),false,1800000 milliseconds,true))""".stripMargin) ) for ((query, rootPlan) <- queryExpectedPairs) { val queryParams = PromQlQueryParams(query, startSeconds, step, endSeconds) From a7d336b44962791f83354293a65fe031722b0ccf Mon Sep 17 00:00:00 2001 From: Alex Theimer Date: Tue, 9 Jan 2024 10:33:51 -0800 Subject: [PATCH 18/27] fix more tests --- .../queryplanner/PlannerHierarchySpec.scala | 98 +++++++++---------- 1 file changed, 49 insertions(+), 49 deletions(-) diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala index 7d0ff1c3c9..f62f745425 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala @@ -910,97 +910,97 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS ("""foo{_ws_ = "demo", _ns_ =~ "remoteNs.*", instance = "Inst-1" } + 5""", """T~ScalarOperationMapper(operator=ADD, scalarOnLhs=false) |-FA1~StaticFuncArgs(5.0,RangeParams(1633913330,300,1634777330)) - |-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(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), + |-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),false,1800000 milliseconds,true)) + |--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),false,1800000 milliseconds,true)) + |--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),false,1800000 milliseconds,true))""".stripMargin), ("""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)) - |-E~BinaryJoinExec(binaryOp=ADD, 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)) - |---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)) - |--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), + """T~ScalarOperationMapper(operator=MUL, scalarOnLhs=true) + |-FA1~StaticFuncArgs(5.0,RangeParams(1633913330,300,1634777330)) + |-E~BinaryJoinExec(binaryOp=ADD, on=None, 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),false,1800000 milliseconds,true)) + |--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),false,1800000 milliseconds,true)) + |---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),false,1800000 milliseconds,true)) + |---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),false,1800000 milliseconds,true)) + |--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),false,1800000 milliseconds,true)) + |---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),false,1800000 milliseconds,true)) + |---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),false,1800000 milliseconds,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), + |--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),false,1800000 milliseconds,true)) + |---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),false,1800000 milliseconds,true)) + |---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),false,1800000 milliseconds,true))""".stripMargin), ("""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~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(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), + |--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),false,1800000 milliseconds,true)) + |---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),false,1800000 milliseconds,true)) + |---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),false,1800000 milliseconds,true))""".stripMargin), ("""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~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(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), + |---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),false,1800000 milliseconds,true)) + |----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),false,1800000 milliseconds,true)) + |----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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |--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),false,1800000 milliseconds,true)) + |--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),false,1800000 milliseconds,true))""".stripMargin), ("""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)) - |--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)) - |-T~VectorFunctionMapper(funcParams=List()) - |--E~ScalarFixedDoubleExec(params = RangeParams(1633913330,300,1634777330), value = 5.0) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,Some(10000),None,true,false,true))""".stripMargin), + """E~BinaryJoinExec(binaryOp=GTR, on=None, 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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |--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),false,1800000 milliseconds,true)) + |--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),false,1800000 milliseconds,true)) + |-T~VectorFunctionMapper(funcParams=List()) + |--E~ScalarFixedDoubleExec(params = RangeParams(1633913330,300,1634777330), value = 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),false,1800000 milliseconds,true))""".stripMargin), // 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~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(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), + |-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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |--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),false,1800000 milliseconds,true)) + |--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),false,1800000 milliseconds,true))""".stripMargin), ("""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~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(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), + |--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),false,1800000 milliseconds,true)) + |---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),false,1800000 milliseconds,true)) + |---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),false,1800000 milliseconds,true))""".stripMargin), ("""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), + |--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),false,1800000 milliseconds,true)) + |---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),false,1800000 milliseconds,true)) + |---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),false,1800000 milliseconds,true))""".stripMargin), ("""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~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(foo{_ws_="demo",_ns_=~"remoteNs.*",instance="Inst-1"},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,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"},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,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), + |--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),false,1800000 milliseconds,true)) + |---E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"remoteNs.*",instance="Inst-1"},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,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),false,1800000 milliseconds,true)) + |---E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"remoteNs.*",instance="Inst-1"},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,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),false,1800000 milliseconds,true))""".stripMargin), ("""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~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{_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), + |-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),false,1800000 milliseconds,true)) + |--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),false,1800000 milliseconds,true)) + |--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),false,1800000 milliseconds,true))""".stripMargin), ("""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~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(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) + |--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),false,1800000 milliseconds,true)) + |---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),false,1800000 milliseconds,true)) + |---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),false,1800000 milliseconds,true))""".stripMargin) ) for ((query, otherPlan) <- queryExpectedPairs) { val queryParams = PromQlQueryParams(query, startSeconds, step, endSeconds) From 23c03481590ee8166f5f1a554b389b67d1818dbe Mon Sep 17 00:00:00 2001 From: Alex Theimer Date: Tue, 9 Jan 2024 10:53:43 -0800 Subject: [PATCH 19/27] fix yet more tests --- .../queryplanner/PlannerHierarchySpec.scala | 168 +++++++++--------- 1 file changed, 84 insertions(+), 84 deletions(-) diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala index f62f745425..1fc8e2d648 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala @@ -2209,171 +2209,171 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS Test(query = """sum(test{job="app"} offset 10m)""", lookbackSec = 300, 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),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), + 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),false,1800000 milliseconds,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(sum(test{job="app"} offset 600s),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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".stripMargin), Test(query = """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)),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), + 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),false,1800000 milliseconds,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(count(rate(test{job="app"}[1200s] offset 600s)),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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".stripMargin), Test(query = """group(rate(test{job="app"}[20m:30s] offset 10m))""", lookbackSec = 1500, 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)),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), + 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),false,1800000 milliseconds,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(group(rate(test{job="app"}[1200s:30s] offset 600s)),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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".stripMargin), Test(query = """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]))),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,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".stripMargin), Test(query = """sgn(test{job="app"} offset 10m)""", lookbackSec = 300, 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),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), + 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),false,1800000 milliseconds,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(sgn(test{job="app"} offset 600s),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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".stripMargin), Test(query = """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)),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), + 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),false,1800000 milliseconds,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(ln(rate(test{job="app"}[1200s] offset 600s)),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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".stripMargin), Test(query = """exp(rate(test{job="app"}[20m:30s] offset 10m))""", lookbackSec = 1500, 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)),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), + 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),false,1800000 milliseconds,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(exp(rate(test{job="app"}[1200s:30s] offset 600s)),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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".stripMargin), Test(query = """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]))),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,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".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"}),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), + 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),false,1800000 milliseconds,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,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),false,1800000 milliseconds,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,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),false,1800000 milliseconds,true))""".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"})),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), + 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),false,1800000 milliseconds,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,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),false,1800000 milliseconds,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,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),false,1800000 milliseconds,true))""".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"})),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), + 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),false,1800000 milliseconds,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,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),false,1800000 milliseconds,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,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),false,1800000 milliseconds,true))""".stripMargin), Test(query = """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])),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), + 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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".stripMargin), Test(query = """count_over_time(foo{job="app1"}[5m]) unless rate(bar{job="app1"}[15m:30s])""", 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])),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,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".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]))),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,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".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]))),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), + 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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".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),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), + 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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".stripMargin), Test(query = """123 + sgn(test{job="app"} offset 10m)""", lookbackSec = 300, 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)),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), + 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),false,1800000 milliseconds,true)) + |-E~PromQlRemoteExec(PromQlQueryParams((123.0 + sgn(test{job="app"} offset 600s)),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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".stripMargin), Test(query = """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))),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), + 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),false,1800000 milliseconds,true)) + |-E~PromQlRemoteExec(PromQlQueryParams((123.0 + sum(rate(test{job="app"}[1200s] offset 600s))),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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".stripMargin), Test(query = """123 + group(rate(test{job="app"}[20m:30s] offset 10m))""", lookbackSec = 1500, 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))),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), + 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),false,1800000 milliseconds,true)) + |-E~PromQlRemoteExec(PromQlQueryParams((123.0 + group(rate(test{job="app"}[1200s:30s] offset 600s))),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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".stripMargin), Test(query = """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])))),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,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".stripMargin), Test(query = """absent(test{job="app"} offset 10m)""", lookbackSec = 300, 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),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), + 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),false,1800000 milliseconds,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(absent(test{job="app"} offset 600s),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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".stripMargin), Test(query = """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)),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), + 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),false,1800000 milliseconds,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(absent(rate(test{job="app"}[1200s] offset 600s)),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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".stripMargin), Test(query = """absent(count_over_time(test{job="app"}[20m:30s] offset 10m))""", lookbackSec = 1500, 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)),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), + 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),false,1800000 milliseconds,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(absent(count_over_time(test{job="app"}[1200s:30s] offset 600s)),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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".stripMargin), Test(query = """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]))),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,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".stripMargin), Test(query = """scalar(test{job="app"} offset 10m)""", lookbackSec = 300, 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),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), + 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),false,1800000 milliseconds,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(scalar(test{job="app"} offset 600s),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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".stripMargin), Test(query = """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)),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), + 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),false,1800000 milliseconds,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(scalar(rate(test{job="app"}[1200s] offset 600s)),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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".stripMargin), Test(query = """scalar(count_over_time(test{job="app"}[20m:30s] offset 10m))""", lookbackSec = 1500, 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)),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), + 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),false,1800000 milliseconds,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(scalar(count_over_time(test{job="app"}[1200s:30s] offset 600s)),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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".stripMargin), Test(query = """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]))),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,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".stripMargin), ) val partitionLocationProvider = new PartitionLocationProvider { override def getPartitions(routingKey: Map[String, String], From a5b2b20b6844b2eac5d80df323c2c0f0a150c650 Mon Sep 17 00:00:00 2001 From: Alex Theimer Date: Tue, 9 Jan 2024 11:48:22 -0800 Subject: [PATCH 20/27] oh so many test fixes --- .../queryplanner/PlannerHierarchySpec.scala | 183 ++++++++---------- 1 file changed, 84 insertions(+), 99 deletions(-) diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala index 1fc8e2d648..29ea888ce0 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala @@ -2431,171 +2431,171 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS Test(query = """sum(test{job="app"} offset 10m)""", lookbackSec = 300, 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),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), + 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),false,1800000 milliseconds,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(sum(test{job="app"} offset 600s),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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".stripMargin), Test(query = """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)),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), + 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),false,1800000 milliseconds,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(count(rate(test{job="app"}[1200s] offset 600s)),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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".stripMargin), Test(query = """group(rate(test{job="app"}[20m:30s] offset 10m))""", lookbackSec = 1500, 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)),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), + 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),false,1800000 milliseconds,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(group(rate(test{job="app"}[1200s:30s] offset 600s)),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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".stripMargin), Test(query = """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]))),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,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".stripMargin), Test(query = """sgn(test{job="app"} offset 10m)""", lookbackSec = 300, 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),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), + 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),false,1800000 milliseconds,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(sgn(test{job="app"} offset 600s),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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".stripMargin), Test(query = """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)),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), + 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),false,1800000 milliseconds,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(ln(rate(test{job="app"}[1200s] offset 600s)),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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".stripMargin), Test(query = """exp(rate(test{job="app"}[20m:30s] offset 10m))""", lookbackSec = 1500, 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)),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), + 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),false,1800000 milliseconds,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(exp(rate(test{job="app"}[1200s:30s] offset 600s)),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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".stripMargin), Test(query = """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]))),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,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".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"}),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), + 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),false,1800000 milliseconds,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,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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".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"})),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), + 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),false,1800000 milliseconds,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,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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".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"})),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), + 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),false,1800000 milliseconds,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,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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".stripMargin), Test(query = """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])),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), + 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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".stripMargin), Test(query = """count_over_time(foo{job="app1"}[5m]) unless rate(bar{job="app1"}[15m:30s])""", 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])),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,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".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]))),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,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".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]))),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), + 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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".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),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), + 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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".stripMargin), Test(query = """123 + sgn(test{job="app"} offset 10m)""", lookbackSec = 300, 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)),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), + 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),false,1800000 milliseconds,true)) + |-E~PromQlRemoteExec(PromQlQueryParams((123.0 + sgn(test{job="app"} offset 600s)),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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".stripMargin), Test(query = """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))),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), + 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),false,1800000 milliseconds,true)) + |-E~PromQlRemoteExec(PromQlQueryParams((123.0 + sum(rate(test{job="app"}[1200s] offset 600s))),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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".stripMargin), Test(query = """123 + group(rate(test{job="app"}[20m:30s] offset 10m))""", lookbackSec = 1500, 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))),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), + 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),false,1800000 milliseconds,true)) + |-E~PromQlRemoteExec(PromQlQueryParams((123.0 + group(rate(test{job="app"}[1200s:30s] offset 600s))),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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".stripMargin), Test(query = """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])))),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,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".stripMargin), Test(query = """absent(test{job="app"} offset 10m)""", lookbackSec = 300, 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),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), + 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),false,1800000 milliseconds,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(absent(test{job="app"} offset 600s),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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".stripMargin), Test(query = """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)),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), + 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),false,1800000 milliseconds,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(absent(rate(test{job="app"}[1200s] offset 600s)),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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".stripMargin), Test(query = """absent(count_over_time(test{job="app"}[20m:30s] offset 10m))""", lookbackSec = 1500, 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)),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), + 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),false,1800000 milliseconds,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(absent(count_over_time(test{job="app"}[1200s:30s] offset 600s)),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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".stripMargin), Test(query = """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]))),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,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".stripMargin), Test(query = """scalar(test{job="app"} offset 10m)""", lookbackSec = 300, 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),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), + 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),false,1800000 milliseconds,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(scalar(test{job="app"} offset 600s),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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".stripMargin), Test(query = """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)),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), + 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),false,1800000 milliseconds,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(scalar(rate(test{job="app"}[1200s] offset 600s)),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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".stripMargin), Test(query = """scalar(count_over_time(test{job="app"}[20m:30s] offset 10m))""", lookbackSec = 1500, 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)),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), + 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),false,1800000 milliseconds,true)) + |-E~PromQlRemoteExec(PromQlQueryParams(scalar(count_over_time(test{job="app"}[1200s:30s] offset 600s)),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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".stripMargin), Test(query = """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]))),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,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true))""".stripMargin) ) val partitionLocationProvider = new PartitionLocationProvider { override def getPartitions(routingKey: Map[String, String], @@ -2624,21 +2624,6 @@ 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") - } - (url, TimeStepParams(params.startSecs, params.stepSecs, params.endSecs)) - }.toSet shouldEqual expectedQueryParams // sanity check validatePlan(root, test.expected) } From dffd5667b6413b54c3c7ed01ca7c002e9c58e5f0 Mon Sep 17 00:00:00 2001 From: Alex Theimer Date: Tue, 9 Jan 2024 13:35:09 -0800 Subject: [PATCH 21/27] fix test --- .../queryplanner/PlannerHierarchySpec.scala | 74 +++++++++---------- 1 file changed, 37 insertions(+), 37 deletions(-) diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala index 29ea888ce0..c6c84f257a 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala @@ -3775,66 +3775,66 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS // 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~StitchRvsExec() on InProcessPlanDispatcher - |-E~StitchRvsExec() on InProcessPlanDispatcher - |--E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + """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),false,1800000 milliseconds,true)) + |-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),false,3 days,true)) + |--E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1881185126],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/testActor],raw) + |----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#-1881185126],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/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(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#-1881185126],raw) + |--E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1881185126],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/testActor],downsample) + |----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#-1881185126],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/testActor],downsample) - |-E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="dummy",_ns_=~"dummy.*",bar="hello"},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=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), + |----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#-1881185126],downsample) + |-E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="dummy",_ns_=~"dummy.*",bar="hello"},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=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),false,1800000 milliseconds,true))""".stripMargin), ("""sum(foo{_ws_="dummy", _ns_=~"dummy.*", bar="hello"})""", """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(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),false,1800000 milliseconds,true)) + |--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),false,3 days,true)) + |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1881185126],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/testActor],raw) + |------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#-1881185126],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/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(_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#-1881185126],raw) + |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1881185126],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/testActor],downsample) + |------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#-1881185126],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/testActor],downsample) - |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="dummy",_ns_=~"dummy.*",bar="hello"}),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=dummy-endpoint, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin), + |------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#-1881185126],downsample) + |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="dummy",_ns_=~"dummy.*",bar="hello"}),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=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),false,1800000 milliseconds,true))""".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 - |-E~StitchRvsExec() on InProcessPlanDispatcher - |--E~StitchRvsExec() on InProcessPlanDispatcher - |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + """E~BinaryJoinExec(binaryOp=ADD, on=None, 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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |--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),false,3 days,true)) + |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1881185126],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/testActor],raw) + |-----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#-1881185126],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/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(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#-1881185126],raw) + |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1881185126],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/testActor],downsample) + |-----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#-1881185126],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/testActor],downsample) - |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="dummy",_ns_=~"dummy.*",bar="hello"},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=dummy-endpoint, requestTimeoutMs=10000) on InProcessPlanDispatcher - |-E~StitchRvsExec() on InProcessPlanDispatcher - |--E~StitchRvsExec() on InProcessPlanDispatcher - |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |-----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#-1881185126],downsample) + |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="dummy",_ns_=~"dummy.*",bar="hello"},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=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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |--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),false,3 days,true)) + |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1881185126],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/testActor],raw) + |-----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#-1881185126],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/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(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#-1881185126],raw) + |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1881185126],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/testActor],downsample) + |-----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#-1881185126],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/testActor],downsample) - |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="dummy",_ns_=~"dummy.*",bar="goodbye"},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=dummy-endpoint, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin) + |-----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#-1881185126],downsample) + |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="dummy",_ns_=~"dummy.*",bar="goodbye"},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=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),false,1800000 milliseconds,true))""".stripMargin) ) for ((query, expected) <- queryExpectedPairs) { val lp = Parser.queryRangeToLogicalPlan(query, timeParams) From 5797171b9c7ea8311127f5b4364bedeb9e943229 Mon Sep 17 00:00:00 2001 From: Alex Theimer Date: Tue, 9 Jan 2024 13:37:10 -0800 Subject: [PATCH 22/27] test fix --- .../queryplanner/PlannerHierarchySpec.scala | 90 +++++++++---------- 1 file changed, 45 insertions(+), 45 deletions(-) diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala index c6c84f257a..6bc154ad55 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala @@ -3882,84 +3882,84 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS // Leaves should be materialized for at most two shards per selector. val queryExpectedPairs = Seq( ("""foo{_ws_="dummy", _ns_=~"dummy.*", bar="hello"}""", - """E~StitchRvsExec() on InProcessPlanDispatcher - |-E~StitchRvsExec() on InProcessPlanDispatcher - |--E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + """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),false,1800000 milliseconds,true)) + |-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),false,3 days,true)) + |--E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-535983087],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/testActor],raw) + |----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#-535983087],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/testActor],raw) + |----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#-535983087],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/testActor],raw) - |--E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) + |----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#-535983087],raw) + |--E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-535983087],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/testActor],downsample) + |----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#-535983087],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/testActor],downsample) + |----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#-535983087],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/testActor],downsample) - |-E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="dummy",_ns_=~"dummy.*",bar="hello"},1633913330,300,1634777330,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""".stripMargin), + |----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#-535983087],downsample) + |-E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="dummy",_ns_=~"dummy.*",bar="hello"},1633913330,300,1634777330,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),false,1800000 milliseconds,true))""".stripMargin), ("""sum(foo{_ws_="dummy", _ns_=~"dummy.*", bar="hello"})""", """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(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),false,1800000 milliseconds,true)) + |--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),false,3 days,true)) + |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-535983087],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/testActor],raw) + |------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#-535983087],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/testActor],raw) + |------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#-535983087],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/testActor],raw) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) + |------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#-535983087],raw) + |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-535983087],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/testActor],downsample) + |------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#-535983087],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/testActor],downsample) + |------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#-535983087],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/testActor],downsample) - |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="dummy",_ns_=~"dummy.*",bar="hello"}),1633913330,300,1634777330,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""".stripMargin), + |------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#-535983087],downsample) + |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="dummy",_ns_=~"dummy.*",bar="hello"}),1633913330,300,1634777330,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),false,1800000 milliseconds,true))""".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 - |-E~StitchRvsExec() on InProcessPlanDispatcher - |--E~StitchRvsExec() on InProcessPlanDispatcher - |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + """E~BinaryJoinExec(binaryOp=ADD, on=None, 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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |--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),false,3 days,true)) + |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-535983087],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/testActor],raw) + |-----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#-535983087],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/testActor],raw) + |-----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#-535983087],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/testActor],raw) - |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) + |-----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#-535983087],raw) + |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-535983087],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/testActor],downsample) + |-----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#-535983087],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/testActor],downsample) + |-----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#-535983087],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/testActor],downsample) - |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="dummy",_ns_=~"dummy.*",bar="hello"},1633913330,300,1634777330,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 - |-E~StitchRvsExec() on InProcessPlanDispatcher - |--E~StitchRvsExec() on InProcessPlanDispatcher - |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |-----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#-535983087],downsample) + |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="dummy",_ns_=~"dummy.*",bar="hello"},1633913330,300,1634777330,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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |--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),false,3 days,true)) + |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-535983087],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/testActor],raw) + |-----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#-535983087],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/testActor],raw) + |-----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#-535983087],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/testActor],raw) - |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) + |-----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#-535983087],raw) + |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-535983087],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/testActor],downsample) + |-----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#-535983087],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/testActor],downsample) + |-----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#-535983087],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/testActor],downsample) - |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="dummy",_ns_=~"dummy.*",bar="goodbye"},1633913330,300,1634777330,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""".stripMargin) + |-----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#-535983087],downsample) + |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="dummy",_ns_=~"dummy.*",bar="goodbye"},1633913330,300,1634777330,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),false,1800000 milliseconds,true))""".stripMargin) ) for ((query, expected) <- queryExpectedPairs) { val lp = Parser.queryRangeToLogicalPlan(query, timeParams) From 42287eb5c3271fb110274686729b0e8212643290 Mon Sep 17 00:00:00 2001 From: Alex Theimer Date: Tue, 9 Jan 2024 14:24:25 -0800 Subject: [PATCH 23/27] fix tests -- need comments --- .../queryplanner/PlannerHierarchySpec.scala | 1287 ++++++++--------- 1 file changed, 600 insertions(+), 687 deletions(-) diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala index 6bc154ad55..6c5c7ff4e0 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala @@ -2974,693 +2974,606 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS case class Test(query: String, expected: String, tschemaEnabled: Set[String] = Set()) val tests = Seq( - // no namespaces with target-schemas enabled - Test(query=s"""sum(foo{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bar"}) by ($TSCHEMA_LABEL)""", - tschemaEnabled=Set(), - expected="""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,true,false,true,Set(),Some(plannerSelector))) - |--E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,false,false,true,Set(),None)) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#37899380],raw) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#37899380],raw) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#37899380],raw) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#37899380],downsample) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#37899380],downsample) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#37899380],downsample) - |--E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,false,false,true,Set(),None)) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#37899380],raw) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#37899380],raw) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#37899380],raw) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#37899380],downsample) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#37899380],downsample) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#37899380],downsample)""".stripMargin), - // one namespace has a target-schema assigned - Test(query=s"""sum(foo{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bar"}) by ($TSCHEMA_LABEL)""", - tschemaEnabled=Set("local1"), - expected="""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,true,false,true,Set(),Some(plannerSelector))) - |--E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,false,false,true,Set(),None)) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1028628326],raw) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1028628326],raw) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1028628326],raw) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1028628326],downsample) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1028628326],downsample) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1028628326],downsample) - |--E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,false,false,true,Set(),None)) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1028628326],raw) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1028628326],raw) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1028628326],downsample) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1028628326],downsample)""".stripMargin), - // both namespaces are assigned target-schemas; should concat results - Test(query=s"""sum(foo{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bar"}) by ($TSCHEMA_LABEL)""", - tschemaEnabled=Set("local1", "local2"), - expected="""E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |-E~StitchRvsExec() on InProcessPlanDispatcher - |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-E~StitchRvsExec() on InProcessPlanDispatcher - |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw)""".stripMargin), - // remove "by" clause; should not concatenate - Test(query=s"""sum(foo{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bar"})""", - tschemaEnabled=Set("local1", "local2"), - expected="""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,true,false,true,Set(),Some(plannerSelector))) - |--E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,false,false,true,Set(),None)) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#888263698],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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#888263698],raw) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#888263698],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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#888263698],downsample) - |--E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,false,false,true,Set(),None)) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#888263698],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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#888263698],raw) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#888263698],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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#888263698],downsample)""".stripMargin), - // "by" clause is missing a target-schema label - Test(query=s"""sum(foo{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bar"}) by (notATargetSchemaLabel)""", - tschemaEnabled=Set("local1", "local2"), - expected="""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,true,false,true,Set(),Some(plannerSelector))) - |--E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,false,false,true,Set(),None)) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-367138549],raw) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(notATargetSchemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-367138549],raw) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-367138549],downsample) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(notATargetSchemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-367138549],downsample) - |--E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,false,false,true,Set(),None)) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-367138549],raw) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(notATargetSchemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-367138549],raw) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-367138549],downsample) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(notATargetSchemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-367138549],downsample)""".stripMargin), - // remote partition, no target-schemas assigned - Test(query=s"""sum(foo{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"}) by ($TSCHEMA_LABEL)""", - tschemaEnabled=Set(), - expected="""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,true,false,true,Set(),Some(plannerSelector))) - |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{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(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,true,false,true,Set(),Some(plannerSelector))) - |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{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(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,true,false,true,Set(),Some(plannerSelector)))""".stripMargin), - // remote partition, one ns has a target-schema assigned - Test(query=s"""sum(foo{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"}) by ($TSCHEMA_LABEL)""", - tschemaEnabled=Set("oneRemote1"), - expected="""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,true,false,true,Set(),Some(plannerSelector))) - |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{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(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,true,false,true,Set(),Some(plannerSelector))) - |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{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(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,true,false,true,Set(),Some(plannerSelector)))""".stripMargin), - // remote partition, both ns have a target-schema assigned - Test(query=s"""sum(foo{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"}) by ($TSCHEMA_LABEL)""", - tschemaEnabled=Set("oneRemote1", "oneRemote2"), - expected="""E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |-E~PromQlRemoteExec(PromQlQueryParams(sum(foo{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(sum(foo{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), - // two remote partitions, both ns have a target-schema assigned - Test(query=s"""sum(foo{_ws_="demo",_ns_=~"$TWO_REMOTE",$TSCHEMA_LABEL="bar"}) by ($TSCHEMA_LABEL)""", - tschemaEnabled=Set("twoRemote1", "twoRemote2"), - expected="""E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |-E~PromQlRemoteExec(PromQlQueryParams(sum(foo{tschemaLabel="bar",_ws_="demo",_ns_="twoRemote1"}) 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=twoRemote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher - |-E~PromQlRemoteExec(PromQlQueryParams(sum(foo{tschemaLabel="bar",_ws_="demo",_ns_="twoRemote2"}) 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=twoRemote2-url, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin), - // remote partition, both ns have a target-schema assigned, but "by" clause is missing - Test(query=s"""sum(foo{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"})""", - tschemaEnabled=Set("oneRemote1", "oneRemote2"), - expected="""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,true,false,true,Set(),Some(plannerSelector))) - |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{tschemaLabel="bar",_ws_="demo",_ns_="oneRemote1"}),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(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,true,false,true,Set(),Some(plannerSelector))) - |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{tschemaLabel="bar",_ws_="demo",_ns_="oneRemote2"}),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(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,true,false,true,Set(),Some(plannerSelector)))""".stripMargin), - // remote/local partitions, no ns has a tschema assigned - Test(query=s"""sum(foo{_ws_="demo",_ns_=~"$BOTH",$TSCHEMA_LABEL="bar"}) by ($TSCHEMA_LABEL)""", - tschemaEnabled=Set(), - expected="""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,true,false,true,Set(),Some(plannerSelector))) - |--E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,false,false,true,Set(),None)) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-996691824],raw) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-996691824],raw) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-996691824],raw) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-996691824],downsample) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-996691824],downsample) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-996691824],downsample) - |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{tschemaLabel="bar",_ws_="demo",_ns_="remote"}) 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(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,true,false,true,Set(),Some(plannerSelector)))""".stripMargin), - // remote/local partitions, local ns has a tschema assigned - Test(query=s"""sum(foo{_ws_="demo",_ns_=~"$BOTH",$TSCHEMA_LABEL="bar"}) by ($TSCHEMA_LABEL)""", - tschemaEnabled=Set("local"), - expected="""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,true,false,true,Set(),Some(plannerSelector))) - |--E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,false,false,true,Set(),None)) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1673714297],raw) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1673714297],raw) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1673714297],downsample) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1673714297],downsample) - |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{tschemaLabel="bar",_ws_="demo",_ns_="remote"}) 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(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,true,false,true,Set(),Some(plannerSelector)))""".stripMargin), - // remote/local partitions, remote ns has a tschema assigned - Test(query=s"""sum(foo{_ws_="demo",_ns_=~"$BOTH",$TSCHEMA_LABEL="bar"}) by ($TSCHEMA_LABEL)""", - tschemaEnabled=Set("remote"), - expected="""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,true,false,true,Set(),Some(plannerSelector))) - |--E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,false,false,true,Set(),None)) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1678078999],raw) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1678078999],raw) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1678078999],raw) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1678078999],downsample) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1678078999],downsample) - |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |-----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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1678078999],downsample) - |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{tschemaLabel="bar",_ws_="demo",_ns_="remote"}) 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(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,true,false,true,Set(),Some(plannerSelector)))""".stripMargin), - // one remote partition, and all ns are assigned a tschema, but tschema label missing from selector - Test(query=s"""sum(foo{_ws_="demo",_ns_=~"$ONE_REMOTE"}) by ($TSCHEMA_LABEL)""", - tschemaEnabled=Set("oneRemote1", "oneRemote2"), - expected="""E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |-E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_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(sum(foo{_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), - // one remote partition, and all ns are assigned a tschema, but tschema label missing from selector and "by" clause - Test(query=s"""sum(foo{_ws_="demo",_ns_=~"$ONE_REMOTE"}) by (notALabel)""", - tschemaEnabled=Set("oneRemote1", "oneRemote2"), - expected="""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,true,false,true,Set(),Some(plannerSelector))) - |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_="oneRemote1"}) by (notALabel),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(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,true,false,true,Set(),Some(plannerSelector))) - |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_="oneRemote2"}) by (notALabel),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(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,true,false,true,Set(),Some(plannerSelector)))""".stripMargin), - // local partition, and all ns are assigned a tschema, but tschema label missing from selector - Test(query=s"""sum(foo{_ws_="demo",_ns_=~"$LOCAL"}) by ($TSCHEMA_LABEL)""", - tschemaEnabled=Set("local1", "local2"), - expected="""E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |-E~StitchRvsExec() on InProcessPlanDispatcher - |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |----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(local1)), 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(tschemaLabel)) - |----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(local1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |----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(local1)), 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(tschemaLabel)) - |----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(local1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-E~StitchRvsExec() on InProcessPlanDispatcher - |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |----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(local2)), 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(tschemaLabel)) - |----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(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |----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(local2)), 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(tschemaLabel)) - |----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(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw)""".stripMargin), - // one remote partition, and all ns are assigned a tschema; nested aggregation with all target-schema labels present in each "by" clause - Test(query=s"""sum(sum(foo{_ws_="demo",_ns_=~"$ONE_REMOTE"}) by ($TSCHEMA_LABEL)) by ($TSCHEMA_LABEL)""", - tschemaEnabled=Set("oneRemote1", "oneRemote2"), - expected="""E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |-E~PromQlRemoteExec(PromQlQueryParams(sum(sum(foo{_ws_="demo",_ns_="oneRemote1"}) by (tschemaLabel)) 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(sum(sum(foo{_ws_="demo",_ns_="oneRemote2"}) by (tschemaLabel)) 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), - // one remote partition, and all ns are assigned a tschema; nested aggregation with target-schema label missing from outer "by" clause - Test(query=s"""sum(sum(foo{_ws_="demo",_ns_=~"$ONE_REMOTE"}) by ($TSCHEMA_LABEL))""", - tschemaEnabled=Set("oneRemote1", "oneRemote2"), - expected="""T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634777330)) - |-E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on InProcessPlanDispatcher - |--T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |---E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |----E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_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(sum(foo{_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), - // one remote partition, and all ns are assigned a tschema; nested aggregation with target-schema label missing from inner "by" clause - Test(query=s"""sum(sum(foo{_ws_="demo",_ns_=~"$ONE_REMOTE"})) by ($TSCHEMA_LABEL)""", - tschemaEnabled=Set("oneRemote1", "oneRemote2"), - expected="""T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634777330)) - |-E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on InProcessPlanDispatcher - |--T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) - |---T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634777330)) - |----E~MultiPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on InProcessPlanDispatcher - |-----E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_="oneRemote1"}),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(sum(foo{_ws_="demo",_ns_="oneRemote2"}),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), - - // ------- begin binary-join tests -------- - - // no namespaces with target-schemas enabled - Test(query=s"""foo{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) baz{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bat"}""", - tschemaEnabled=Set(), - expected="""E~BinaryJoinExec(binaryOp=ADD, on=Some(List(tschemaLabel)), ignoring=List()) on InProcessPlanDispatcher - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |--E~StitchRvsExec() on InProcessPlanDispatcher - |---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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), 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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |---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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), 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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--E~StitchRvsExec() on InProcessPlanDispatcher - |---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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), 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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |---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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), 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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |--E~StitchRvsExec() on InProcessPlanDispatcher - |---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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(baz))), 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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |---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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(baz))), 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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--E~StitchRvsExec() on InProcessPlanDispatcher - |---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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(baz))), 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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |---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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(baz))), 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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw)""".stripMargin), - // one namespace has a target-schema assigned - Test(query=s"""foo{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) baz{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bat"}""", - tschemaEnabled=Set("local1"), - expected="""E~BinaryJoinExec(binaryOp=ADD, on=Some(List(tschemaLabel)), ignoring=List()) on InProcessPlanDispatcher - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |--E~StitchRvsExec() on InProcessPlanDispatcher - |---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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), 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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |---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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), 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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--E~StitchRvsExec() on InProcessPlanDispatcher - |---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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |---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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |--E~StitchRvsExec() on InProcessPlanDispatcher - |---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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(baz))), 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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |---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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(baz))), 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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--E~StitchRvsExec() on InProcessPlanDispatcher - |---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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |---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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw)""".stripMargin), - // both namespaces are assigned target-schemas; should concat results - Test(query=s"""foo{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) baz{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bat"}""", - tschemaEnabled=Set("local1", "local2"), - expected="""E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |-E~StitchRvsExec() on InProcessPlanDispatcher - |--E~BinaryJoinExec(binaryOp=ADD, on=Some(List(tschemaLabel)), ignoring=List()) 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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), 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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |--E~BinaryJoinExec(binaryOp=ADD, on=Some(List(tschemaLabel)), ignoring=List()) 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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), 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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-E~StitchRvsExec() on InProcessPlanDispatcher - |--E~BinaryJoinExec(binaryOp=ADD, on=Some(List(tschemaLabel)), ignoring=List()) 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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(baz))), 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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |--E~BinaryJoinExec(binaryOp=ADD, on=Some(List(tschemaLabel)), ignoring=List()) 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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(baz))), 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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw)""".stripMargin), - // remove "on" clause; should not concatenate - Test(query=s"""foo{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bar"} + baz{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bat"}""", - tschemaEnabled=Set("local1", "local2"), - expected="""E~BinaryJoinExec(binaryOp=ADD, on=None, ignoring=List()) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,true,false,true,Set(),Some(plannerSelector))) - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,true,false,true,Set(),Some(plannerSelector))) - |--E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,false,false,true,Set(),None)) - |---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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#2094049380],raw) - |---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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#2094049380],downsample) - |--E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,false,false,true,Set(),None)) - |---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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#2094049380],raw) - |---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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#2094049380],downsample) - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,true,false,true,Set(),Some(plannerSelector))) - |--E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,false,false,true,Set(),None)) - |---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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#2094049380],raw) - |---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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#2094049380],downsample) - |--E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,false,false,true,Set(),None)) - |---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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#2094049380],raw) - |---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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#2094049380],downsample)""".stripMargin), - // "on" clause is missing a target-schema label - Test(query=s"""foo{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bar"} + on (notATargetSchemaLabel) baz{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bat"}""", - tschemaEnabled=Set("local1", "local2"), - expected="""E~BinaryJoinExec(binaryOp=ADD, on=Some(List(notATargetSchemaLabel)), ignoring=List()) on InProcessPlanDispatcher - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |--E~StitchRvsExec() on InProcessPlanDispatcher - |---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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |---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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--E~StitchRvsExec() on InProcessPlanDispatcher - |---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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |---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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |--E~StitchRvsExec() on InProcessPlanDispatcher - |---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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |---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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--E~StitchRvsExec() on InProcessPlanDispatcher - |---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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |---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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw)""".stripMargin), - // remote partition, no target-schemas assigned - Test(query=s"""foo{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) baz{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bat"}""", - tschemaEnabled=Set(), - expected="""E~BinaryJoinExec(binaryOp=ADD, on=Some(List(tschemaLabel)), ignoring=List()) on InProcessPlanDispatcher - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |--E~PromQlRemoteExec(PromQlQueryParams(baz{tschemaLabel="bat",_ws_="demo",_ns_="oneRemote1"},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(baz{tschemaLabel="bat",_ws_="demo",_ns_="oneRemote2"},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~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |--E~PromQlRemoteExec(PromQlQueryParams(foo{tschemaLabel="bar",_ws_="demo",_ns_="oneRemote1"},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"},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), - // remote partition, one ns has a target-schema assigned - Test(query=s"""foo{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) baz{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bat"}""", - tschemaEnabled=Set("oneRemote1"), - expected="""E~BinaryJoinExec(binaryOp=ADD, on=Some(List(tschemaLabel)), ignoring=List()) on InProcessPlanDispatcher - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |--E~PromQlRemoteExec(PromQlQueryParams(baz{tschemaLabel="bat",_ws_="demo",_ns_="oneRemote1"},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(baz{tschemaLabel="bat",_ws_="demo",_ns_="oneRemote2"},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~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |--E~PromQlRemoteExec(PromQlQueryParams(foo{tschemaLabel="bar",_ws_="demo",_ns_="oneRemote1"},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"},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), - // remote partition, both ns have a target-schema assigned - Test(query=s"""foo{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) baz{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bat"}""", - tschemaEnabled=Set("oneRemote1", "oneRemote2"), - expected="""E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |-E~PromQlRemoteExec(PromQlQueryParams((foo{tschemaLabel="bar",_ws_="demo",_ns_="oneRemote1"} + on(tschemaLabel) baz{tschemaLabel="bat",_ws_="demo",_ns_="oneRemote1"}),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) baz{tschemaLabel="bat",_ws_="demo",_ns_="oneRemote2"}),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), - // two remote partitions, both ns have a target-schema assigned - Test(query=s"""foo{_ws_="demo",_ns_=~"$TWO_REMOTE",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) baz{_ws_="demo",_ns_=~"$TWO_REMOTE",$TSCHEMA_LABEL="bat"}""", - tschemaEnabled=Set("twoRemote1", "twoRemote2"), - expected="""E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |-E~PromQlRemoteExec(PromQlQueryParams((foo{tschemaLabel="bar",_ws_="demo",_ns_="twoRemote1"} + on(tschemaLabel) baz{tschemaLabel="bat",_ws_="demo",_ns_="twoRemote1"}),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=twoRemote1-url, requestTimeoutMs=10000) on InProcessPlanDispatcher - |-E~PromQlRemoteExec(PromQlQueryParams((foo{tschemaLabel="bar",_ws_="demo",_ns_="twoRemote2"} + on(tschemaLabel) baz{tschemaLabel="bat",_ws_="demo",_ns_="twoRemote2"}),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=twoRemote2-url, requestTimeoutMs=10000) on InProcessPlanDispatcher""".stripMargin), - // remote partition, both ns have a target-schema assigned, but "on" clause is missing - Test(query=s"""foo{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"} + baz{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bat"}""", - tschemaEnabled=Set("oneRemote1", "oneRemote2"), - expected="""E~BinaryJoinExec(binaryOp=ADD, on=None, ignoring=List()) on InProcessPlanDispatcher - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |--E~PromQlRemoteExec(PromQlQueryParams(baz{tschemaLabel="bat",_ws_="demo",_ns_="oneRemote1"},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(baz{tschemaLabel="bat",_ws_="demo",_ns_="oneRemote2"},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~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |--E~PromQlRemoteExec(PromQlQueryParams(foo{tschemaLabel="bar",_ws_="demo",_ns_="oneRemote1"},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"},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), - // remote/local partitions, no ns has a tschema assigned - Test(query=s"""foo{_ws_="demo",_ns_=~"$BOTH",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) baz{_ws_="demo",_ns_=~"$BOTH",$TSCHEMA_LABEL="bat"}""", - tschemaEnabled=Set(), - expected="""E~BinaryJoinExec(binaryOp=ADD, on=Some(List(tschemaLabel)), ignoring=List()) on InProcessPlanDispatcher - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |--E~PromQlRemoteExec(PromQlQueryParams(baz{tschemaLabel="bat",_ws_="demo",_ns_="remote"},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~StitchRvsExec() on InProcessPlanDispatcher - |---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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(baz))), 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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |---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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(baz))), 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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |--E~PromQlRemoteExec(PromQlQueryParams(foo{tschemaLabel="bar",_ws_="demo",_ns_="remote"},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~StitchRvsExec() on InProcessPlanDispatcher - |---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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(foo))), 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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |---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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(foo))), 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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw)""".stripMargin), - // remote/local partitions, local ns has a tschema assigned - Test(query=s"""foo{_ws_="demo",_ns_=~"$BOTH",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) baz{_ws_="demo",_ns_=~"$BOTH",$TSCHEMA_LABEL="bat"}""", - tschemaEnabled=Set("local"), - expected="""E~BinaryJoinExec(binaryOp=ADD, on=Some(List(tschemaLabel)), ignoring=List()) on InProcessPlanDispatcher - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |--E~PromQlRemoteExec(PromQlQueryParams(baz{tschemaLabel="bat",_ws_="demo",_ns_="remote"},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~StitchRvsExec() on InProcessPlanDispatcher - |---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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |---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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |--E~PromQlRemoteExec(PromQlQueryParams(foo{tschemaLabel="bar",_ws_="demo",_ns_="remote"},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~StitchRvsExec() on InProcessPlanDispatcher - |---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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |---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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw)""".stripMargin), - // remote/local partitions, remote ns has a tschema assigned - Test(query=s"""foo{_ws_="demo",_ns_=~"$BOTH",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) baz{_ws_="demo",_ns_=~"$BOTH",$TSCHEMA_LABEL="bat"}""", - tschemaEnabled=Set("remote"), - expected="""E~BinaryJoinExec(binaryOp=ADD, on=Some(List(tschemaLabel)), ignoring=List()) on InProcessPlanDispatcher - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |--E~PromQlRemoteExec(PromQlQueryParams(baz{tschemaLabel="bat",_ws_="demo",_ns_="remote"},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~StitchRvsExec() on InProcessPlanDispatcher - |---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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(baz))), 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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |---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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(baz))), 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(tschemaLabel,Equals(bat)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |--E~PromQlRemoteExec(PromQlQueryParams(foo{tschemaLabel="bar",_ws_="demo",_ns_="remote"},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~StitchRvsExec() on InProcessPlanDispatcher - |---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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(foo))), 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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |---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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(foo))), 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(tschemaLabel,Equals(bar)), ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw)""".stripMargin), - // one remote partition, and all ns are assigned a tschema, but tschema label missing from selector - Test(query=s"""foo{_ws_="demo",_ns_=~"$ONE_REMOTE"} + on ($TSCHEMA_LABEL) baz{_ws_="demo",_ns_=~"$ONE_REMOTE"}""", - tschemaEnabled=Set("oneRemote1", "oneRemote2"), - expected="""E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |-E~PromQlRemoteExec(PromQlQueryParams((foo{_ws_="demo",_ns_="oneRemote1"} + on(tschemaLabel) baz{_ws_="demo",_ns_="oneRemote1"}),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{_ws_="demo",_ns_="oneRemote2"} + on(tschemaLabel) baz{_ws_="demo",_ns_="oneRemote2"}),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), - // one remote partition, and all ns are assigned a tschema, but tschema label missing from selector and "by" clause - Test(query=s"""foo{_ws_="demo",_ns_=~"$ONE_REMOTE"} + on (notATargetSchemaLabel) baz{_ws_="demo",_ns_=~"$ONE_REMOTE"}""", - tschemaEnabled=Set("oneRemote1", "oneRemote2"), - expected="""E~BinaryJoinExec(binaryOp=ADD, on=Some(List(notATargetSchemaLabel)), ignoring=List()) on InProcessPlanDispatcher - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |--E~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_="oneRemote1"},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(baz{_ws_="demo",_ns_="oneRemote2"},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~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_="oneRemote1"},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{_ws_="demo",_ns_="oneRemote2"},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), - // local partition, and all ns are assigned a tschema, but tschema label missing from selector - Test(query=s"""foo{_ws_="demo",_ns_=~"$LOCAL"} + on ($TSCHEMA_LABEL) baz{_ws_="demo",_ns_=~"$LOCAL"}""", - tschemaEnabled=Set("local1", "local2"), - expected="""E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |-E~StitchRvsExec() on InProcessPlanDispatcher - |--E~BinaryJoinExec(binaryOp=ADD, on=Some(List(tschemaLabel)), ignoring=List()) 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_,Equals(local1)), ColumnFilter(_metric_,Equals(baz))), 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=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), 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_,Equals(local1)), ColumnFilter(_metric_,Equals(baz))), 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_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |--E~BinaryJoinExec(binaryOp=ADD, on=Some(List(tschemaLabel)), ignoring=List()) 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_,Equals(local1)), ColumnFilter(_metric_,Equals(baz))), 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=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), 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_,Equals(local1)), ColumnFilter(_metric_,Equals(baz))), 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_,Equals(local1)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-E~StitchRvsExec() on InProcessPlanDispatcher - |--E~BinaryJoinExec(binaryOp=ADD, on=Some(List(tschemaLabel)), ignoring=List()) 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_,Equals(local2)), ColumnFilter(_metric_,Equals(baz))), 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=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), 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_,Equals(local2)), ColumnFilter(_metric_,Equals(baz))), 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_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],downsample) - |--E~BinaryJoinExec(binaryOp=ADD, on=Some(List(tschemaLabel)), ignoring=List()) 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_,Equals(local2)), ColumnFilter(_metric_,Equals(baz))), 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=0, chunkMethod=TimeRangeChunkScan(1634172830000,1634777330000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), 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_,Equals(local2)), ColumnFilter(_metric_,Equals(baz))), 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_,Equals(local2)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw)""".stripMargin), - // one remote partition, and all ns are assigned a tschema; nested join - Test(query=s"""foo{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) (baz{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"} - on($TSCHEMA_LABEL) bak{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"})""", - tschemaEnabled=Set("oneRemote1", "oneRemote2"), - expected="""E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |-E~PromQlRemoteExec(PromQlQueryParams((foo{tschemaLabel="bar",_ws_="demo",_ns_="oneRemote1"} + on(tschemaLabel) (baz{tschemaLabel="bar",_ws_="demo",_ns_="oneRemote1"} - on(tschemaLabel) bak{tschemaLabel="bar",_ws_="demo",_ns_="oneRemote1"})),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) (baz{tschemaLabel="bar",_ws_="demo",_ns_="oneRemote2"} - on(tschemaLabel) bak{tschemaLabel="bar",_ws_="demo",_ns_="oneRemote2"})),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), - // one remote partition, and all ns are assigned a tschema; nested aggregation with target-schema label missing from outer "by" clause - Test(query=s"""foo{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"} + on (notATargetSchemaLabel) (baz{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"} - on($TSCHEMA_LABEL) bak{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"})""", - tschemaEnabled=Set("oneRemote1", "oneRemote2"), - expected="""E~BinaryJoinExec(binaryOp=ADD, on=Some(List(notATargetSchemaLabel)), ignoring=List()) on InProcessPlanDispatcher - |-E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |--E~PromQlRemoteExec(PromQlQueryParams((baz{tschemaLabel="bar",_ws_="demo",_ns_="oneRemote1"} - on(tschemaLabel) bak{tschemaLabel="bar",_ws_="demo",_ns_="oneRemote1"}),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((baz{tschemaLabel="bar",_ws_="demo",_ns_="oneRemote2"} - on(tschemaLabel) bak{tschemaLabel="bar",_ws_="demo",_ns_="oneRemote2"}),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~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |--E~PromQlRemoteExec(PromQlQueryParams(foo{tschemaLabel="bar",_ws_="demo",_ns_="oneRemote1"},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"},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), - // one remote partition, and all ns are assigned a tschema; nested aggregation with target-schema label missing from inner "by" clause - Test(query=s"""foo{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) (baz{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"} - on (notATargetSchemaLabel) bak{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"})""", - tschemaEnabled=Set("oneRemote1", "oneRemote2"), - expected="""E~BinaryJoinExec(binaryOp=ADD, on=Some(List(tschemaLabel)), ignoring=List()) on InProcessPlanDispatcher - |-E~BinaryJoinExec(binaryOp=SUB, on=Some(List(notATargetSchemaLabel)), ignoring=List()) on InProcessPlanDispatcher - |--E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |---E~PromQlRemoteExec(PromQlQueryParams(bak{tschemaLabel="bar",_ws_="demo",_ns_="oneRemote1"},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(bak{tschemaLabel="bar",_ws_="demo",_ns_="oneRemote2"},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~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |---E~PromQlRemoteExec(PromQlQueryParams(baz{tschemaLabel="bar",_ws_="demo",_ns_="oneRemote1"},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(baz{tschemaLabel="bar",_ws_="demo",_ns_="oneRemote2"},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~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |--E~PromQlRemoteExec(PromQlQueryParams(foo{tschemaLabel="bar",_ws_="demo",_ns_="oneRemote1"},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"},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), - - // ----------------- begin compound agg/join tests --------------------- - - // one remote partition, and all ns are assigned a tschema; nested join - Test(query=s"""sum(foo{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) baz{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"}) by ($TSCHEMA_LABEL)""", - tschemaEnabled=Set("oneRemote1", "oneRemote2"), - expected="""E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |-E~PromQlRemoteExec(PromQlQueryParams(sum((foo{tschemaLabel="bar",_ws_="demo",_ns_="oneRemote1"} + on(tschemaLabel) 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(sum((foo{tschemaLabel="bar",_ws_="demo",_ns_="oneRemote2"} + on(tschemaLabel) 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), - - // one remote partition, and all ns are assigned a tschema; nested aggregation - Test(query=s"""foo{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) sum(baz{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"}) by ($TSCHEMA_LABEL)""", - tschemaEnabled=Set("oneRemote1", "oneRemote2"), - expected="""E~MultiPartitionDistConcatExec() on InProcessPlanDispatcher - |-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) + Test(query = s"""sum(foo{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bar"}) by ($TSCHEMA_LABEL)""", + tschemaEnabled = Set(), + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192),false,3 days,true)) + |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1634173130,300,1634777330)) + |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],raw) + |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |----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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],raw) + |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |----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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],raw) + |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634172830)) + |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],downsample) + |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |----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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],downsample) + |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |----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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],downsample)""".stripMargin), + Test(query = s"""sum(foo{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bar"}) by ($TSCHEMA_LABEL)""", + tschemaEnabled = Set("local1"), + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192),false,3 days,true)) + |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1634173130,300,1634777330)) + |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],raw) + |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |----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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],raw) + |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |----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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],raw) + |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634172830)) + |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],downsample) + |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |----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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],downsample) + |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |----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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],downsample)""".stripMargin), + Test(query = s"""sum(foo{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bar"}) by ($TSCHEMA_LABEL)""", + tschemaEnabled = Set("local1", "local2"), + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192),false,3 days,true)) + |-E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],raw) + |--T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1634173130,300,1634777330)) + |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],raw) + |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |-----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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),false,1800000 milliseconds,true)) + |--T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1634173130,300,1634777330)) + |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],raw) + |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |-----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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),false,1800000 milliseconds,true)) + |-E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],downsample) + |--T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634172830)) + |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],downsample) + |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |-----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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),false,1800000 milliseconds,true)) + |--T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634172830)) + |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],downsample) + |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |-----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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),false,1800000 milliseconds,true))""".stripMargin), + Test(query = s"""sum(foo{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bar"})""", + tschemaEnabled = Set("local1", "local2"), + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192),false,3 days,true)) + |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1634173130,300,1634777330)) + |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],raw) + |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634172830)) + |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],downsample)""".stripMargin), + Test(query = s"""sum(foo{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bar"}) by (notATargetSchemaLabel)""", + tschemaEnabled = Set("local1", "local2"), + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192),false,3 days,true)) + |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1634173130,300,1634777330)) + |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],raw) + |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(notATargetSchemaLabel)) + |----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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],raw) + |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(notATargetSchemaLabel)) + |----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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],raw) + |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634172830)) + |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],downsample) + |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(notATargetSchemaLabel)) + |----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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],downsample) + |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(notATargetSchemaLabel)) + |----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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],downsample)""".stripMargin), + Test(query = s"""sum(foo{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"}) by ($TSCHEMA_LABEL)""", + tschemaEnabled = Set(), + expected = """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),false,1800000 milliseconds,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true))""".stripMargin), + Test(query = s"""sum(foo{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"}) by ($TSCHEMA_LABEL)""", + tschemaEnabled = Set("oneRemote1"), + expected = """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),false,1800000 milliseconds,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true))""".stripMargin), + Test(query = s"""sum(foo{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"}) by ($TSCHEMA_LABEL)""", + tschemaEnabled = Set("oneRemote1", "oneRemote2"), + expected = """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),false,1800000 milliseconds,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true))""".stripMargin), + Test(query = s"""sum(foo{_ws_="demo",_ns_=~"$TWO_REMOTE",$TSCHEMA_LABEL="bar"}) by ($TSCHEMA_LABEL)""", + tschemaEnabled = Set("twoRemote1", "twoRemote2"), + expected = """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),false,1800000 milliseconds,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"twoRemote.*",tschemaLabel="bar"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=twoRemote1-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),false,1800000 milliseconds,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"twoRemote.*",tschemaLabel="bar"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=twoRemote2-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),false,1800000 milliseconds,true))""".stripMargin), + Test(query = s"""sum(foo{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"})""", + tschemaEnabled = Set("oneRemote1", "oneRemote2"), + expected = """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),false,1800000 milliseconds,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"}),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"}),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true))""".stripMargin), + Test(query = s"""sum(foo{_ws_="demo",_ns_=~"$BOTH",$TSCHEMA_LABEL="bar"}) by ($TSCHEMA_LABEL)""", + tschemaEnabled = Set(), + expected = """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),false,1800000 milliseconds,true)) + |--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),false,3 days,true)) + |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],raw) + |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |-----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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],raw) + |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |-----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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],raw) + |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],downsample) + |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |-----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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],downsample) + |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |-----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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],downsample) + |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"both.*",tschemaLabel="bar"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true))""".stripMargin), + Test(query = s"""sum(foo{_ws_="demo",_ns_=~"$BOTH",$TSCHEMA_LABEL="bar"}) by ($TSCHEMA_LABEL)""", + tschemaEnabled = Set("local"), + expected = """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),false,1800000 milliseconds,true)) + |--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),false,3 days,true)) + |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],raw) + |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |-----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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],raw) + |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |-----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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],raw) + |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],downsample) + |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |-----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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],downsample) + |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |-----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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],downsample) + |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"both.*",tschemaLabel="bar"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true))""".stripMargin), + Test(query = s"""sum(foo{_ws_="demo",_ns_=~"$BOTH",$TSCHEMA_LABEL="bar"}) by ($TSCHEMA_LABEL)""", + tschemaEnabled = Set("remote"), + expected = """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),false,1800000 milliseconds,true)) + |--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),false,3 days,true)) + |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],raw) + |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |-----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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],raw) + |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |-----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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],raw) + |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],downsample) + |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |-----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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],downsample) + |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |-----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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],downsample) + |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"both.*",tschemaLabel="bar"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true))""".stripMargin), + Test(query = s"""sum(foo{_ws_="demo",_ns_=~"$ONE_REMOTE"}) by ($TSCHEMA_LABEL)""", + tschemaEnabled = Set("oneRemote1", "oneRemote2"), + expected = """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),false,1800000 milliseconds,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"oneRemote.*"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"oneRemote.*"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true))""".stripMargin), + Test(query = s"""sum(foo{_ws_="demo",_ns_=~"$ONE_REMOTE"}) by (notALabel)""", + tschemaEnabled = Set("oneRemote1", "oneRemote2"), + expected = """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),false,1800000 milliseconds,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"oneRemote.*"}) by (notALabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"oneRemote.*"}) by (notALabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true))""".stripMargin), + Test(query = s"""sum(foo{_ws_="demo",_ns_=~"$LOCAL"}) by ($TSCHEMA_LABEL)""", + tschemaEnabled = Set("local1", "local2"), + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192),false,3 days,true)) + |-E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],raw) + |--T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1634173130,300,1634777330)) + |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],raw) + |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |-----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(local.*)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),false,1800000 milliseconds,true)) + |--T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1634173130,300,1634777330)) + |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],raw) + |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |-----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(local.*)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),false,1800000 milliseconds,true)) + |-E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],downsample) + |--T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634172830)) + |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],downsample) + |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |-----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(local.*)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),false,1800000 milliseconds,true)) + |--T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634172830)) + |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],downsample) + |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |-----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(local.*)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),false,1800000 milliseconds,true))""".stripMargin), + Test(query = s"""sum(sum(foo{_ws_="demo",_ns_=~"$ONE_REMOTE"}) by ($TSCHEMA_LABEL)) by ($TSCHEMA_LABEL)""", + tschemaEnabled = Set("oneRemote1", "oneRemote2"), + expected = """T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634777330)) + |-E~LocalPartitionReduceAggregateExec(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),false,1800000 milliseconds,true)) + |--T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |---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),false,1800000 milliseconds,true)) + |-----E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"oneRemote.*"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) + |-----E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"oneRemote.*"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true))""".stripMargin), + Test(query = s"""sum(sum(foo{_ws_="demo",_ns_=~"$ONE_REMOTE"}) by ($TSCHEMA_LABEL))""", + tschemaEnabled = Set("oneRemote1", "oneRemote2"), + expected = """T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634777330)) + |-E~LocalPartitionReduceAggregateExec(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),false,1800000 milliseconds,true)) + |--T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=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),false,1800000 milliseconds,true)) + |-----E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"oneRemote.*"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) + |-----E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"oneRemote.*"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true))""".stripMargin), + Test(query = s"""sum(sum(foo{_ws_="demo",_ns_=~"$ONE_REMOTE"})) by ($TSCHEMA_LABEL)""", + tschemaEnabled = Set("oneRemote1", "oneRemote2"), + expected = """T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634777330)) + |-E~LocalPartitionReduceAggregateExec(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),false,1800000 milliseconds,true)) + |--T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |---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),false,1800000 milliseconds,true)) + |-----E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"oneRemote.*"}),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) + |-----E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"oneRemote.*"}),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true))""".stripMargin), + Test(query = s"""foo{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) baz{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bat"}""", + tschemaEnabled = Set(), + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192),false,3 days,true)) + |-E~BinaryJoinExec(binaryOp=ADD, on=Some(List(tschemaLabel)), ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(local.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(local.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],raw) + |-E~BinaryJoinExec(binaryOp=ADD, on=Some(List(tschemaLabel)), ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(local.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(local.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],downsample)""".stripMargin), + Test(query = s"""foo{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) baz{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bat"}""", + tschemaEnabled = Set("local1"), + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192),false,3 days,true)) + |-E~BinaryJoinExec(binaryOp=ADD, on=Some(List(tschemaLabel)), ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(local.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(local.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],raw) + |-E~BinaryJoinExec(binaryOp=ADD, on=Some(List(tschemaLabel)), ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(local.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(local.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],downsample)""".stripMargin), + Test(query = s"""foo{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) baz{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bat"}""", + tschemaEnabled = Set("local1", "local2"), + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192),false,3 days,true)) + |-E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],raw) + |--E~BinaryJoinExec(binaryOp=ADD, on=Some(List(tschemaLabel)), ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),false,1800000 milliseconds,true)) + |---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(local.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),false,1800000 milliseconds,true)) + |--E~BinaryJoinExec(binaryOp=ADD, on=Some(List(tschemaLabel)), ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),false,1800000 milliseconds,true)) + |---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(local.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),false,1800000 milliseconds,true)) + |-E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],downsample) + |--E~BinaryJoinExec(binaryOp=ADD, on=Some(List(tschemaLabel)), ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),false,1800000 milliseconds,true)) + |---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(local.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),false,1800000 milliseconds,true)) + |--E~BinaryJoinExec(binaryOp=ADD, on=Some(List(tschemaLabel)), ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),false,1800000 milliseconds,true)) + |---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(local.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),false,1800000 milliseconds,true))""".stripMargin), + Test(query = s"""foo{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bar"} + baz{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bat"}""", + tschemaEnabled = Set("local1", "local2"), + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192),false,3 days,true)) + |-E~BinaryJoinExec(binaryOp=ADD, on=None, ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(local.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(local.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],raw) + |-E~BinaryJoinExec(binaryOp=ADD, on=None, ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(local.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(local.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],downsample)""".stripMargin), + Test(query = s"""foo{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bar"} + on (notATargetSchemaLabel) baz{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bat"}""", + tschemaEnabled = Set("local1", "local2"), + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192),false,3 days,true)) + |-E~BinaryJoinExec(binaryOp=ADD, on=Some(List(notATargetSchemaLabel)), ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(local.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(local.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],raw) + |-E~BinaryJoinExec(binaryOp=ADD, on=Some(List(notATargetSchemaLabel)), ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(local.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(local.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],downsample)""".stripMargin), + Test(query = s"""foo{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) baz{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bat"}""", + tschemaEnabled = Set(), + expected = """E~BinaryJoinExec(binaryOp=ADD, on=Some(List(tschemaLabel)), 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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bat"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bat"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true))""".stripMargin), + Test(query = s"""foo{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) baz{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bat"}""", + tschemaEnabled = Set("oneRemote1"), + expected = """E~BinaryJoinExec(binaryOp=ADD, on=Some(List(tschemaLabel)), 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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bat"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bat"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true))""".stripMargin), + Test(query = s"""foo{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) baz{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bat"}""", + tschemaEnabled = Set("oneRemote1", "oneRemote2"), + expected = """E~BinaryJoinExec(binaryOp=ADD, on=Some(List(tschemaLabel)), 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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bat"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bat"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true))""".stripMargin), + Test(query = s"""foo{_ws_="demo",_ns_=~"$TWO_REMOTE",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) baz{_ws_="demo",_ns_=~"$TWO_REMOTE",$TSCHEMA_LABEL="bat"}""", + tschemaEnabled = Set("twoRemote1", "twoRemote2"), + expected = """E~BinaryJoinExec(binaryOp=ADD, on=Some(List(tschemaLabel)), 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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"twoRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=twoRemote1-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),false,1800000 milliseconds,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"twoRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=twoRemote2-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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"twoRemote.*",tschemaLabel="bat"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=twoRemote1-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),false,1800000 milliseconds,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"twoRemote.*",tschemaLabel="bat"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=twoRemote2-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),false,1800000 milliseconds,true))""".stripMargin), + Test(query = s"""foo{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"} + baz{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bat"}""", + tschemaEnabled = Set("oneRemote1", "oneRemote2"), + expected = """E~BinaryJoinExec(binaryOp=ADD, on=None, 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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bat"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bat"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true))""".stripMargin), + Test(query = s"""foo{_ws_="demo",_ns_=~"$BOTH",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) baz{_ws_="demo",_ns_=~"$BOTH",$TSCHEMA_LABEL="bat"}""", + tschemaEnabled = Set(), + expected = """E~BinaryJoinExec(binaryOp=ADD, on=Some(List(tschemaLabel)), 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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |--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),false,3 days,true)) + |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],raw) + |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],downsample) + |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"both.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |--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),false,3 days,true)) + |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(both.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(both.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],raw) + |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(both.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(both.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],downsample) + |--E~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"both.*",tschemaLabel="bat"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true))""".stripMargin), + Test(query = s"""foo{_ws_="demo",_ns_=~"$BOTH",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) baz{_ws_="demo",_ns_=~"$BOTH",$TSCHEMA_LABEL="bat"}""", + tschemaEnabled = Set("local"), + expected = """E~BinaryJoinExec(binaryOp=ADD, on=Some(List(tschemaLabel)), 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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |--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),false,3 days,true)) + |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],raw) + |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],downsample) + |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"both.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |--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),false,3 days,true)) + |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(both.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(both.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],raw) + |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(both.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(both.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],downsample) + |--E~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"both.*",tschemaLabel="bat"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true))""".stripMargin), + Test(query = s"""foo{_ws_="demo",_ns_=~"$BOTH",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) baz{_ws_="demo",_ns_=~"$BOTH",$TSCHEMA_LABEL="bat"}""", + tschemaEnabled = Set("remote"), + expected = """E~BinaryJoinExec(binaryOp=ADD, on=Some(List(tschemaLabel)), 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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |--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),false,3 days,true)) + |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],raw) + |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],downsample) + |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"both.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |--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),false,3 days,true)) + |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(both.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(both.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],raw) + |---E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(both.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(both.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],downsample) + |--E~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"both.*",tschemaLabel="bat"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true))""".stripMargin), + Test(query = s"""foo{_ws_="demo",_ns_=~"$ONE_REMOTE"} + on ($TSCHEMA_LABEL) baz{_ws_="demo",_ns_=~"$ONE_REMOTE"}""", + tschemaEnabled = Set("oneRemote1", "oneRemote2"), + expected = """E~BinaryJoinExec(binaryOp=ADD, on=Some(List(tschemaLabel)), 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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"oneRemote.*"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"oneRemote.*"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"oneRemote.*"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"oneRemote.*"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true))""".stripMargin), + Test(query = s"""foo{_ws_="demo",_ns_=~"$ONE_REMOTE"} + on (notATargetSchemaLabel) baz{_ws_="demo",_ns_=~"$ONE_REMOTE"}""", + tschemaEnabled = Set("oneRemote1", "oneRemote2"), + expected = """E~BinaryJoinExec(binaryOp=ADD, on=Some(List(notATargetSchemaLabel)), 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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"oneRemote.*"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"oneRemote.*"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"oneRemote.*"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"oneRemote.*"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true))""".stripMargin), + Test(query = s"""foo{_ws_="demo",_ns_=~"$LOCAL"} + on ($TSCHEMA_LABEL) baz{_ws_="demo",_ns_=~"$LOCAL"}""", + tschemaEnabled = Set("local1", "local2"), + expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192),false,3 days,true)) + |-E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],raw) + |--E~BinaryJoinExec(binaryOp=ADD, on=Some(List(tschemaLabel)), ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(local.*)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),false,1800000 milliseconds,true)) + |---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(local.*)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),false,1800000 milliseconds,true)) + |--E~BinaryJoinExec(binaryOp=ADD, on=Some(List(tschemaLabel)), ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(local.*)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),false,1800000 milliseconds,true)) + |---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(local.*)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),false,1800000 milliseconds,true)) + |-E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],downsample) + |--E~BinaryJoinExec(binaryOp=ADD, on=Some(List(tschemaLabel)), ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(local.*)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),false,1800000 milliseconds,true)) + |---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(local.*)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),false,1800000 milliseconds,true)) + |--E~BinaryJoinExec(binaryOp=ADD, on=Some(List(tschemaLabel)), ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(local.*)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),false,1800000 milliseconds,true)) + |---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(local.*)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),false,1800000 milliseconds,true))""".stripMargin), + Test(query = s"""foo{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) (baz{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"} - on($TSCHEMA_LABEL) bak{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"})""", + tschemaEnabled = Set("oneRemote1", "oneRemote2"), + expected = """E~BinaryJoinExec(binaryOp=ADD, on=Some(List(tschemaLabel)), 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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) + |-E~BinaryJoinExec(binaryOp=SUB, on=Some(List(tschemaLabel)), 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),false,1800000 milliseconds,true)) + |--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),false,1800000 milliseconds,true)) + |---E~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) + |---E~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) + |--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),false,1800000 milliseconds,true)) + |---E~PromQlRemoteExec(PromQlQueryParams(bak{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) + |---E~PromQlRemoteExec(PromQlQueryParams(bak{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true))""".stripMargin), + Test(query = s"""foo{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"} + on (notATargetSchemaLabel) (baz{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"} - on($TSCHEMA_LABEL) bak{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"})""", + tschemaEnabled = Set("oneRemote1", "oneRemote2"), + expected = """E~BinaryJoinExec(binaryOp=ADD, on=Some(List(notATargetSchemaLabel)), 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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) + |-E~BinaryJoinExec(binaryOp=SUB, on=Some(List(tschemaLabel)), 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),false,1800000 milliseconds,true)) + |--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),false,1800000 milliseconds,true)) + |---E~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) + |---E~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) + |--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),false,1800000 milliseconds,true)) + |---E~PromQlRemoteExec(PromQlQueryParams(bak{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) + |---E~PromQlRemoteExec(PromQlQueryParams(bak{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true))""".stripMargin), + Test(query = s"""foo{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) (baz{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"} - on (notATargetSchemaLabel) bak{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"})""", + tschemaEnabled = Set("oneRemote1", "oneRemote2"), + expected = """E~BinaryJoinExec(binaryOp=ADD, on=Some(List(tschemaLabel)), 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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) + |-E~BinaryJoinExec(binaryOp=SUB, on=Some(List(notATargetSchemaLabel)), 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),false,1800000 milliseconds,true)) + |--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),false,1800000 milliseconds,true)) + |---E~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) + |---E~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) + |--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),false,1800000 milliseconds,true)) + |---E~PromQlRemoteExec(PromQlQueryParams(bak{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) + |---E~PromQlRemoteExec(PromQlQueryParams(bak{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true))""".stripMargin), + Test(query = s"""sum(foo{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) baz{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"}) by ($TSCHEMA_LABEL)""", + tschemaEnabled = Set("oneRemote1", "oneRemote2"), + expected = """T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634777330)) + |-E~LocalPartitionReduceAggregateExec(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),false,1800000 milliseconds,true)) + |--T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) + |---E~BinaryJoinExec(binaryOp=ADD, on=Some(List(tschemaLabel)), 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),false,1800000 milliseconds,true)) + |----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),false,1800000 milliseconds,true)) + |-----E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) + |-----E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) + |----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),false,1800000 milliseconds,true)) + |-----E~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) + |-----E~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true))""".stripMargin), + Test(query = s"""foo{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) sum(baz{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"}) by ($TSCHEMA_LABEL)""", + tschemaEnabled = Set("oneRemote1", "oneRemote2"), + expected = """E~BinaryJoinExec(binaryOp=ADD, on=Some(List(tschemaLabel)), 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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) + |--E~PromQlRemoteExec(PromQlQueryParams(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) + |-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),false,1800000 milliseconds,true)) + |---E~PromQlRemoteExec(PromQlQueryParams(sum(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) + |---E~PromQlRemoteExec(PromQlQueryParams(sum(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true))""".stripMargin) ) for (test <- tests) { def tschemaProviderFunc(filters: Seq[ColumnFilter]): Seq[TargetSchemaChange] = { From d90b08f61dfd9b3a17638358659f4bb8dc4e2c43 Mon Sep 17 00:00:00 2001 From: Alex Theimer Date: Tue, 9 Jan 2024 14:40:17 -0800 Subject: [PATCH 24/27] re-add comments --- .../queryplanner/PlannerHierarchySpec.scala | 46 +++++++++++++++++++ 1 file changed, 46 insertions(+) diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala index 6c5c7ff4e0..bd38467827 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala @@ -2974,6 +2974,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS case class Test(query: String, expected: String, tschemaEnabled: Set[String] = Set()) val tests = Seq( + // no namespaces with target-schemas enabled Test(query = s"""sum(foo{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bar"}) by ($TSCHEMA_LABEL)""", tschemaEnabled = Set(), expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192),false,3 days,true)) @@ -2993,6 +2994,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) |----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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],downsample)""".stripMargin), + // one namespace has a target-schema assigned Test(query = s"""sum(foo{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bar"}) by ($TSCHEMA_LABEL)""", tschemaEnabled = Set("local1"), expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192),false,3 days,true)) @@ -3012,6 +3014,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) |----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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],downsample)""".stripMargin), + // both namespaces are assigned target-schemas; should concat results Test(query = s"""sum(foo{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bar"}) by ($TSCHEMA_LABEL)""", tschemaEnabled = Set("local1", "local2"), expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192),false,3 days,true)) @@ -3037,6 +3040,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) |-----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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),false,1800000 milliseconds,true))""".stripMargin), + // remove "by" clause; should not concatenate Test(query = s"""sum(foo{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bar"})""", tschemaEnabled = Set("local1", "local2"), expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192),false,3 days,true)) @@ -3056,6 +3060,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(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],downsample)""".stripMargin), + // "by" clause is missing a target-schema label Test(query = s"""sum(foo{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bar"}) by (notATargetSchemaLabel)""", tschemaEnabled = Set("local1", "local2"), expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192),false,3 days,true)) @@ -3075,36 +3080,42 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(notATargetSchemaLabel)) |----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(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],downsample)""".stripMargin), + // remote partition, no target-schemas assigned Test(query = s"""sum(foo{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"}) by ($TSCHEMA_LABEL)""", tschemaEnabled = Set(), expected = """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),false,1800000 milliseconds,true)) |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true))""".stripMargin), + // remote partition, one ns has a target-schema assigned Test(query = s"""sum(foo{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"}) by ($TSCHEMA_LABEL)""", tschemaEnabled = Set("oneRemote1"), expected = """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),false,1800000 milliseconds,true)) |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true))""".stripMargin), + // remote partition, both ns have a target-schema assigned Test(query = s"""sum(foo{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"}) by ($TSCHEMA_LABEL)""", tschemaEnabled = Set("oneRemote1", "oneRemote2"), expected = """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),false,1800000 milliseconds,true)) |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true))""".stripMargin), + // two remote partitions, both ns have a target-schema assigned Test(query = s"""sum(foo{_ws_="demo",_ns_=~"$TWO_REMOTE",$TSCHEMA_LABEL="bar"}) by ($TSCHEMA_LABEL)""", tschemaEnabled = Set("twoRemote1", "twoRemote2"), expected = """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),false,1800000 milliseconds,true)) |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"twoRemote.*",tschemaLabel="bar"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=twoRemote1-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),false,1800000 milliseconds,true)) |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"twoRemote.*",tschemaLabel="bar"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=twoRemote2-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),false,1800000 milliseconds,true))""".stripMargin), + // remote partition, both ns have a target-schema assigned, but "by" clause is missing Test(query = s"""sum(foo{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"})""", tschemaEnabled = Set("oneRemote1", "oneRemote2"), expected = """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),false,1800000 milliseconds,true)) |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"}),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"}),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true))""".stripMargin), + // remote/local partitions, no ns has a tschema assigned Test(query = s"""sum(foo{_ws_="demo",_ns_=~"$BOTH",$TSCHEMA_LABEL="bar"}) by ($TSCHEMA_LABEL)""", tschemaEnabled = Set(), expected = """T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634777330)) @@ -3125,6 +3136,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |-----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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],downsample) |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"both.*",tschemaLabel="bar"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true))""".stripMargin), + // remote/local partitions, local ns has a tschema assigned Test(query = s"""sum(foo{_ws_="demo",_ns_=~"$BOTH",$TSCHEMA_LABEL="bar"}) by ($TSCHEMA_LABEL)""", tschemaEnabled = Set("local"), expected = """T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634777330)) @@ -3145,6 +3157,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |-----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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],downsample) |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"both.*",tschemaLabel="bar"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true))""".stripMargin), + // remote/local partitions, remote ns has a tschema assigned Test(query = s"""sum(foo{_ws_="demo",_ns_=~"$BOTH",$TSCHEMA_LABEL="bar"}) by ($TSCHEMA_LABEL)""", tschemaEnabled = Set("remote"), expected = """T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634777330)) @@ -3165,18 +3178,21 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |-----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(both.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],downsample) |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"both.*",tschemaLabel="bar"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true))""".stripMargin), + // one remote partition, and all ns are assigned a tschema, but tschema label missing from selector Test(query = s"""sum(foo{_ws_="demo",_ns_=~"$ONE_REMOTE"}) by ($TSCHEMA_LABEL)""", tschemaEnabled = Set("oneRemote1", "oneRemote2"), expected = """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),false,1800000 milliseconds,true)) |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"oneRemote.*"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"oneRemote.*"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true))""".stripMargin), + // one remote partition, and all ns are assigned a tschema, but tschema label missing from selector and "by" clause Test(query = s"""sum(foo{_ws_="demo",_ns_=~"$ONE_REMOTE"}) by (notALabel)""", tschemaEnabled = Set("oneRemote1", "oneRemote2"), expected = """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),false,1800000 milliseconds,true)) |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"oneRemote.*"}) by (notALabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) |--E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"oneRemote.*"}) by (notALabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true))""".stripMargin), + // local partition, and all ns are assigned a tschema, but tschema label missing from selector Test(query = s"""sum(foo{_ws_="demo",_ns_=~"$LOCAL"}) by ($TSCHEMA_LABEL)""", tschemaEnabled = Set("local1", "local2"), expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192),false,3 days,true)) @@ -3202,6 +3218,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List(tschemaLabel)) |-----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(local.*)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),false,1800000 milliseconds,true))""".stripMargin), + // one remote partition, and all ns are assigned a tschema; nested aggregation with all target-schema labels present in each "by" clause Test(query = s"""sum(sum(foo{_ws_="demo",_ns_=~"$ONE_REMOTE"}) by ($TSCHEMA_LABEL)) by ($TSCHEMA_LABEL)""", tschemaEnabled = Set("oneRemote1", "oneRemote2"), expected = """T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634777330)) @@ -3211,6 +3228,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |----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),false,1800000 milliseconds,true)) |-----E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"oneRemote.*"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) |-----E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"oneRemote.*"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true))""".stripMargin), + // one remote partition, and all ns are assigned a tschema; nested aggregation with target-schema label missing from outer "by" clause Test(query = s"""sum(sum(foo{_ws_="demo",_ns_=~"$ONE_REMOTE"}) by ($TSCHEMA_LABEL))""", tschemaEnabled = Set("oneRemote1", "oneRemote2"), expected = """T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634777330)) @@ -3220,6 +3238,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |----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),false,1800000 milliseconds,true)) |-----E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"oneRemote.*"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) |-----E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"oneRemote.*"}) by (tschemaLabel),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true))""".stripMargin), + // one remote partition, and all ns are assigned a tschema; nested aggregation with target-schema label missing from inner "by" clause Test(query = s"""sum(sum(foo{_ws_="demo",_ns_=~"$ONE_REMOTE"})) by ($TSCHEMA_LABEL)""", tschemaEnabled = Set("oneRemote1", "oneRemote2"), expected = """T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634777330)) @@ -3229,6 +3248,10 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |----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),false,1800000 milliseconds,true)) |-----E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"oneRemote.*"}),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) |-----E~PromQlRemoteExec(PromQlQueryParams(sum(foo{_ws_="demo",_ns_=~"oneRemote.*"}),1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true))""".stripMargin), + + // ------- begin binary-join tests -------- + + // no namespaces with target-schemas enabled Test(query = s"""foo{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) baz{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bat"}""", tschemaEnabled = Set(), expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192),false,3 days,true)) @@ -3250,6 +3273,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=0, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(local.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(local.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],downsample)""".stripMargin), + // one namespace has a target-schema assigned Test(query = s"""foo{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) baz{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bat"}""", tschemaEnabled = Set("local1"), expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192),false,3 days,true)) @@ -3271,6 +3295,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(local.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(local.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],downsample)""".stripMargin), + // both namespaces are assigned target-schemas; should concat results Test(query = s"""foo{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) baz{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bat"}""", tschemaEnabled = Set("local1", "local2"), expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192),false,3 days,true)) @@ -3296,6 +3321,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(local.*)), ColumnFilter(tschemaLabel,Equals(bar)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),false,1800000 milliseconds,true)) |---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(local.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),false,1800000 milliseconds,true))""".stripMargin), + // remove "on" clause; should not concatenate Test(query = s"""foo{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bar"} + baz{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bat"}""", tschemaEnabled = Set("local1", "local2"), expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192),false,3 days,true)) @@ -3317,6 +3343,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(local.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(local.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],downsample)""".stripMargin), + // "on" clause is missing a target-schema label Test(query = s"""foo{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bar"} + on (notATargetSchemaLabel) baz{_ws_="demo",_ns_=~"$LOCAL",$TSCHEMA_LABEL="bat"}""", tschemaEnabled = Set("local1", "local2"), expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192),false,3 days,true)) @@ -3338,6 +3365,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(local.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],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(local.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],downsample)""".stripMargin), + // remote partition, no target-schemas assigned Test(query = s"""foo{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) baz{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bat"}""", tschemaEnabled = Set(), expected = """E~BinaryJoinExec(binaryOp=ADD, on=Some(List(tschemaLabel)), 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),false,1800000 milliseconds,true)) @@ -3347,6 +3375,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |-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),false,1800000 milliseconds,true)) |--E~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bat"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) |--E~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bat"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true))""".stripMargin), + // remote partition, one ns has a target-schema assigned Test(query = s"""foo{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) baz{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bat"}""", tschemaEnabled = Set("oneRemote1"), expected = """E~BinaryJoinExec(binaryOp=ADD, on=Some(List(tschemaLabel)), 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),false,1800000 milliseconds,true)) @@ -3356,6 +3385,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |-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),false,1800000 milliseconds,true)) |--E~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bat"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) |--E~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bat"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true))""".stripMargin), + // remote partition, both ns have a target-schema assigned Test(query = s"""foo{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) baz{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bat"}""", tschemaEnabled = Set("oneRemote1", "oneRemote2"), expected = """E~BinaryJoinExec(binaryOp=ADD, on=Some(List(tschemaLabel)), 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),false,1800000 milliseconds,true)) @@ -3365,6 +3395,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |-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),false,1800000 milliseconds,true)) |--E~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bat"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) |--E~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bat"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true))""".stripMargin), + // two remote partitions, both ns have a target-schema assigned Test(query = s"""foo{_ws_="demo",_ns_=~"$TWO_REMOTE",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) baz{_ws_="demo",_ns_=~"$TWO_REMOTE",$TSCHEMA_LABEL="bat"}""", tschemaEnabled = Set("twoRemote1", "twoRemote2"), expected = """E~BinaryJoinExec(binaryOp=ADD, on=Some(List(tschemaLabel)), 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),false,1800000 milliseconds,true)) @@ -3374,6 +3405,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |-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),false,1800000 milliseconds,true)) |--E~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"twoRemote.*",tschemaLabel="bat"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=twoRemote1-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),false,1800000 milliseconds,true)) |--E~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"twoRemote.*",tschemaLabel="bat"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=twoRemote2-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),false,1800000 milliseconds,true))""".stripMargin), + // remote partition, both ns have a target-schema assigned, but "on" clause is missing Test(query = s"""foo{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"} + baz{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bat"}""", tschemaEnabled = Set("oneRemote1", "oneRemote2"), expected = """E~BinaryJoinExec(binaryOp=ADD, on=None, 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),false,1800000 milliseconds,true)) @@ -3383,6 +3415,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |-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),false,1800000 milliseconds,true)) |--E~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bat"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) |--E~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bat"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true))""".stripMargin), + // remote/local partitions, no ns has a tschema assigned Test(query = s"""foo{_ws_="demo",_ns_=~"$BOTH",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) baz{_ws_="demo",_ns_=~"$BOTH",$TSCHEMA_LABEL="bat"}""", tschemaEnabled = Set(), expected = """E~BinaryJoinExec(binaryOp=ADD, on=Some(List(tschemaLabel)), 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),false,1800000 milliseconds,true)) @@ -3412,6 +3445,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |----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(both.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],downsample) |--E~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"both.*",tschemaLabel="bat"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true))""".stripMargin), + // remote/local partitions, local ns has a tschema assigned Test(query = s"""foo{_ws_="demo",_ns_=~"$BOTH",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) baz{_ws_="demo",_ns_=~"$BOTH",$TSCHEMA_LABEL="bat"}""", tschemaEnabled = Set("local"), expected = """E~BinaryJoinExec(binaryOp=ADD, on=Some(List(tschemaLabel)), 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),false,1800000 milliseconds,true)) @@ -3441,6 +3475,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |----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(both.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],downsample) |--E~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"both.*",tschemaLabel="bat"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true))""".stripMargin), + // remote/local partitions, remote ns has a tschema assigned Test(query = s"""foo{_ws_="demo",_ns_=~"$BOTH",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) baz{_ws_="demo",_ns_=~"$BOTH",$TSCHEMA_LABEL="bat"}""", tschemaEnabled = Set("remote"), expected = """E~BinaryJoinExec(binaryOp=ADD, on=Some(List(tschemaLabel)), 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),false,1800000 milliseconds,true)) @@ -3470,6 +3505,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |----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(both.*)), ColumnFilter(tschemaLabel,Equals(bat)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-431108414],downsample) |--E~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"both.*",tschemaLabel="bat"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true))""".stripMargin), + // one remote partition, and all ns are assigned a tschema, but tschema label missing from selector Test(query = s"""foo{_ws_="demo",_ns_=~"$ONE_REMOTE"} + on ($TSCHEMA_LABEL) baz{_ws_="demo",_ns_=~"$ONE_REMOTE"}""", tschemaEnabled = Set("oneRemote1", "oneRemote2"), expected = """E~BinaryJoinExec(binaryOp=ADD, on=Some(List(tschemaLabel)), 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),false,1800000 milliseconds,true)) @@ -3479,6 +3515,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |-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),false,1800000 milliseconds,true)) |--E~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"oneRemote.*"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) |--E~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"oneRemote.*"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true))""".stripMargin), + // one remote partition, and all ns are assigned a tschema, but tschema label missing from selector and "by" clause Test(query = s"""foo{_ws_="demo",_ns_=~"$ONE_REMOTE"} + on (notATargetSchemaLabel) baz{_ws_="demo",_ns_=~"$ONE_REMOTE"}""", tschemaEnabled = Set("oneRemote1", "oneRemote2"), expected = """E~BinaryJoinExec(binaryOp=ADD, on=Some(List(notATargetSchemaLabel)), 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),false,1800000 milliseconds,true)) @@ -3488,6 +3525,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |-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),false,1800000 milliseconds,true)) |--E~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"oneRemote.*"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) |--E~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"oneRemote.*"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true))""".stripMargin), + // local partition, and all ns are assigned a tschema, but tschema label missing from selector Test(query = s"""foo{_ws_="demo",_ns_=~"$LOCAL"} + on ($TSCHEMA_LABEL) baz{_ws_="demo",_ns_=~"$LOCAL"}""", tschemaEnabled = Set("local1", "local2"), expected = """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,None,None,None,100,false,false,true,Set(),None,Map(filodb-query-exec-aggregate-large-container -> 65536, filodb-query-exec-metadataexec -> 8192),false,3 days,true)) @@ -3513,6 +3551,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=1, chunkMethod=TimeRangeChunkScan(1633913030000,1634172830000), filters=List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,EqualsRegex(local.*)), ColumnFilter(_metric_,Equals(foo))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),false,1800000 milliseconds,true)) |---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(local.*)), ColumnFilter(_metric_,Equals(baz))), colName=None, schema=None) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),false,1800000 milliseconds,true))""".stripMargin), + // one remote partition, and all ns are assigned a tschema; nested join Test(query = s"""foo{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) (baz{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"} - on($TSCHEMA_LABEL) bak{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"})""", tschemaEnabled = Set("oneRemote1", "oneRemote2"), expected = """E~BinaryJoinExec(binaryOp=ADD, on=Some(List(tschemaLabel)), 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),false,1800000 milliseconds,true)) @@ -3526,6 +3565,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |--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),false,1800000 milliseconds,true)) |---E~PromQlRemoteExec(PromQlQueryParams(bak{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) |---E~PromQlRemoteExec(PromQlQueryParams(bak{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true))""".stripMargin), + // one remote partition, and all ns are assigned a tschema; nested aggregation with target-schema label missing from outer "by" clause Test(query = s"""foo{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"} + on (notATargetSchemaLabel) (baz{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"} - on($TSCHEMA_LABEL) bak{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"})""", tschemaEnabled = Set("oneRemote1", "oneRemote2"), expected = """E~BinaryJoinExec(binaryOp=ADD, on=Some(List(notATargetSchemaLabel)), 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),false,1800000 milliseconds,true)) @@ -3539,6 +3579,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |--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),false,1800000 milliseconds,true)) |---E~PromQlRemoteExec(PromQlQueryParams(bak{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) |---E~PromQlRemoteExec(PromQlQueryParams(bak{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true))""".stripMargin), + // one remote partition, and all ns are assigned a tschema; nested aggregation with target-schema label missing from inner "by" clause Test(query = s"""foo{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) (baz{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"} - on (notATargetSchemaLabel) bak{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"})""", tschemaEnabled = Set("oneRemote1", "oneRemote2"), expected = """E~BinaryJoinExec(binaryOp=ADD, on=Some(List(tschemaLabel)), 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),false,1800000 milliseconds,true)) @@ -3552,6 +3593,10 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |--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),false,1800000 milliseconds,true)) |---E~PromQlRemoteExec(PromQlQueryParams(bak{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) |---E~PromQlRemoteExec(PromQlQueryParams(bak{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true))""".stripMargin), + + // ----------------- begin compound agg/join tests --------------------- + + // one remote partition, and all ns are assigned a tschema; nested join Test(query = s"""sum(foo{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) baz{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"}) by ($TSCHEMA_LABEL)""", tschemaEnabled = Set("oneRemote1", "oneRemote2"), expected = """T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(1633913330,300,1634777330)) @@ -3564,6 +3609,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |----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),false,1800000 milliseconds,true)) |-----E~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true)) |-----E~PromQlRemoteExec(PromQlQueryParams(baz{_ws_="demo",_ns_=~"oneRemote.*",tschemaLabel="bar"},1633913330,300,1634777330,None,false), PlannerParams(filodb,None,None,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=remote-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),false,1800000 milliseconds,true))""".stripMargin), + // one remote partition, and all ns are assigned a tschema; nested aggregation Test(query = s"""foo{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"} + on ($TSCHEMA_LABEL) sum(baz{_ws_="demo",_ns_=~"$ONE_REMOTE",$TSCHEMA_LABEL="bar"}) by ($TSCHEMA_LABEL)""", tschemaEnabled = Set("oneRemote1", "oneRemote2"), expected = """E~BinaryJoinExec(binaryOp=ADD, on=Some(List(tschemaLabel)), 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),false,1800000 milliseconds,true)) From 87bebd051cd001cc578d83c23ede5f1c0dfbf9c0 Mon Sep 17 00:00:00 2001 From: Alex Theimer Date: Fri, 12 Jan 2024 11:45:03 -0800 Subject: [PATCH 25/27] additional fixes --- .../queryplanner/MultiPartitionPlanner.scala | 4 +++- .../queryplanner/SingleClusterPlanner.scala | 4 +++- query/src/main/scala/filodb/query/PlanEnums.scala | 2 +- query/src/main/scala/filodb/query/exec/StitchRvsExec.scala | 4 ++-- 4 files changed, 9 insertions(+), 5 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala index f8413e0207..7a8ba27dba 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala @@ -137,7 +137,9 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider stepSecs = 1 ) val newContext = qContext.copy(origQueryParams = newPromQlParams) - materializeForPartition(thisPartitionLp, pa, newContext) + // Materialize this as an instant, raw-data query. + materializeForPartition(thisPartitionLp, pa, newContext, + Some(TimeRange(thisPartitionEndMs, thisPartitionEndMs))) }) PlanResult( Seq( if (execPlans.tail == Seq.empty) execPlans.head diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala index dcc4d991c5..632eed7036 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala @@ -270,7 +270,9 @@ class SingleClusterPlanner(val dataset: Dataset, shardColumns.contains(filter.column) && !filter.filter.isInstanceOf[Equals] } val filterGroups = if (hasNonEqualsShardKeyFilter) { - shardKeyMatcher(rawFilters).map(LogicalPlanUtils.upsertFilters(rawFilters, _)) + val shardKeyFilters = rawFilters.filter(f => dataset.options.nonMetricShardColumns.contains(f.column)) + shardKeyMatcher(shardKeyFilters) + .map(LogicalPlanUtils.upsertFilters(rawFilters, _)) } else { Seq(rawFilters) } diff --git a/query/src/main/scala/filodb/query/PlanEnums.scala b/query/src/main/scala/filodb/query/PlanEnums.scala index 86ee959ea0..93fb876ad5 100644 --- a/query/src/main/scala/filodb/query/PlanEnums.scala +++ b/query/src/main/scala/filodb/query/PlanEnums.scala @@ -124,7 +124,7 @@ object BinaryOperator extends Enum[BinaryOperator] { case object SUB extends MathOperator("-", 4) case object ADD extends MathOperator("+", 4) case object MUL extends MathOperator("*", 5) - case object MOD extends MathOperator("+", 5) + case object MOD extends MathOperator("%", 5) case object DIV extends MathOperator("/", 5) case object POW extends MathOperator("^", 6, true) case object LAND extends SetOperator("and", 2) diff --git a/query/src/main/scala/filodb/query/exec/StitchRvsExec.scala b/query/src/main/scala/filodb/query/exec/StitchRvsExec.scala index 88e960d42d..e85bf0c0ae 100644 --- a/query/src/main/scala/filodb/query/exec/StitchRvsExec.scala +++ b/query/src/main/scala/filodb/query/exec/StitchRvsExec.scala @@ -81,9 +81,9 @@ object StitchRvsExec { // The second condition checks if these values are equal within the tolerable limits and if yes, do not // emit NaN. // TODO: Make the second check and tolerance configurable? - if (minsWithoutNan.tail.isEmpty) { + if (minsWithoutNan.size == 1) { minsWithoutNan.head - } else if (enableApproximatelyEqualCheck && + } else if (minsWithoutNan.size > 1 && enableApproximatelyEqualCheck && minsWithoutNan.map(x => (x.getDouble(1) * weight).toLong / weight).toSet.size == 1) { minsWithoutNan.head } else { From ffb6a984bfbc1e9146a75cd865d166adf078e805 Mon Sep 17 00:00:00 2001 From: Alex Theimer Date: Fri, 12 Jan 2024 12:01:44 -0800 Subject: [PATCH 26/27] update tests --- .../queryplanner/MultiPartitionPlannerSpec.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala index 8c91d4469e..3c6e9f2da9 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala @@ -1670,8 +1670,8 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers with PlanValida |----T~PeriodicSamplesMapper(start=6999000, step=100000, end=7599000, window=Some(600000), functionId=Some(Rate), rawSource=false, offsetMs=None) |-----E~StitchRvsExec() on InProcessPlanDispatcher |------E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=3, chunkMethod=TimeRangeChunkScan(6400000,7599000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=19, chunkMethod=TimeRangeChunkScan(6400000,7599000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=3, chunkMethod=TimeRangeChunkScan(7000000,7599000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=19, chunkMethod=TimeRangeChunkScan(7000000,7599000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) |------E~PromQlRemoteExec(PromQlQueryParams(test{job="app"}[600s],6999,1,6999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,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=10000) on InProcessPlanDispatcher |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(2000,100,10000)) |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) @@ -1702,8 +1702,8 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers with PlanValida |-----T~PeriodicSamplesMapper(start=6999000, step=100000, end=7599000, window=Some(600000), functionId=Some(Rate), rawSource=false, offsetMs=None) |------E~StitchRvsExec() on InProcessPlanDispatcher |-------E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=3, chunkMethod=TimeRangeChunkScan(6400000,7599000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=19, chunkMethod=TimeRangeChunkScan(6400000,7599000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=3, chunkMethod=TimeRangeChunkScan(7000000,7599000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=19, chunkMethod=TimeRangeChunkScan(7000000,7599000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) |-------E~PromQlRemoteExec(PromQlQueryParams(test{job="app"}[600s],6999,1,6999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,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=10000) on InProcessPlanDispatcher |--T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(6999,100,7599)) |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on InProcessPlanDispatcher @@ -1711,8 +1711,8 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers with PlanValida |-----T~PeriodicSamplesMapper(start=6999000, step=100000, end=7599000, window=Some(300000), functionId=Some(Rate), rawSource=false, offsetMs=None) |------E~StitchRvsExec() on InProcessPlanDispatcher |-------E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=11, chunkMethod=TimeRangeChunkScan(6400000,7599000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) - |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=27, chunkMethod=TimeRangeChunkScan(6400000,7599000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=11, chunkMethod=TimeRangeChunkScan(7000000,7599000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) + |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=27, chunkMethod=TimeRangeChunkScan(7000000,7599000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) |-------E~PromQlRemoteExec(PromQlQueryParams(bar{job="app"}[300s],6999,1,6999,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,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=10000) on InProcessPlanDispatcher |-E~BinaryJoinExec(binaryOp=ADD, on=None, ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testActor],raw) |--T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(2000,100,10000)) From 0ffd391f1e195c00e5e6be84037604e30f006d67 Mon Sep 17 00:00:00 2001 From: Alex Theimer Date: Fri, 12 Jan 2024 12:02:38 -0800 Subject: [PATCH 27/27] Revert "DROP: printPairs" This reverts commit 2054a7ff87eefd6acfcf1100f4ec198de3368d2e. --- .../queryplanner/PlannerHierarchySpec.scala | 15 --------------- 1 file changed, 15 deletions(-) diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala index bd38467827..5b2c9a55be 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala @@ -18,9 +18,6 @@ import filodb.prometheus.parse.Parser.Antlr import filodb.query.{BadQueryException, IntervalSelector, LabelCardinality, PlanValidationSpec, RawSeries} import filodb.query.exec._ - -import scala.collection.mutable - // scalastyle:off line.size.limit // scalastyle:off number.of.methods class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationSpec { @@ -593,18 +590,6 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS validatePlan(execPlan, expected) } - class QueryExpectedPairs { - val pairs = new mutable.ArrayBuffer[(String, String)] - def addPair(query: String, plan: ExecPlan): Unit = { - pairs.append((query, plan.printTree().replaceAll("\n", "\n |"))) - } - def printPairs(): Unit = { - for ((query, expected) <- pairs) { - println(s"""(\"\"\"$query\"\"\",\n \"\"\"$expected\"\"\".stripMargin),""") - } - } - } - it("should not pushdown root scalar operation into a RemoteExec's promql when query spans multiple partitions") { val queryExpectedPairs = Seq(