From f63de9116f707b78a82e6ad8ef7327f6121e5260 Mon Sep 17 00:00:00 2001 From: Brian-Yu Date: Thu, 16 Jan 2025 10:08:34 -0800 Subject: [PATCH] Revert "Merge remote-tracking branch 'upstream/develop' into develop" This reverts commit 32c6ba8ad9014828ecfc71a09ae6429d9cc17874, reversing changes made to a4c8efaf46797dac10a3c6601ffa6751420a8a50. --- .github/workflows/scala.yml | 2 +- .../GrpcPlanDispatcher.scala | 5 +- .../filodb.coordinator/NodeClusterActor.scala | 9 +- .../filodb.coordinator/ShardStatus.scala | 24 -- .../client/ClusterOps.scala | 16 -- .../queryplanner/DefaultPlanner.scala | 31 ++- .../queryplanner/MultiPartitionPlanner.scala | 46 ++-- .../filodb/coordinator/ProtoConverters.scala | 4 - .../v2/NewNodeCoordinatorActor.scala | 60 +---- .../filodb.coordinator/ShardMapperSpec.scala | 98 +------- .../queryplanner/LogicalPlanParserSpec.scala | 226 +++--------------- .../MultiPartitionPlannerSpec.scala | 195 +++++++-------- .../queryplanner/PlannerHierarchySpec.scala | 18 +- .../scala/filodb.core/query/RangeVector.scala | 27 ++- grpc/src/main/protobuf/query_service.proto | 1 - .../scala/filodb/http/ClusterApiRoute.scala | 17 +- .../main/scala/filodb/query/LogicalPlan.scala | 31 +-- .../main/scala/filodb/query/PlanEnums.scala | 1 - .../aggregator/CountValuesRowAggregator.scala | 28 +-- .../aggregator/TopBottomKRowAggregator.scala | 47 ++-- .../util/HierarchicalQueryExperience.scala | 137 +++-------- .../HierarchicalQueryExperienceSpec.scala | 47 ++-- 22 files changed, 277 insertions(+), 793 deletions(-) diff --git a/.github/workflows/scala.yml b/.github/workflows/scala.yml index e0d75a4e4b..4855a2a24e 100644 --- a/.github/workflows/scala.yml +++ b/.github/workflows/scala.yml @@ -9,7 +9,7 @@ on: jobs: test: - runs-on: ubuntu-22.04 + runs-on: ubuntu-latest steps: - name: Checkout diff --git a/coordinator/src/main/scala/filodb.coordinator/GrpcPlanDispatcher.scala b/coordinator/src/main/scala/filodb.coordinator/GrpcPlanDispatcher.scala index f78ca28c96..e0a588355f 100644 --- a/coordinator/src/main/scala/filodb.coordinator/GrpcPlanDispatcher.scala +++ b/coordinator/src/main/scala/filodb.coordinator/GrpcPlanDispatcher.scala @@ -1,5 +1,6 @@ package filodb.coordinator +import com.typesafe.scalalogging.StrictLogging import io.grpc.Metadata import io.grpc.stub.{MetadataUtils, StreamObserver} import java.net.InetAddress @@ -28,7 +29,7 @@ object GrpcPlanDispatcher { Runtime.getRuntime.addShutdownHook(new Thread(() => channelMap.values.foreach(_.shutdown()))) } -case class GrpcPlanDispatcher(endpoint: String, requestTimeoutMs: Long) extends PlanDispatcher { +case class GrpcPlanDispatcher(endpoint: String, requestTimeoutMs: Long) extends PlanDispatcher with StrictLogging { val clusterName = InetAddress.getLocalHost().getHostName() @@ -66,6 +67,8 @@ case class GrpcPlanDispatcher(endpoint: String, requestTimeoutMs: Long) extends val genericRemoteExec = plan.execPlan.asInstanceOf[GenericRemoteExec] import filodb.coordinator.ProtoConverters._ val protoPlan = genericRemoteExec.execPlan.toExecPlanContainerProto + logger.debug(s"Query ${plan.execPlan.queryContext.queryId} proto plan size is ${protoPlan.toByteArray.length}B") + logger.debug(s"Query ${plan.execPlan.queryContext.queryId} exec plan ${genericRemoteExec.execPlan.printTree()}") val queryContextProto = genericRemoteExec.execPlan.queryContext.toProto val remoteExecPlan : RemoteExecPlan = RemoteExecPlan.newBuilder() .setExecPlan(protoPlan) diff --git a/coordinator/src/main/scala/filodb.coordinator/NodeClusterActor.scala b/coordinator/src/main/scala/filodb.coordinator/NodeClusterActor.scala index 0aaf370a59..b4df27a496 100644 --- a/coordinator/src/main/scala/filodb.coordinator/NodeClusterActor.scala +++ b/coordinator/src/main/scala/filodb.coordinator/NodeClusterActor.scala @@ -95,20 +95,13 @@ object NodeClusterActor { final case class DatasetUnknown(ref: DatasetRef) extends ErrorResponse final case class BadSchema(message: String) extends ErrorResponse final case class BadData(message: String) extends ErrorResponse - final case class InternalServiceError(message: String) extends ErrorResponse - // Cluster state info commands + // Cluste state info commands // Returns a Seq[DatasetRef] case object ListRegisteredDatasets // Returns CurrentShardSnapshot or DatasetUnknown final case class GetShardMap(ref: DatasetRef) - /** - * @param ref compressed ShardMap information for sending over the wire. - * IMPORTANT: Only works with ClusteringV2 shard assignment strategy. - */ - final case class GetShardMapV2(ref: DatasetRef) - /** Registers sending actor to receive `ShardMapUpdate` whenever it changes. DeathWatch * will be used on the sending actors to watch for updates. On subscribe, will * immediately send back the current state via a `ShardMapUpdate` message. diff --git a/coordinator/src/main/scala/filodb.coordinator/ShardStatus.scala b/coordinator/src/main/scala/filodb.coordinator/ShardStatus.scala index eeb6e803de..77814a91be 100644 --- a/coordinator/src/main/scala/filodb.coordinator/ShardStatus.scala +++ b/coordinator/src/main/scala/filodb.coordinator/ShardStatus.scala @@ -11,30 +11,6 @@ sealed trait ShardAction extends Serializable final case class CurrentShardSnapshot(ref: DatasetRef, map: ShardMapper) extends ShardAction with Response -/** - * Optimized form of the ShardMapper state representation. - * NOTE: It doesn't track the shard status updates from coordinator or Ingestion actors. It is just - * a wrapper which compresses the response of ShardMapper state to reduce network transmission costs. - * - * @param nodeCountInCluster Number of replicas in the filodb cluster - * @param numShards Number of shards in the filodb cluster - * @param k8sHostFormat K8s host format. Valid ONLY for ClusterV2 shard assignment strategy - * @param shardState ByteArray. Each bit of the byte represents the shard status. - * For example: lets say we have 4 shards with following status: - * Seq[ShardStatusAssigned, ShardStatusRecovery, ShardStatusAssigned, ShardStatusAssigned] - * Then the shardState would be an array of single byte whose bit representation is - 1000 0000 - * Explanation - corresponding bit is set to 1 if the shard is assigned, else 0 - */ -final case class ShardMapperV2(nodeCountInCluster: Int, numShards: Int, k8sHostFormat: String, - shardState: Array[Byte]) - -/** - * Response to GetShardMapV2 request. Uses the optimized ShardMapperV2 representation. Only applicable - * for ClusterV2 shard assignment strategy. - * @param map ShardMapperV2 - */ -final case class ShardSnapshot(map: ShardMapperV2) extends ShardAction with Response - /** * Full state of all shards, sent to all ingestion actors. They react by starting/stopping * ingestion for the shards they own or no longer own. The version is expected to be global diff --git a/coordinator/src/main/scala/filodb.coordinator/client/ClusterOps.scala b/coordinator/src/main/scala/filodb.coordinator/client/ClusterOps.scala index 208b55ad6d..e3780f8ba1 100644 --- a/coordinator/src/main/scala/filodb.coordinator/client/ClusterOps.scala +++ b/coordinator/src/main/scala/filodb.coordinator/client/ClusterOps.scala @@ -47,20 +47,4 @@ trait ClusterOps extends ClientBase with StrictLogging { } } } - - /** - * ShardMapperV2 is an optimization of the response size over the ShardMapper and GetShardMap ask call - * @return Some(ShardMapperV2) if the dataset is registered, None if dataset not found - */ - def getShardMapperV2(dataset: DatasetRef, v2Enabled: Boolean, - timeout: FiniteDuration = 30.seconds): Option[ShardMapperV2] = { - require(v2Enabled, s"ClusterV2 ShardAssignment is must for this operation") - val actor = Some(nodeCoordinator) - actor.flatMap { ref => - Client.actorAsk(ref, GetShardMapV2(dataset), timeout) { - case ShardSnapshot(shardMapperV2) => Some(shardMapperV2) - case _ => None - } - } - } } \ No newline at end of file diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/DefaultPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/DefaultPlanner.scala index 3eb14d830f..48e161b047 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/DefaultPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/DefaultPlanner.scala @@ -18,7 +18,7 @@ import filodb.query.LogicalPlan._ import filodb.query.exec._ import filodb.query.exec.InternalRangeFunction.Last -//scalastyle:off file.size.limit + /** * Intermediate Plan Result includes the exec plan(s) along with any state to be passed up the * plan building call tree during query planning. @@ -122,8 +122,10 @@ trait DefaultPlanner { } else lp val series = walkLogicalPlanTree(logicalPlanWithoutBucket.series, qContext, forceInProcess) - // the series is raw and supports raw export, it's going to yield an iterator - val rawSource = logicalPlanWithoutBucket.series.isRaw + val rawSource = logicalPlanWithoutBucket.series.isRaw && (logicalPlanWithoutBucket.series match { + case r: RawSeries => !r.supportsRemoteDataCall + case _ => true + }) // the series is raw and supports raw export, its going to yield an iterator /* Last function is used to get the latest value in the window for absent_over_time If no data is present AbsentFunctionMapper will return range vector with value 1 */ @@ -201,7 +203,10 @@ trait DefaultPlanner { } else (None, None, lp) val rawSeries = walkLogicalPlanTree(lpWithoutBucket.rawSeries, qContext, forceInProcess) - val rawSource = lpWithoutBucket.rawSeries.isRaw + val rawSource = lpWithoutBucket.rawSeries.isRaw && (lpWithoutBucket.rawSeries match { + case r: RawSeries => !r.supportsRemoteDataCall + case _ => true + }) rawSeries.plans.foreach(_.addRangeVectorTransformer(PeriodicSamplesMapper(lp.startMs, lp.stepMs, lp.endMs, window = None, functionId = None, stepMultipleNotationUsed = false, funcParams = Nil, @@ -229,17 +234,11 @@ trait DefaultPlanner { lp: ApplyMiscellaneousFunction, forceInProcess: Boolean = false): PlanResult = { val vectors = walkLogicalPlanTree(lp.vectors, qContext, forceInProcess) - if (lp.function == MiscellaneousFunctionId.OptimizeWithAgg) { - // Optimize with aggregation is a no-op, doing no transformation. It must pass through - // the execution plan to apply optimization logic correctly during aggregation. - vectors - } else { - if (lp.function == MiscellaneousFunctionId.HistToPromVectors) - vectors.plans.foreach(_.addRangeVectorTransformer(HistToPromSeriesMapper(schemas.part))) - else - vectors.plans.foreach(_.addRangeVectorTransformer(MiscellaneousFunctionMapper(lp.function, lp.stringArgs))) - vectors - } + if (lp.function == MiscellaneousFunctionId.HistToPromVectors) + vectors.plans.foreach(_.addRangeVectorTransformer(HistToPromSeriesMapper(schemas.part))) + else + vectors.plans.foreach(_.addRangeVectorTransformer(MiscellaneousFunctionMapper(lp.function, lp.stringArgs))) + vectors } def materializeApplyInstantFunctionRaw(qContext: QueryContext, @@ -827,7 +826,7 @@ object PlannerUtil extends StrictLogging { rewritePlanWithRemoteRawExport(_, rangeSelector, additionalLookbackMs).asInstanceOf[FunctionArgsPlan]), series = rewritePlanWithRemoteRawExport(lp.series, rangeSelector, additionalLookbackMs) .asInstanceOf[RawSeriesLikePlan]) - // won't bother rewriting and adjusting the start and end for metadata calls + // wont bother rewriting and adjusting the start and end for metadata calls case lp: MetadataQueryPlan => lp case lp: TsCardinalities => lp } diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala index 161f1f2d6e..877efac863 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala @@ -118,23 +118,23 @@ class MultiPartitionPlanner(val partitionLocationProvider: PartitionLocationProv // MultiPartitionPlanner has capability to stitch across time partitions, however, the logic is mostly broken // and not well tested. The logic below would not work well for any kind of subquery since their actual // start and ends are different from the start/end parameter of the query context. If we are to implement - // stitching across time, we need to pass proper parameters to getPartitions() call + // stitching across time, we need to to pass proper parameters to getPartitions() call if (forceInProcess) { - // If InProcess is required, we will rely on the DefaultPlanner's implementation as the expectation is that the + // If inprocess is required, we will rely on the DefaultPlanner's implementation as the expectation is that the // raw series is doing a remote call to get all the data. logicalPlan match { - case lp: RawSeries if lp.supportsRemoteDataCall => + case lp: RawSeries if lp.supportsRemoteDataCall=> val params = qContext.origQueryParams.asInstanceOf[PromQlQueryParams] val rs = lp.rangeSelector.asInstanceOf[IntervalSelector] val (rawExportStart, rawExportEnd) = (rs.from - lp.offsetMs.getOrElse(0L) - lp.lookbackMs.getOrElse(0L), rs.to - lp.offsetMs.getOrElse(0L)) - val partitions = getPartitions(lp, params) - assert(partitions.nonEmpty, s"Unexpected to see partitions empty for logicalPlan=$lp and param=$params") + val partition = getPartitions(lp, params) + assert(partition.nonEmpty, s"Unexpected to see partitions empty for logicalPlan=$lp and param=$params") // Raw export from both involved partitions for the entire time duration as the shard-key migration // is not guaranteed to happen exactly at the time of split - val execPlans = partitions.map(pa => { + val execPlans = partition.map(pa => { val (thisPartitionStartMs, thisPartitionEndMs) = (rawExportStart, rawExportEnd) val timeRangeOverride = TimeRange(thisPartitionEndMs, thisPartitionEndMs) val totalOffsetThisPartitionMs = thisPartitionEndMs - thisPartitionStartMs @@ -155,7 +155,7 @@ class MultiPartitionPlanner(val partitionLocationProvider: PartitionLocationProv , inProcessPlanDispatcher, None, execPlans.sortWith((x, _) => !x.isInstanceOf[PromQlRemoteExec]), enableApproximatelyEqualCheck = queryConfig.routingConfig.enableApproximatelyEqualCheckInStitch) - } + } ) ) case _ : LogicalPlan => super.defaultWalkLogicalPlanTree(logicalPlan, qContext, forceInProcess) @@ -191,7 +191,8 @@ class MultiPartitionPlanner(val partitionLocationProvider: PartitionLocationProv generateRemoteExecParams(qContext, startMs, endMs) } // Single partition but remote, send the entire plan remotely - if (isPartitionEnabledForGrpc(partitionName, grpcEndpoint)) { + if (grpcEndpoint.isDefined && !(queryConfig.grpcPartitionsDenyList.contains("*") || + queryConfig.grpcPartitionsDenyList.contains(partitionName.toLowerCase))) { val endpoint = grpcEndpoint.get val channel = channels.getOrElseUpdate(endpoint, GrpcCommonUtils.buildChannelFromEndpoint(endpoint)) PromQLGrpcRemoteExec(channel, remoteHttpTimeoutMs, remoteContext, inProcessPlanDispatcher, @@ -387,17 +388,6 @@ class MultiPartitionPlanner(val partitionLocationProvider: PartitionLocationProv PlanResult(execPlan:: Nil) } - /** - * @param partitionName partition name - * @param grpcEndPoint grpc endpoint - * @return true if partition is enabled with an grpc endpoint - */ - private def isPartitionEnabledForGrpc(partitionName: String, grpcEndPoint: Option[String]): Boolean = { - grpcEndPoint.isDefined && - !(queryConfig.grpcPartitionsDenyList.contains("*") || - queryConfig.grpcPartitionsDenyList.contains(partitionName.toLowerCase)) - } - /** * If the argument partition is local, materialize the LogicalPlan with the local planner. * Otherwise, create a PromQlRemoteExec. @@ -425,7 +415,9 @@ class MultiPartitionPlanner(val partitionLocationProvider: PartitionLocationProv localPartitionPlanner.materialize(lpWithUpdatedTime, qContextWithOverride) } else { val ctx = generateRemoteExecParams(qContextWithOverride, timeRange.startMs, timeRange.endMs) - if (isPartitionEnabledForGrpc(partitionName, grpcEndpoint)) { + if (grpcEndpoint.isDefined && + !(queryConfig.grpcPartitionsDenyList.contains("*") || + queryConfig.grpcPartitionsDenyList.contains(partitionName.toLowerCase))) { val channel = channels.getOrElseUpdate(grpcEndpoint.get, GrpcCommonUtils.buildChannelFromEndpoint(grpcEndpoint.get)) PromQLGrpcRemoteExec(channel, remoteHttpTimeoutMs, ctx, inProcessPlanDispatcher, @@ -594,10 +586,10 @@ class MultiPartitionPlanner(val partitionLocationProvider: PartitionLocationProv val assignmentRanges = getAssignmentQueryRanges(partitions, timeRange, lookbackMs = lookbackMs, offsetMs = offsetMs, stepMsOpt = stepMsOpt) val execPlans = if (assignmentRanges.isEmpty) { - // Assignment ranges empty means we can't run this query fully on one partition and needs - // remote raw export. Check if the total time of raw export is within the limits, if not return Empty result. + // Assignment ranges empty means we cant run this query fully on one partition and needs + // remote raw export Check if the total time of raw export is within the limits, if not return Empty result // While it may seem we don't tune the lookback of the leaf raw queries to exactly what we need from each - // partition, in reality it doesn't matter as despite a longer lookback, the actual data exported will be at most + // partition, in reality it doesnt matter as despite a longer lookback, the actual data exported will be at most // what that partition contains. val (startTime, endTime) = (qParams.startSecs, qParams.endSecs) val totalExpectedRawExport = (endTime - startTime) + lookbackMs + offsetMs.max @@ -617,8 +609,7 @@ class MultiPartitionPlanner(val partitionLocationProvider: PartitionLocationProv } Seq(EmptyResultExec(qContext, dataset.ref, inProcessPlanDispatcher)) } - } - else { + } else { // materialize a plan for each range/assignment pair val (_, execPlans) = assignmentRanges.foldLeft( (None: Option[(PartitionAssignment, TimeRange)], ListBuffer.empty[ExecPlan])) { @@ -661,13 +652,14 @@ class MultiPartitionPlanner(val partitionLocationProvider: PartitionLocationProv // leaf logical plan) with supportsRemoteDataCall = true figure out if this range can entirely be selected // from partition p1 or p2 // + // Do not perform raw exports if the export is beyond a certain value for example // foo{}[10d] or foo[2d] offset 8d both will export 10 days of raw data which might cause heap pressure // and OOMs. The max cross partition raw export config can control such queries from bring the process // down but simpler queries with few minutes or even hour or two of lookback/offset will continue to work // seamlessly with no data gaps - // Note that at the moment, while planning, we only can look at what's the max time range we can support. - // We still don't have capabilities to check the expected number of timeseries scanned or bytes scanned + // Note that at the moment, while planning, we only can look at whats the max time range we can support. + // We still dont have capabilities to check the expected number of timeseries scanned or bytes scanned // and adding capabilities to give up a "part" of query execution if the runtime number of bytes of ts // scanned goes high isn't available. To start with the time range scanned as a static configuration will // be good enough and can be enhanced in future as required. diff --git a/coordinator/src/main/scala/filodb/coordinator/ProtoConverters.scala b/coordinator/src/main/scala/filodb/coordinator/ProtoConverters.scala index 6334cc6839..5fee8cab4d 100644 --- a/coordinator/src/main/scala/filodb/coordinator/ProtoConverters.scala +++ b/coordinator/src/main/scala/filodb/coordinator/ProtoConverters.scala @@ -1075,8 +1075,6 @@ object ProtoConverters { case filodb.query.MiscellaneousFunctionId.LabelJoin => GrpcMultiPartitionQueryService.MiscellaneousFunctionId.LABEL_JOIN case filodb.query.MiscellaneousFunctionId.HistToPromVectors => GrpcMultiPartitionQueryService.MiscellaneousFunctionId.HIST_TO_PROM_VECTORS - case filodb.query.MiscellaneousFunctionId.OptimizeWithAgg => - GrpcMultiPartitionQueryService.MiscellaneousFunctionId.OPTIMIZE_WITH_AGG } function } @@ -1089,8 +1087,6 @@ object ProtoConverters { case GrpcMultiPartitionQueryService.MiscellaneousFunctionId.LABEL_JOIN => filodb.query.MiscellaneousFunctionId.LabelJoin case GrpcMultiPartitionQueryService.MiscellaneousFunctionId.HIST_TO_PROM_VECTORS => filodb.query.MiscellaneousFunctionId.HistToPromVectors - case GrpcMultiPartitionQueryService.MiscellaneousFunctionId.OPTIMIZE_WITH_AGG => - filodb.query.MiscellaneousFunctionId.OptimizeWithAgg case GrpcMultiPartitionQueryService.MiscellaneousFunctionId.UNRECOGNIZED => throw new IllegalArgumentException(s"Unrecognized MiscellaneousFunctionId ${f}") } diff --git a/coordinator/src/main/scala/filodb/coordinator/v2/NewNodeCoordinatorActor.scala b/coordinator/src/main/scala/filodb/coordinator/v2/NewNodeCoordinatorActor.scala index 120d5d045b..6f93038b6e 100644 --- a/coordinator/src/main/scala/filodb/coordinator/v2/NewNodeCoordinatorActor.scala +++ b/coordinator/src/main/scala/filodb/coordinator/v2/NewNodeCoordinatorActor.scala @@ -20,6 +20,7 @@ import filodb.query.QueryCommand final case class GetShardMapScatter(ref: DatasetRef) case object LocalShardsHealthRequest case class DatasetShardHealth(dataset: DatasetRef, shard: Int, status: ShardStatus) +case class LocalShardsHealthResponse(shardStatus: Seq[DatasetShardHealth]) object NewNodeCoordinatorActor { @@ -29,36 +30,6 @@ object NewNodeCoordinatorActor { clusterDiscovery: FiloDbClusterDiscovery, settings: FilodbSettings): Props = Props(new NewNodeCoordinatorActor(memStore, clusterDiscovery, settings)) - - - /** - * Converts a ShardMapper.statuses to a bitmap representation where the bit is set to: - * - 1, if ShardStatus == ShardStatusActive - * - 0, any other ShardStatus like ShardStatusAssigned, ShardStatusRecovery etc. - * WHY this is the case ? This is because, the QueryActor is can only execute the query on the active shards. - * - * NOTE: bitmap is byte aligned. So extra bits are padded with 0. - * - * EXAMPLE - Following are some example of shards with statuses and their bit representation as below: - * Status | BitMap Representation | Hex Representation - * --------------------------------------------------------------------------------------------------------------- - * Assigned, Active, Recovery, Error | 0100 0000 | 0x40 - * Active, Active, Active, Active | 1111 0000 | 0xF0 - * Error, Active, Active, Error, Active, Active | 0110 1100 | 0x6C - * - * @param shardMapper ShardMapper object which stores the bitmap representation - * @return A byte array where each byte represents 8 shards and the bit is set to 1 if the shard is active. Extra bits - * are padded with 0. - */ - def shardMapperBitMapRepresentation(shardMapper: ShardMapper) : Array[Byte] = { - val byteArray = new Array[Byte]((shardMapper.statuses.length + 7) / 8) - for (i <- shardMapper.statuses.indices) { - if (shardMapper.statuses(i) == ShardStatusActive) { - byteArray(i / 8) = (byteArray(i / 8) | (1 << (7 - (i % 8)))).toByte - } - } - byteArray - } } private[filodb] final class NewNodeCoordinatorActor(memStore: TimeSeriesStore, @@ -201,7 +172,6 @@ private[filodb] final class NewNodeCoordinatorActor(memStore: TimeSeriesStore, withQueryActor(originator, dataset) { _.tell(QueryActor.ThrowException(dataset), originator) } } - // scalastyle:off method.length def shardManagementHandlers: Receive = LoggingReceive { // sent by ingestion actors when shard status changes case ev: ShardEvent => try { @@ -216,32 +186,6 @@ private[filodb] final class NewNodeCoordinatorActor(memStore: TimeSeriesStore, sender() ! CurrentShardSnapshot(g.ref, clusterDiscovery.shardMapper(g.ref)) } catch { case e: Exception => logger.error(s"[ClusterV2] Error occurred when processing message $g", e) - // send a response to avoid blocking of akka caller for long time - sender() ! InternalServiceError(s"Exception while executing GetShardMap for dataset: ${g.ref.dataset}") - } - /* - * requested from HTTP API - * What is the trade-off between GetShardMap vs GetShardMapV2 ? - * - * No | Ask Call | Size of Response (256 Shards) | Compute Used - * ------------------------------------------------------------------------------------------------------------- - * 1 | GetShardMap | ~37KB | Baseline - Uses ShardMapper for shard update tracking - * 2 | GetShardMapV2 | 172 Bytes with padding | Additional CPU used to convert ShardMapper to BitMap - * | Will save CPU at the caller by avoiding string parsing - * */ - case g: GetShardMapV2 => - try { - val shardBitMap = NewNodeCoordinatorActor.shardMapperBitMapRepresentation(clusterDiscovery.shardMapper(g.ref)) - val shardMapperV2 = ShardMapperV2( - settings.minNumNodes.get, - ingestionConfigs(g.ref).numShards, - settings.k8sHostFormat.get, - shardBitMap) - sender() ! ShardSnapshot(shardMapperV2) - } catch { case e: Exception => - logger.error(s"[ClusterV2] Error occurred when processing message $g", e) - // send a response to avoid blocking of akka caller for long time - sender() ! InternalServiceError(s"Exception while executing GetShardMapV2 for dataset: ${g.ref.dataset}") } // requested from peer NewNodeCoordActors upon them receiving GetShardMap call @@ -270,8 +214,8 @@ private[filodb] final class NewNodeCoordinatorActor(memStore: TimeSeriesStore, } catch { case e: Exception => logger.error(s"[ClusterV2] Error occurred when processing message LocalShardsHealthRequest", e) } + } - // scalastyle:on method.length def initHandler: Receive = { case InitNewNodeCoordinatorActor => initialize() diff --git a/coordinator/src/test/scala/filodb.coordinator/ShardMapperSpec.scala b/coordinator/src/test/scala/filodb.coordinator/ShardMapperSpec.scala index cadbf9a97f..874e209f31 100644 --- a/coordinator/src/test/scala/filodb.coordinator/ShardMapperSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/ShardMapperSpec.scala @@ -2,7 +2,7 @@ package filodb.coordinator import akka.actor.ActorRef import akka.testkit._ -import filodb.coordinator.v2.NewNodeCoordinatorActor + import filodb.core._ object ShardMapperSpec extends ActorSpecConfig @@ -303,100 +303,4 @@ class ShardMapperSpec extends ActorTest(ShardMapperSpec.getNewSystem) { Seq(0, 3).forall(s => map.coordForShard(s) == newCoord) shouldEqual true assert(coord = newCoord, shards = Seq(0, 3), numAssignedShards = 4, unassignedShards = 28) } - - it ("test bitmap conversion of shard mapper") { - val numShards = 32 - val shardMapper = new ShardMapper(numShards) // default init to ShardStatusUnassigned - var bitRep = NewNodeCoordinatorActor.shardMapperBitMapRepresentation(shardMapper) - bitRep.length shouldEqual 4 - bitRep.forall (x => x == 0x00.toByte) shouldEqual true // no bit should be set at this point - - // move everyone to assigned - shardMapper.registerNode(shardMapper.statuses.indices, TestProbe().ref) - shardMapper.assignedShards.length shouldEqual 32 - - // status updated to assigned but bitmap representation should NOT yet be set to 1 - bitRep = NewNodeCoordinatorActor.shardMapperBitMapRepresentation(shardMapper) - bitRep.length shouldEqual 4 - bitRep.forall (x => x == 0x00.toByte) shouldEqual true // no bit should be set at this point - - // move first 8 and last 8 shards to active - for (i <- 0 to 7) { - shardMapper.updateFromEvent(IngestionStarted(dataset, i, TestProbe().ref)) - } - for (i <- 24 to 31) { - shardMapper.updateFromEvent(IngestionStarted(dataset, i, TestProbe().ref)) - } - shardMapper.activeShards().size shouldEqual 16 - bitRep = NewNodeCoordinatorActor.shardMapperBitMapRepresentation(shardMapper) - // 1111 1111 0000 0000 0000 0000 1111 1111 - bitRep(0) shouldEqual 0xFF.toByte - bitRep(1) shouldEqual 0x00.toByte - bitRep(2) shouldEqual 0x00.toByte - bitRep(3) shouldEqual 0xFF.toByte - } - - it ("test bitmap conversion of shard mapper with 256 shards") { - val shardMapper = new ShardMapper(256) // default init to ShardStatusUnassigned - var bitRep = NewNodeCoordinatorActor.shardMapperBitMapRepresentation(shardMapper) - bitRep.length shouldEqual 32 - bitRep.forall (x => x == 0x00.toByte) shouldEqual true // no bit should be set at this point - shardMapper.registerNode(shardMapper.statuses.indices, TestProbe().ref) - shardMapper.assignedShards.length shouldEqual 256 - - // make all shards active - for (i <- 0 to 255) { - shardMapper.updateFromEvent(IngestionStarted(dataset, i, TestProbe().ref)) - } - // check if all the bits are set correctly - bitRep = NewNodeCoordinatorActor.shardMapperBitMapRepresentation(shardMapper) - bitRep.forall (x => x == 0xFF.toByte) shouldEqual true - - // make some shards in recovery mode - for (i <- 60 to 63) { - shardMapper.updateFromEvent(RecoveryInProgress(dataset, i, TestProbe().ref, 50)) - } - // make some shards in down mode - for (i <- 64 to 67) { - shardMapper.updateFromEvent(ShardDown(dataset, i, TestProbe().ref)) - } - - shardMapper.activeShards().size shouldEqual 248 - shardMapper.notActiveShards().size shouldEqual 8 - bitRep = NewNodeCoordinatorActor.shardMapperBitMapRepresentation(shardMapper) - - // first 60 shards are active - for (i <- 0 to 6) { - bitRep(i) shouldEqual 0xFF.toByte - } - - // shards 56-63 should be 1111 0000 - bitRep(7) shouldEqual 0xF0.toByte - - // shards 64-71 should be 0000 1111 - bitRep(8) shouldEqual 0x0F.toByte - - // last 188 shards are active - for (i <- 9 to 31) { - bitRep(i) shouldEqual 0xFF.toByte - } - } - - it ("test padding is set correctly in non 8 byte aligned number of shards") { - val shardMapper = new ShardMapper(2) // default init to ShardStatusUnassigned - var bitRep = NewNodeCoordinatorActor.shardMapperBitMapRepresentation(shardMapper) - bitRep.length shouldEqual 1 - bitRep.forall (x => x == 0x00.toByte) shouldEqual true // no bit should be set at this point - shardMapper.registerNode(shardMapper.statuses.indices, TestProbe().ref) - shardMapper.assignedShards.length shouldEqual 2 - for (i <- 0 to 1) { - shardMapper.updateFromEvent(IngestionStarted(dataset, i, TestProbe().ref)) - } - bitRep = NewNodeCoordinatorActor.shardMapperBitMapRepresentation(shardMapper) - bitRep(0) shouldEqual 0xC0.toByte // 1100 0000 - padding for last 6 shards - shardMapper.updateFromEvent(ShardDown(dataset, 1, TestProbe().ref)) - - bitRep = NewNodeCoordinatorActor.shardMapperBitMapRepresentation(shardMapper) - bitRep(0) shouldEqual 0x80.toByte // 1000 0000 - } } diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/LogicalPlanParserSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/LogicalPlanParserSpec.scala index 2c93b2379e..7d011421bc 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/LogicalPlanParserSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/LogicalPlanParserSpec.scala @@ -5,9 +5,8 @@ import org.scalatest.matchers.should.Matchers import filodb.prometheus.ast.TimeStepParams import filodb.prometheus.parse.Parser import filodb.query.LogicalPlan.getColumnFilterGroup -import filodb.query.MiscellaneousFunctionId.OptimizeWithAgg import filodb.query.util.{ExcludeAggRule, HierarchicalQueryExperienceParams, IncludeAggRule} -import filodb.query.{Aggregate, ApplyMiscellaneousFunction, BinaryJoin, IntervalSelector, RawSeries, SeriesKeysByFilters} +import filodb.query.{Aggregate, BinaryJoin, IntervalSelector, RawSeries, SeriesKeysByFilters} class LogicalPlanParserSpec extends AnyFunSpec with Matchers { @@ -349,8 +348,8 @@ class LogicalPlanParserSpec extends AnyFunSpec with Matchers { // CASE 1 - BinaryJoin (lhs = Aggregate, rhs = Aggregate) - Both lhs and rhs should be updated val binaryJoinAggregationBothOptimization = "sum(metric1:::agg{aggTag=\"app\"}) + sum(metric2:::agg{aggTag=\"app\"})" var lp = Parser.queryRangeToLogicalPlan(binaryJoinAggregationBothOptimization, t) - val includeAggRule = IncludeAggRule(nextLevelAggregatedMetricSuffix, nextLevelAggregationTags, "2") - val includeParams = HierarchicalQueryExperienceParams(":::", Map("agg" -> Set(includeAggRule))) + val includeAggRule = IncludeAggRule(nextLevelAggregatedMetricSuffix, nextLevelAggregationTags) + val includeParams = HierarchicalQueryExperienceParams(":::", Map("agg" -> includeAggRule)) var lpUpdated = lp.useHigherLevelAggregatedMetric(includeParams) lpUpdated.isInstanceOf[BinaryJoin] shouldEqual true lpUpdated.asInstanceOf[BinaryJoin].lhs.isInstanceOf[Aggregate] shouldEqual true @@ -396,8 +395,8 @@ class LogicalPlanParserSpec extends AnyFunSpec with Matchers { val t = TimeStepParams(700, 1000, 10000) val nextLevelAggregatedMetricSuffix = "agg_2" val nextLevelAggregationTags = Set("aggTag", "aggTag2", "aggTag3", "aggTag4") - val includeAggRule = IncludeAggRule(nextLevelAggregatedMetricSuffix, nextLevelAggregationTags, "2") - val includeParams = HierarchicalQueryExperienceParams(":::", Map("agg" -> Set(includeAggRule))) + val includeAggRule = IncludeAggRule(nextLevelAggregatedMetricSuffix, nextLevelAggregationTags) + val includeParams = HierarchicalQueryExperienceParams(":::", Map("agg" -> includeAggRule)) // CASE 1 - Aggregate with by clause - should update the metric name as `by` clause labels are part of include tags var query = "sum(rate(my_counter:::agg{aggTag=\"spark\", aggTag2=\"app\"}[5m])) by (aggTag4, aggTag3)" var lp = Parser.queryRangeToLogicalPlan(query, t) @@ -469,8 +468,8 @@ class LogicalPlanParserSpec extends AnyFunSpec with Matchers { val t = TimeStepParams(700, 1000, 10000) val nextLevelAggregatedMetricSuffix = "agg_2" val nextLevelAggregationExcludeTags = Set("excludeAggTag", "excludeAggTag2") - val excludeAggRule = ExcludeAggRule(nextLevelAggregatedMetricSuffix, nextLevelAggregationExcludeTags, "2") - val excludeParams = HierarchicalQueryExperienceParams(":::", Map("agg" -> Set(excludeAggRule))) + val excludeAggRule = ExcludeAggRule(nextLevelAggregatedMetricSuffix, nextLevelAggregationExcludeTags) + val excludeParams = HierarchicalQueryExperienceParams(":::", Map("agg" -> excludeAggRule)) // CASE 1 - should update the metric name as `by` clause labels are not part of exclude tags var query = "sum(rate(my_counter:::agg{job=\"spark\", application=\"app\"}[5m])) by (host)" var lp = Parser.queryRangeToLogicalPlan(query, t) @@ -551,8 +550,8 @@ class LogicalPlanParserSpec extends AnyFunSpec with Matchers { val t = TimeStepParams(700, 1000, 10000) val nextLevelAggregatedMetricSuffix = "agg_2" val nextLevelAggregationExcludeTags = Set("excludeAggTag", "excludeAggTag2") - val excludeAggRule = ExcludeAggRule(nextLevelAggregatedMetricSuffix, nextLevelAggregationExcludeTags, "2") - val excludeParams = HierarchicalQueryExperienceParams(":::", Map("agg" -> Set(excludeAggRule))) + val excludeAggRule = ExcludeAggRule(nextLevelAggregatedMetricSuffix, nextLevelAggregationExcludeTags) + val excludeParams = HierarchicalQueryExperienceParams(":::", Map("agg" -> excludeAggRule)) // CASE 1 - should update since the exclude tags are subset of the without clause labels var query = "sum(rate(my_counter:::agg{job=\"spark\", application=\"app\"}[5m])) without (excludeAggTag2, excludeAggTag)" var lp = Parser.queryRangeToLogicalPlan(query, t) @@ -642,8 +641,8 @@ class LogicalPlanParserSpec extends AnyFunSpec with Matchers { val t = TimeStepParams(700, 1000, 10000) val nextLevelAggregatedMetricSuffix = "agg_2" val nextLevelAggregationTags = Set("job", "application", "instance", "version") - val includeAggRule = IncludeAggRule(nextLevelAggregatedMetricSuffix, nextLevelAggregationTags, "2") - val includeParams = HierarchicalQueryExperienceParams(":::", Map("agg" -> Set(includeAggRule))) + val includeAggRule = IncludeAggRule(nextLevelAggregatedMetricSuffix, nextLevelAggregationTags) + val includeParams = HierarchicalQueryExperienceParams(":::", Map("agg" -> includeAggRule)) // All the cases should not be updated since without clause with include tags is not supported as of now var query = "sum(rate(my_counter:::agg{job=\"spark\", application=\"app\"}[5m])) without (version, instance)" var lp = Parser.queryRangeToLogicalPlan(query, t) @@ -702,8 +701,8 @@ class LogicalPlanParserSpec extends AnyFunSpec with Matchers { val t = TimeStepParams(700, 1000, 10000) val nextLevelAggregatedMetricSuffix = "agg_2" val nextLevelAggregationTags = Set("job", "application", "instance", "version") - val includeAggRule = IncludeAggRule(nextLevelAggregatedMetricSuffix, nextLevelAggregationTags, "2") - val includeParams = HierarchicalQueryExperienceParams(":::", Map("agg" -> Set(includeAggRule))) + val includeAggRule = IncludeAggRule(nextLevelAggregatedMetricSuffix, nextLevelAggregationTags) + val includeParams = HierarchicalQueryExperienceParams(":::", Map("agg" -> includeAggRule)) // CASE 1: Raw queries lp should not be updated directly var query = "my_counter:::agg{job=\"spark\", application=\"app\"}[5m]" var lp = Parser.queryToLogicalPlan(query, t.start, t.step) @@ -729,8 +728,8 @@ class LogicalPlanParserSpec extends AnyFunSpec with Matchers { val t = TimeStepParams(700, 1000, 10000) val nextLevelAggregatedMetricSuffix = "agg_2" val nextLevelAggregationTags = Set("job", "application", "instance", "version") - val includeAggRule = IncludeAggRule(nextLevelAggregatedMetricSuffix, nextLevelAggregationTags, "2") - val includeParams = HierarchicalQueryExperienceParams(":::", Map("agg" -> Set(includeAggRule))) + val includeAggRule = IncludeAggRule(nextLevelAggregatedMetricSuffix, nextLevelAggregationTags) + val includeParams = HierarchicalQueryExperienceParams(":::", Map("agg" -> includeAggRule)) // CASE 1: count aggregate should not be allowed var query = "count(my_gauge:::agg{job=\"spark\", application=\"app\"})" var lp = Parser.queryToLogicalPlan(query, t.start, t.step) @@ -769,8 +768,8 @@ class LogicalPlanParserSpec extends AnyFunSpec with Matchers { // with includeTags var nextLevelAggregationTags = Set("aggTag1", "aggTag2", "aggTag3") - val includeAggRule = IncludeAggRule(nextLevelAggregatedMetricSuffix, nextLevelAggregationTags, "2") - val includeParams = HierarchicalQueryExperienceParams(":::", Map("agg" -> Set(includeAggRule))) + val includeAggRule = IncludeAggRule(nextLevelAggregatedMetricSuffix, nextLevelAggregationTags) + val includeParams = HierarchicalQueryExperienceParams(":::", Map("agg" -> includeAggRule)) // CASE 1: Should update the metric name since aggTag1/2 is part of include tags and aggTag4 is a .* regex var query = "sum(sum(my_counter:::agg{aggTag1=\"spark\", aggTag2=\"app\", aggTag4=~\".*\"}) by (aggTag1, aggTag2))" @@ -793,8 +792,8 @@ class LogicalPlanParserSpec extends AnyFunSpec with Matchers { // with excludeTags nextLevelAggregationTags = Set("excludeAggTag1", "excludeAggTag2") - val excludeAggRule = ExcludeAggRule(nextLevelAggregatedMetricSuffix, nextLevelAggregationTags, "2") - val excludeParams = HierarchicalQueryExperienceParams(":::", Map("agg" -> Set(excludeAggRule))) + val excludeAggRule = ExcludeAggRule(nextLevelAggregatedMetricSuffix, nextLevelAggregationTags) + val excludeParams = HierarchicalQueryExperienceParams(":::", Map("agg" -> excludeAggRule)) // CASE 3: should update since excludeTags are only used with .* regex query = "sum by (aggTag1, aggTag2) (sum by (aggTag1, aggTag2) (my_gauge:::agg{aggTag1=\"a\", aggTag2=\"b\", excludeAggTag2=~\".*\"}))" @@ -821,8 +820,8 @@ class LogicalPlanParserSpec extends AnyFunSpec with Matchers { val t = TimeStepParams(700, 1000, 10000) val nextLevelAggregatedMetricSuffix = "agg_2" var nextLevelAggregationTags = Set("aggTag1", "aggTag2", "aggTag3") - val includeAggRule = IncludeAggRule(nextLevelAggregatedMetricSuffix, nextLevelAggregationTags, "2") - val includeParams = HierarchicalQueryExperienceParams(":::", Map("agg" -> Set(includeAggRule))) + val includeAggRule = IncludeAggRule(nextLevelAggregatedMetricSuffix, nextLevelAggregationTags) + val includeParams = HierarchicalQueryExperienceParams(":::", Map("agg" -> includeAggRule)) // CASE 1: should update the metric name as `by` clause labels are part of include tags var query = "sum(sum(my_counter:::agg{aggTag1=\"spark\", aggTag2=\"app\"}) by (aggTag1, aggTag2, aggTag3))" var lp = Parser.queryToLogicalPlan(query, t.start, t.step) @@ -852,8 +851,8 @@ class LogicalPlanParserSpec extends AnyFunSpec with Matchers { ) // using excludeTags nextLevelAggregationTags = Set("excludeAggTag1", "excludeAggTag2") - val excludeAggRule = ExcludeAggRule(nextLevelAggregatedMetricSuffix, nextLevelAggregationTags, "2") - val excludeParams = HierarchicalQueryExperienceParams(":::", Map("agg" -> Set(excludeAggRule))) + val excludeAggRule = ExcludeAggRule(nextLevelAggregatedMetricSuffix, nextLevelAggregationTags) + val excludeParams = HierarchicalQueryExperienceParams(":::", Map("agg" -> excludeAggRule)) // CASE 4: should update since excludeTags are not used query = "sum by (aggTag1, aggTag2) (sum by (aggTag1, aggTag2) (my_gauge:::agg{aggTag1=\"a\", aggTag2=\"b\"}))" lp = Parser.queryRangeToLogicalPlan(query, t) @@ -877,11 +876,11 @@ class LogicalPlanParserSpec extends AnyFunSpec with Matchers { it("LogicalPlan update for BinaryJoin with multiple agg rules and suffixes") { // common parameters val t = TimeStepParams(700, 1000, 10000) - val includeAggRule = IncludeAggRule("suffix1_2", Set("includeTag1", "includeTag2", "includeTag3"), "2") - val excludeAggRule = ExcludeAggRule("suffix2_2", Set("excludeTag1", "excludeTag2"), "2") + val includeAggRule = IncludeAggRule("suffix1_2", Set("includeTag1", "includeTag2", "includeTag3")) + val excludeAggRule = ExcludeAggRule("suffix2_2", Set("excludeTag1", "excludeTag2")) // Query with multiple agg rules and suffixes val includeParams = HierarchicalQueryExperienceParams(":::", - Map("suffix1" -> Set(includeAggRule), "suffix2" -> Set(excludeAggRule))) + Map("suffix1" -> includeAggRule, "suffix2" -> excludeAggRule)) // CASE 1 - should update - simple binary join with two different aggregated metrics and suffixes, both of which are satisfying the next level aggregation metric constraints var query = "sum(my_gauge:::suffix1{includeTag1=\"spark\", includeTag2=\"filodb\"}) + sum(your_gauge:::suffix2{notExcludeTag1=\"spark\", notExcludeTag2=\"filodb\"})" var lp = Parser.queryRangeToLogicalPlan(query, t) @@ -923,11 +922,11 @@ class LogicalPlanParserSpec extends AnyFunSpec with Matchers { it("LogicalPlan update for BinaryJoin with multiple agg rules and suffixes with by clauses") { // common parameters val t = TimeStepParams(700, 1000, 10000) - val includeAggRule = IncludeAggRule("suffix1_2", Set("includeTag1", "includeTag2", "includeTag3"), "2") - val excludeAggRule = ExcludeAggRule("suffix2_2", Set("excludeTag1", "excludeTag2"), "2") + val includeAggRule = IncludeAggRule("suffix1_2", Set("includeTag1", "includeTag2", "includeTag3")) + val excludeAggRule = ExcludeAggRule("suffix2_2", Set("excludeTag1", "excludeTag2")) // Query with multiple agg rules and suffixes val includeParams = HierarchicalQueryExperienceParams(":::", - Map("suffix1" -> Set(includeAggRule), "suffix2" -> Set(excludeAggRule))) + Map("suffix1" -> includeAggRule, "suffix2" -> excludeAggRule)) // CASE 1 - should update - both lhs and rhs are satisfying the next level aggregation metric constraints var query = "sum(my_gauge:::suffix1{includeTag1=\"spark\", includeTag2=\"filodb\"}) by (includeTag3, includeTag1) + sum(your_gauge:::suffix2{notExcludeTag1=\"spark\", notExcludeTag2=\"filodb\"}) by (notExcludeTag1)" var lp = Parser.queryRangeToLogicalPlan(query, t) @@ -969,11 +968,11 @@ class LogicalPlanParserSpec extends AnyFunSpec with Matchers { it("LogicalPlan update for BinaryJoin with multiple agg rules and suffixes with by and without clauses") { // common parameters val t = TimeStepParams(700, 1000, 10000) - val includeAggRule = IncludeAggRule("suffix1_2", Set("includeTag1", "includeTag2", "includeTag3"), "2") - val excludeAggRule = ExcludeAggRule("suffix2_2", Set("excludeTag1", "excludeTag2"), "2") + val includeAggRule = IncludeAggRule("suffix1_2", Set("includeTag1", "includeTag2", "includeTag3")) + val excludeAggRule = ExcludeAggRule("suffix2_2", Set("excludeTag1", "excludeTag2")) // Query with multiple agg rules and suffixes val includeParams = HierarchicalQueryExperienceParams(":::", - Map("suffix1" -> Set(includeAggRule), "suffix2" -> Set(excludeAggRule))) + Map("suffix1" -> includeAggRule, "suffix2" -> excludeAggRule)) // CASE 1 - should update - both lhs and rhs are satisfying the next level aggregation metric constraints var query = "sum(my_gauge:::suffix1{includeTag1=\"spark\", includeTag2=\"filodb\"}) by (includeTag3, includeTag1) + sum(your_gauge:::suffix2{notExcludeTag1=\"spark\", notExcludeTag2=\"filodb\"}) without (excludeTag1, excludeTag2)" var lp = Parser.queryRangeToLogicalPlan(query, t) @@ -997,11 +996,11 @@ class LogicalPlanParserSpec extends AnyFunSpec with Matchers { it("LogicalPlan should not update when next level aggregation metric suffix is not matching agg rules") { // common parameters val t = TimeStepParams(700, 1000, 10000) - val includeAggRule = IncludeAggRule("suffix1_2", Set("includeTag1", "includeTag2", "includeTag3"), "2") - val excludeAggRule = ExcludeAggRule("suffix2_2", Set("excludeTag1", "excludeTag2"), "2") + val includeAggRule = IncludeAggRule("suffix1_2", Set("includeTag1", "includeTag2", "includeTag3")) + val excludeAggRule = ExcludeAggRule("suffix2_2", Set("excludeTag1", "excludeTag2")) // Query with multiple agg rules and suffixes val includeParams = HierarchicalQueryExperienceParams(":::", - Map("suffix1" -> Set(includeAggRule), "suffix2" -> Set(excludeAggRule))) + Map("suffix1" -> includeAggRule, "suffix2" -> excludeAggRule)) // CASE 1 - should not update - both lhs and rhs metric are not using suffix passed for lp update var query = "sum(my_gauge:::no_rule{includeTag1=\"spark\", includeTag2=\"filodb\"}) by (includeTag3, includeTag1)" var lp = Parser.queryRangeToLogicalPlan(query, t) @@ -1038,159 +1037,4 @@ class LogicalPlanParserSpec extends AnyFunSpec with Matchers { updatedMetricNamesSet.contains("my_gauge:::no_rule").shouldEqual(true) // not updated updatedMetricNamesSet.contains("your_gauge:::no_rule2").shouldEqual(true) // not updated } - - it("should correctly apply optimize_with_agg function to a promql query") { - val timeParamsSec1 = TimeStepParams(1000, 10, 10000) - val query1 = """optimize_with_agg(sum(rate(mns_gmail_authenticate_request_ms{_ws_="acs-icloud",_ns_="mail-notifications",app="mail-notifications",env="prod",_type_="prom-histogram"}[5m])))""" - val lp1 = Parser.queryRangeToLogicalPlan(query1, timeParamsSec1) - lp1.isInstanceOf[ApplyMiscellaneousFunction].shouldEqual(true) - lp1.asInstanceOf[ApplyMiscellaneousFunction].function .shouldEqual(OptimizeWithAgg) - LogicalPlanUtils.getLogicalPlanTreeStringRepresentation(lp1) shouldEqual "ApplyMiscellaneousFunction(Aggregate(PeriodicSeriesWithWindowing(RawSeries)))" - - - val timeParamsSec2 = TimeStepParams(1000, 10, 10000) - val query2 = """optimize_with_agg(sum(my_gauge:::suffix1{includeTag1="spark", includeTag2="filodb"}) by (includeTag3, includeTag1) + sum(your_gauge:::suffix2{notExcludeTag1="spark", notExcludeTag2="filodb"}) without (excludeTag1, excludeTag2))""" - val lp2 = Parser.queryRangeToLogicalPlan(query2, timeParamsSec2) - lp2.isInstanceOf[ApplyMiscellaneousFunction].shouldEqual(true) - lp2.asInstanceOf[ApplyMiscellaneousFunction].function .shouldEqual(OptimizeWithAgg) - LogicalPlanUtils.getLogicalPlanTreeStringRepresentation(lp2) shouldEqual "ApplyMiscellaneousFunction(BinaryJoin(Aggregate(PeriodicSeries(RawSeries)),Aggregate(PeriodicSeries(RawSeries))))" - - val timeParamsSec3 = TimeStepParams(1000, 10, 10000) - val query3 = """optimize_with_agg(sum(my_gauge:::no_rule{includeTag1="spark", includeTag2="filodb"}) by (includeTag3, includeTag1) + sum(your_gauge:::suffix2{notExcludeTag1="spark", notExcludeTag2="filodb"}) by (notExcludeTag1))""" - val lp3 = Parser.queryRangeToLogicalPlan(query3, timeParamsSec3) - lp3.isInstanceOf[ApplyMiscellaneousFunction].shouldEqual(true) - lp3.asInstanceOf[ApplyMiscellaneousFunction].function .shouldEqual(OptimizeWithAgg) - LogicalPlanUtils.getLogicalPlanTreeStringRepresentation(lp3) shouldEqual "ApplyMiscellaneousFunction(BinaryJoin(Aggregate(PeriodicSeries(RawSeries)),Aggregate(PeriodicSeries(RawSeries))))" - } - - it("Logical plan should update to use the aggregated metric from raw metric") { - val t = TimeStepParams(700, 1000, 10000) - // CASE 1 - BinaryJoin (lhs = Aggregate, rhs = Aggregate) - Both lhs and rhs should be updated - val binaryJoinAggregationBothOptimization = "optimize_with_agg(sum(rate(metric1{aggTag=\"app\"}[5m])) + sum(rate(metric2{aggTag=\"app\"}[5m])))" - var lp = Parser.queryRangeToLogicalPlan(binaryJoinAggregationBothOptimization, t) - val includeAggRule1 = IncludeAggRule("agg", Set("aggTag", "aggTag2", "aggTag3"), "1") - val includeAggRule2 = IncludeAggRule("agg_2", Set("aggTag", "aggTag2"), "2") - val includeParams = HierarchicalQueryExperienceParams(":::", Map.empty, - Map("metric1" -> Set(includeAggRule1, includeAggRule2), "metric2" -> Set(includeAggRule1, includeAggRule2))) - var lpUpdated = lp.useHigherLevelAggregatedMetric(includeParams) - var filterGroups = getColumnFilterGroup(lpUpdated) - filterGroups.foreach( - filterSet => filterSet.filter( x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] - .endsWith("agg_2").shouldEqual(true) - ) - // CASE 2 - BinaryJoin (lhs = Aggregate, rhs = Aggregate) - rhs should be updated to level-2, while lhs should be updated to level-1 - val binaryJoinAggregationRHSOptimization = "optimize_with_agg(sum(rate(metric1{aggTag3=\"abc\"}[5m])) + sum(rate(metric2{aggTag=\"app\"}[5m])))" - lp = Parser.queryRangeToLogicalPlan(binaryJoinAggregationRHSOptimization, t) - lpUpdated = lp.useHigherLevelAggregatedMetric(includeParams) - filterGroups = getColumnFilterGroup(lpUpdated.asInstanceOf[ApplyMiscellaneousFunction].vectors.asInstanceOf[BinaryJoin].rhs) - filterGroups.foreach( - filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] - .shouldEqual("metric2:::agg_2") - ) - filterGroups = getColumnFilterGroup(lpUpdated.asInstanceOf[ApplyMiscellaneousFunction].vectors.asInstanceOf[BinaryJoin].lhs) - filterGroups.foreach( - filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] - .shouldEqual("metric1:::agg") - ) - // CASE 3 - BinaryJoin (lhs = Aggregate, rhs = Aggregate) - lhs should be updated to level-2 and rhs should not since it is - // not an aggregated metric, even if both the metrics qualify for aggregation - val binaryJoinAggregationLHSOptimization = "optimize_with_agg(sum(rate(metric1{aggTag=\"abc\"}[5m])) + sum(rate(metric2{nonAggTag=\"app\"}[5m])))" - lp = Parser.queryRangeToLogicalPlan(binaryJoinAggregationLHSOptimization, t) - lpUpdated = lp.useHigherLevelAggregatedMetric(includeParams) - filterGroups = getColumnFilterGroup(lpUpdated.asInstanceOf[ApplyMiscellaneousFunction].vectors.asInstanceOf[BinaryJoin].rhs) - filterGroups.foreach( - filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] - .shouldEqual("metric2") - ) - filterGroups = getColumnFilterGroup(lpUpdated.asInstanceOf[ApplyMiscellaneousFunction].vectors.asInstanceOf[BinaryJoin].lhs) - filterGroups.foreach( - filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] - .shouldEqual("metric1:::agg_2") - ) - // CASE 4 - BinaryJoin (lhs = Aggregate, rhs = Aggregate) - // - lhs should not updated to agg as optimize_with_agg is not applied to it. - // - rhs should be updated to level-1 agg as optimize_with_agg is applied to it with the aggTag3 which is only present in level-1 - val binaryJoinAggregationRHSOptimization2 = "sum(rate(metric1{aggTag=\"abc\"}[5m])) + optimize_with_agg(sum(rate(metric2{aggTag3=\"app\"}[5m])))" - lp = Parser.queryRangeToLogicalPlan(binaryJoinAggregationRHSOptimization2, t) - lpUpdated = lp.useHigherLevelAggregatedMetric(includeParams) - filterGroups = getColumnFilterGroup(lpUpdated.asInstanceOf[BinaryJoin].rhs) - filterGroups.foreach( - filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] - .shouldEqual("metric2:::agg") - ) - filterGroups = getColumnFilterGroup(lpUpdated.asInstanceOf[BinaryJoin].lhs) - filterGroups.foreach( - filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] - .shouldEqual("metric1")// raw metric not updated since optimize_with_agg is not applied to it - ) - // CASE 5 - No optimization since optimize_with_agg is not used in lhs or rhs - val binaryJoinAggregationNoOptimization = "sum(rate(metric1{aggTag=\"abc\"}[5m])) + sum(rate(metric2{aggTag3=\"app\"}[5m]))" - lp = Parser.queryRangeToLogicalPlan(binaryJoinAggregationNoOptimization, t) - lpUpdated = lp.useHigherLevelAggregatedMetric(includeParams) - filterGroups = getColumnFilterGroup(lpUpdated.asInstanceOf[BinaryJoin].rhs) - filterGroups.foreach( - filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] - .shouldEqual("metric2") - ) - filterGroups = getColumnFilterGroup(lpUpdated.asInstanceOf[BinaryJoin].lhs) - filterGroups.foreach( - filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] - .shouldEqual("metric1")// raw metric not updated since optimize_with_agg is not applied to it - ) - } - - it ("HQE Logical Plan update with mix mode") { - val t = TimeStepParams(700, 1000, 10000) - // CASE 1 - BinaryJoin (lhs = Aggregated Metric, rhs = raw) - Both lhs and rhs should be updated - val binaryJoinAggregationBothOptimization = "optimize_with_agg(sum(rate(metric1:::agg{aggTag=\"app\"}[5m])) + sum(rate(metric2{aggTag=\"app\"}[5m])))" - var lp = Parser.queryRangeToLogicalPlan(binaryJoinAggregationBothOptimization, t) - val includeAggRule1 = IncludeAggRule("agg", Set("aggTag", "aggTag2", "aggTag3"), "1") - val includeAggRule2 = IncludeAggRule("agg_2", Set("aggTag", "aggTag2"), "2") - val includeParams = HierarchicalQueryExperienceParams(":::", - Map("agg" -> Set(includeAggRule2)), - Map("metric1" -> Set(includeAggRule1, includeAggRule2), "metric2" -> Set(includeAggRule1, includeAggRule2))) - var lpUpdated = lp.useHigherLevelAggregatedMetric(includeParams) - var filterGroups = getColumnFilterGroup(lpUpdated) - filterGroups.foreach( - filterSet => filterSet.filter( x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] - .endsWith("agg_2").shouldEqual(true) - ) - // CASE 2 - BinaryJoin (lhs = Aggregate, rhs = Aggregate) - both lhs and rhs should be updated with optimize_with_agg flag - val binaryJoinAggregationBothOptimization2 = "optimize_with_agg(sum(rate(metric1:::agg{aggTag=\"abc\"}[5m])) + sum(rate(metric2:::agg{aggTag=\"app\"}[5m])))" - lp = Parser.queryRangeToLogicalPlan(binaryJoinAggregationBothOptimization2, t) - lpUpdated = lp.useHigherLevelAggregatedMetric(includeParams) - filterGroups = getColumnFilterGroup(lpUpdated) - filterGroups.foreach( - filterSet => filterSet.filter( x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] - .endsWith("agg_2").shouldEqual(true) - ) - // CASE 3 - BinaryJoin (lhs = Aggregate, rhs = Aggregate) - both lhs and rhs should be updated - val binaryJoinAggregationBothOptimization3 = "optimize_with_agg(sum(rate(metric1{aggTag3=\"abc\"}[5m]))) + sum(rate(metric2:::agg{aggTag=\"app\"}[5m]))" - lp = Parser.queryRangeToLogicalPlan(binaryJoinAggregationBothOptimization3, t) - lpUpdated = lp.useHigherLevelAggregatedMetric(includeParams) - filterGroups = getColumnFilterGroup(lpUpdated.asInstanceOf[BinaryJoin].rhs) - filterGroups.foreach( - filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] - .shouldEqual("metric2:::agg_2") - ) - filterGroups = getColumnFilterGroup(lpUpdated.asInstanceOf[BinaryJoin].lhs) - filterGroups.foreach( - filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] - .shouldEqual("metric1:::agg") - ) - // CASE 4 - Should not update lhs and rhs since they are using non included tags - val binaryJoinAggregationNoOptimization = "optimize_with_agg(sum(rate(metric1{nonAggTag3=\"abc\"}[5m]))) + sum(rate(metric2:::agg{nonAggTag=\"app\"}[5m]))" - lp = Parser.queryRangeToLogicalPlan(binaryJoinAggregationNoOptimization, t) - lpUpdated = lp.useHigherLevelAggregatedMetric(includeParams) - filterGroups = getColumnFilterGroup(lpUpdated.asInstanceOf[BinaryJoin].rhs) - filterGroups.foreach( - filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] - .shouldEqual("metric2:::agg") - ) - filterGroups = getColumnFilterGroup(lpUpdated.asInstanceOf[BinaryJoin].lhs) - filterGroups.foreach( - filterSet => filterSet.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] - .shouldEqual("metric1") - ) - } } diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala index 0c8bf6dd39..f40712935b 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala @@ -82,8 +82,8 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers with PlanValida def twoPartitions(timeRange: TimeRange): List[PartitionAssignment] = List( PartitionAssignment("remote", "remote-url", TimeRange(startSeconds * 1000 - lookbackMs, - localPartitionStart * 1000 - 1), Some("grpcEndpoint1")), PartitionAssignment("remote2", "remote-url2", - TimeRange(localPartitionStart * 1000, endSeconds * 1000), Some("grpcEndpoint2"))) + localPartitionStart * 1000 - 1)), PartitionAssignment("remote2", "remote-url2", + TimeRange(localPartitionStart * 1000, endSeconds * 1000))) val partitionLocationProvider = new PartitionLocationProvider { override def getPartitions(routingKey: Map[String, String], timeRange: TimeRange): List[PartitionAssignment] = { @@ -101,13 +101,13 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers with PlanValida val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams, plannerParams = PlannerParams(processMultiPartition = true))) - 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),RoutingConfig(true,3 days,true,300000),CachingConfig(true,2048))) - |-E~PromQLGrpcRemoteExec(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,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), queryEndpoint=grpcEndpoint1.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),RoutingConfig(true,3 days,true,300000),CachingConfig(true,2048))) - |-T~PeriodicSamplesMapper(start=3000000, step=100000, end=3599000, window=None, functionId=None, rawSource=true, 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),RoutingConfig(true,3 days,true,300000),CachingConfig(true,2048))) - |---E~PromQLGrpcRemoteExec(PromQlQueryParams(test{job="app"}[900s],3599,1,3599,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), queryEndpoint=grpcEndpoint2.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),RoutingConfig(true,3 days,true,300000),CachingConfig(true,2048))) - |---E~PromQLGrpcRemoteExec(PromQlQueryParams(test{job="app"}[900s],3599,1,3599,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), queryEndpoint=grpcEndpoint1.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),RoutingConfig(true,3 days,true,300000),CachingConfig(true,2048))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams(test{job = "app"},3600,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,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), queryEndpoint=grpcEndpoint2.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),RoutingConfig(true,3 days,true,300000),CachingConfig(true,2048)))""".stripMargin + 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),RoutingConfig(true,3 days,true,300000))) + |-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,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), 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),RoutingConfig(true,3 days,true,300000))) + |-T~PeriodicSamplesMapper(start=3000000, step=100000, end=3599000, 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),RoutingConfig(true,3 days,true,300000))) + |---E~PromQlRemoteExec(PromQlQueryParams(test{job="app"}[900s],3599,1,3599,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), 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),RoutingConfig(true,3 days,true,300000))) + |---E~PromQlRemoteExec(PromQlQueryParams(test{job="app"}[900s],3599,1,3599,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), 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),RoutingConfig(true,3 days,true,300000))) + |-E~PromQlRemoteExec(PromQlQueryParams(test{job = "app"},3600,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,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), 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),RoutingConfig(true,3 days,true,300000)))""".stripMargin validatePlan(execPlan, expectedPlanTree) } @@ -824,11 +824,10 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers with PlanValida 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, - secondPartitionStart * 1000 - 1), Some("grpcEndpoint1")), + secondPartitionStart * 1000 - 1)), PartitionAssignment("remote2", "remote-url2", TimeRange(secondPartitionStart * 1000, - thirdPartitionStart * 1000 - 1), Some("grpcEndpoint2")), - PartitionAssignment("remote3", "remote-url3", TimeRange(thirdPartitionStart * 1000, endSeconds * 1000), - Some("grpcEndpoint3"))) + thirdPartitionStart * 1000 - 1)), + PartitionAssignment("remote3", "remote-url3", TimeRange(thirdPartitionStart * 1000, endSeconds * 1000))) else Nil } override def getMetadataPartitions(nonMetricShardKeyFilters: Seq[ColumnFilter], timeRange: TimeRange): List[PartitionAssignment] = @@ -844,20 +843,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),RoutingConfig(true,3 days,true,300000),CachingConfig(true,2048))) - |-E~PromQLGrpcRemoteExec(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,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), queryEndpoint=grpcEndpoint1.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),RoutingConfig(true,3 days,true,300000),CachingConfig(true,2048))) - |-T~PeriodicSamplesMapper(start=4000000, step=100000, end=4599000, window=None, functionId=None, rawSource=true, 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),RoutingConfig(true,3 days,true,300000),CachingConfig(true,2048))) - |---E~PromQLGrpcRemoteExec(PromQlQueryParams(test{job="app"}[900s],4599,1,4599,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), queryEndpoint=grpcEndpoint3.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),RoutingConfig(true,3 days,true,300000),CachingConfig(true,2048))) - |---E~PromQLGrpcRemoteExec(PromQlQueryParams(test{job="app"}[900s],4599,1,4599,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), queryEndpoint=grpcEndpoint2.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),RoutingConfig(true,3 days,true,300000),CachingConfig(true,2048))) - |---E~PromQLGrpcRemoteExec(PromQlQueryParams(test{job="app"}[900s],4599,1,4599,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), queryEndpoint=grpcEndpoint1.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),RoutingConfig(true,3 days,true,300000),CachingConfig(true,2048))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams(test{job = "app"},4600,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,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), queryEndpoint=grpcEndpoint2.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),RoutingConfig(true,3 days,true,300000),CachingConfig(true,2048))) - |-T~PeriodicSamplesMapper(start=7000000, step=100000, end=7599000, window=None, functionId=None, rawSource=true, 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),RoutingConfig(true,3 days,true,300000),CachingConfig(true,2048))) - |---E~PromQLGrpcRemoteExec(PromQlQueryParams(test{job="app"}[900s],7599,1,7599,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), queryEndpoint=grpcEndpoint3.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),RoutingConfig(true,3 days,true,300000),CachingConfig(true,2048))) - |---E~PromQLGrpcRemoteExec(PromQlQueryParams(test{job="app"}[900s],7599,1,7599,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), queryEndpoint=grpcEndpoint2.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),RoutingConfig(true,3 days,true,300000),CachingConfig(true,2048))) - |---E~PromQLGrpcRemoteExec(PromQlQueryParams(test{job="app"}[900s],7599,1,7599,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), queryEndpoint=grpcEndpoint1.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),RoutingConfig(true,3 days,true,300000),CachingConfig(true,2048))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams(test{job = "app"},7600,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,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), queryEndpoint=grpcEndpoint3.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),RoutingConfig(true,3 days,true,300000),CachingConfig(true,2048)))""".stripMargin + 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),RoutingConfig(true,3 days,true,300000))) + |-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,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), 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),RoutingConfig(true,3 days,true,300000))) + |-T~PeriodicSamplesMapper(start=4000000, step=100000, end=4599000, 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),RoutingConfig(true,3 days,true,300000))) + |---E~PromQlRemoteExec(PromQlQueryParams(test{job="app"}[900s],4599,1,4599,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), 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),RoutingConfig(true,3 days,true,300000))) + |---E~PromQlRemoteExec(PromQlQueryParams(test{job="app"}[900s],4599,1,4599,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), 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),RoutingConfig(true,3 days,true,300000))) + |---E~PromQlRemoteExec(PromQlQueryParams(test{job="app"}[900s],4599,1,4599,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), 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),RoutingConfig(true,3 days,true,300000))) + |-E~PromQlRemoteExec(PromQlQueryParams(test{job = "app"},4600,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,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), 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),RoutingConfig(true,3 days,true,300000))) + |-T~PeriodicSamplesMapper(start=7000000, step=100000, end=7599000, 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),RoutingConfig(true,3 days,true,300000))) + |---E~PromQlRemoteExec(PromQlQueryParams(test{job="app"}[900s],7599,1,7599,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), 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),RoutingConfig(true,3 days,true,300000))) + |---E~PromQlRemoteExec(PromQlQueryParams(test{job="app"}[900s],7599,1,7599,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), 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),RoutingConfig(true,3 days,true,300000))) + |---E~PromQlRemoteExec(PromQlQueryParams(test{job="app"}[900s],7599,1,7599,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), 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),RoutingConfig(true,3 days,true,300000))) + |-E~PromQlRemoteExec(PromQlQueryParams(test{job = "app"},7600,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,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), 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),RoutingConfig(true,3 days,true,300000)))""".stripMargin validatePlan(execPlan, expectedRawPlan) } @@ -1602,15 +1601,18 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers with PlanValida } it("should generate generate a raw export from remote from multiple partitions and stitch") { + val p1StartSecs = 1000 val p1EndSecs = 6999 val p2StartSecs = 7000 val p2EndSecs = 15000 val stepSecs = 100 + val queryStartSecs = 12000 + val subqueryLookbackSecs = 9000 def twoPartitions(): List[PartitionAssignment] = List( - PartitionAssignment("remote", "remote-url", TimeRange(p1StartSecs * 1000, p1EndSecs * 1000), Some("grpcEndpoint1")), - PartitionAssignment("local", "local-url", TimeRange(p2StartSecs * 1000, p2EndSecs * 1000), Some("grpcEndpoint1")) + PartitionAssignment("remote", "remote-url", TimeRange(p1StartSecs * 1000, p1EndSecs * 1000)), + PartitionAssignment("local", "local-url", TimeRange(p2StartSecs * 1000, p2EndSecs * 1000)) ) val partitionLocationProvider = new PartitionLocationProvider { @@ -1630,25 +1632,25 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers with PlanValida } val expectedPlanWithRemoteExport = - 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),RoutingConfig(true,3 days,true,300000),CachingConfig(true,2048))) - |-E~PromQLGrpcRemoteExec(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,true,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), queryEndpoint=grpcEndpoint1.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),RoutingConfig(true,3 days,true,300000),CachingConfig(true,2048))) + 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),RoutingConfig(true,3 days,true,300000))) + |-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,true,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), 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),RoutingConfig(true,3 days,true,300000))) |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(7000,100,7899)) - |--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),RoutingConfig(true,3 days,true,300000),CachingConfig(true,2048))) + |--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),RoutingConfig(true,3 days,true,300000))) |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |----T~PeriodicSamplesMapper(start=7000000, step=100000, end=7899000, window=Some(600000), functionId=Some(Rate), rawSource=true, 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),RoutingConfig(true,3 days,true,300000),CachingConfig(true,2048))) - |------E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1345229141],raw) - |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=3, chunkMethod=TimeRangeChunkScan(6399000,7899000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1345229141],raw) - |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=19, chunkMethod=TimeRangeChunkScan(6399000,7899000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1345229141],raw) - |------E~PromQLGrpcRemoteExec(PromQlQueryParams(test{job="app"}[1500s],7899,1,7899,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), queryEndpoint=grpcEndpoint1.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),RoutingConfig(true,3 days,true,300000),CachingConfig(true,2048))) + |----T~PeriodicSamplesMapper(start=7000000, step=100000, end=7899000, 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),RoutingConfig(true,3 days,true,300000))) + |------E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#885676802],raw) + |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=3, chunkMethod=TimeRangeChunkScan(6399000,7899000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#885676802],raw) + |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=19, chunkMethod=TimeRangeChunkScan(6399000,7899000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#885676802],raw) + |------E~PromQlRemoteExec(PromQlQueryParams(test{job="app"}[1500s],7899,1,7899,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), 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),RoutingConfig(true,3 days,true,300000))) |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(7900,100,10000)) - |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1345229141],raw) + |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#885676802],raw) |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) |----T~PeriodicSamplesMapper(start=7900000, step=100000, end=10000000, window=Some(600000), functionId=Some(Rate), rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=3, chunkMethod=TimeRangeChunkScan(7300000,10000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1345229141],raw) + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=3, chunkMethod=TimeRangeChunkScan(7300000,10000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#885676802],raw) |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) |----T~PeriodicSamplesMapper(start=7900000, step=100000, end=10000000, window=Some(600000), functionId=Some(Rate), rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=19, chunkMethod=TimeRangeChunkScan(7300000,10000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1345229141],raw)""".stripMargin + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=19, chunkMethod=TimeRangeChunkScan(7300000,10000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#885676802],raw)""".stripMargin val engine = new MultiPartitionPlanner(partitionLocationProvider, localPlanner, "local", dataset, queryConfig.copy(routingConfig = queryConfig.routingConfig.copy(supportRemoteRawExport = true))) @@ -1661,44 +1663,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),RoutingConfig(true,3 days,true,300000),CachingConfig(true,2048))) - |-E~PromQLGrpcRemoteExec(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,true,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), queryEndpoint=grpcEndpoint1.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),RoutingConfig(true,3 days,true,300000),CachingConfig(true,2048))) - |-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),RoutingConfig(true,3 days,true,300000),CachingConfig(true,2048))) + """E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),RoutingConfig(true,3 days,true,300000))) + |-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,true,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), 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),RoutingConfig(true,3 days,true,300000))) + |-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),RoutingConfig(true,3 days,true,300000))) |--T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(7000,100,7899)) - |---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),RoutingConfig(true,3 days,true,300000),CachingConfig(true,2048))) + |---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),RoutingConfig(true,3 days,true,300000))) |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |-----T~PeriodicSamplesMapper(start=7000000, step=100000, end=7899000, window=Some(600000), functionId=Some(Rate), rawSource=true, 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),RoutingConfig(true,3 days,true,300000),CachingConfig(true,2048))) - |-------E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1680817524],raw) - |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=3, chunkMethod=TimeRangeChunkScan(6399000,7899000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1680817524],raw) - |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=19, chunkMethod=TimeRangeChunkScan(6399000,7899000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1680817524],raw) - |-------E~PromQLGrpcRemoteExec(PromQlQueryParams(test{job="app"}[1500s],7899,1,7899,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), queryEndpoint=grpcEndpoint1.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),RoutingConfig(true,3 days,true,300000),CachingConfig(true,2048))) + |-----T~PeriodicSamplesMapper(start=7000000, step=100000, end=7899000, 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),RoutingConfig(true,3 days,true,300000))) + |-------E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-3#479288168],raw) + |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=3, chunkMethod=TimeRangeChunkScan(6399000,7899000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-3#479288168],raw) + |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=19, chunkMethod=TimeRangeChunkScan(6399000,7899000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-3#479288168],raw) + |-------E~PromQlRemoteExec(PromQlQueryParams(test{job="app"}[1500s],7899,1,7899,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), 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),RoutingConfig(true,3 days,true,300000))) |--T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(7000,100,7899)) - |---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),RoutingConfig(true,3 days,true,300000),CachingConfig(true,2048))) + |---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),RoutingConfig(true,3 days,true,300000))) |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |-----T~PeriodicSamplesMapper(start=7000000, step=100000, end=7899000, window=Some(300000), functionId=Some(Rate), rawSource=true, 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),RoutingConfig(true,3 days,true,300000),CachingConfig(true,2048))) - |-------E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1680817524],raw) - |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=11, chunkMethod=TimeRangeChunkScan(6699000,7899000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1680817524],raw) - |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=27, chunkMethod=TimeRangeChunkScan(6699000,7899000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1680817524],raw) - |-------E~PromQLGrpcRemoteExec(PromQlQueryParams(bar{job="app"}[1200s],7899,1,7899,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), queryEndpoint=grpcEndpoint1.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),RoutingConfig(true,3 days,true,300000),CachingConfig(true,2048))) - |-E~BinaryJoinExec(binaryOp=ADD, on=None, ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1680817524],raw) + |-----T~PeriodicSamplesMapper(start=7000000, step=100000, end=7899000, 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),RoutingConfig(true,3 days,true,300000))) + |-------E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-3#479288168],raw) + |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=11, chunkMethod=TimeRangeChunkScan(6699000,7899000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-3#479288168],raw) + |--------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=27, chunkMethod=TimeRangeChunkScan(6699000,7899000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-3#479288168],raw) + |-------E~PromQlRemoteExec(PromQlQueryParams(bar{job="app"}[1200s],7899,1,7899,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), 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),RoutingConfig(true,3 days,true,300000))) + |-E~BinaryJoinExec(binaryOp=ADD, on=None, ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-3#479288168],raw) |--T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(7900,100,10000)) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1680817524],raw) + |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-3#479288168],raw) |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) |-----T~PeriodicSamplesMapper(start=7900000, step=100000, end=10000000, window=Some(600000), functionId=Some(Rate), rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=3, chunkMethod=TimeRangeChunkScan(7300000,10000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1680817524],raw) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=3, chunkMethod=TimeRangeChunkScan(7300000,10000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-3#479288168],raw) |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) |-----T~PeriodicSamplesMapper(start=7900000, step=100000, end=10000000, window=Some(600000), functionId=Some(Rate), rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=19, chunkMethod=TimeRangeChunkScan(7300000,10000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1680817524],raw) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=19, chunkMethod=TimeRangeChunkScan(7300000,10000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-3#479288168],raw) |--T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(7900,100,10000)) - |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1680817524],raw) + |---E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-3#479288168],raw) |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) |-----T~PeriodicSamplesMapper(start=7900000, step=100000, end=10000000, window=Some(300000), functionId=Some(Rate), rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=11, chunkMethod=TimeRangeChunkScan(7600000,10000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1680817524],raw) + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=11, chunkMethod=TimeRangeChunkScan(7600000,10000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-3#479288168],raw) |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) |-----T~PeriodicSamplesMapper(start=7900000, step=100000, end=10000000, window=Some(300000), functionId=Some(Rate), rawSource=true, offsetMs=None) - |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=27, chunkMethod=TimeRangeChunkScan(7600000,10000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1680817524],raw)""".stripMargin + |------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=27, chunkMethod=TimeRangeChunkScan(7600000,10000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(bar))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-3#479288168],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)) @@ -1709,29 +1711,31 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers with PlanValida validatePlan(execPlan2, expectedPlanWithRemoteExec1) + + // Planner with period of uncertainty should still generate steps that are aligned with start and step, // that is should be snapped correctly val expectedPlanWithRemoteExportAndUncertainty = - 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),RoutingConfig(true,3 days,true,3000),CachingConfig(true,2048))) - |-E~PromQLGrpcRemoteExec(PromQlQueryParams(sum(rate(test{job = "app"}[10m])),1000,400,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,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), queryEndpoint=grpcEndpoint1.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),RoutingConfig(true,3 days,true,3000),CachingConfig(true,2048))) + 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),RoutingConfig(true,3 days,true,3000))) + |-E~PromQlRemoteExec(PromQlQueryParams(sum(rate(test{job = "app"}[10m])),1000,400,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,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), 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),RoutingConfig(true,3 days,true,3000))) |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(7000,400,7799)) - |--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),RoutingConfig(true,3 days,true,3000),CachingConfig(true,2048))) + |--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),RoutingConfig(true,3 days,true,3000))) |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |----T~PeriodicSamplesMapper(start=7000000, step=400000, end=7799000, window=Some(600000), functionId=Some(Rate), rawSource=true, 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),RoutingConfig(true,3 days,true,3000),CachingConfig(true,2048))) - |------E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#654367497],raw) - |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=3, chunkMethod=TimeRangeChunkScan(6399000,7799000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#654367497],raw) - |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=19, chunkMethod=TimeRangeChunkScan(6399000,7799000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#654367497],raw) - |------E~PromQLGrpcRemoteExec(PromQlQueryParams(test{job="app"}[1400s],7799,1,7799,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), queryEndpoint=grpcEndpoint1.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),RoutingConfig(true,3 days,true,3000),CachingConfig(true,2048))) + |----T~PeriodicSamplesMapper(start=7000000, step=400000, end=7799000, 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),RoutingConfig(true,3 days,true,3000))) + |------E~LocalPartitionDistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1666791464],raw) + |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=3, chunkMethod=TimeRangeChunkScan(6399000,7799000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1666791464],raw) + |-------E~MultiSchemaPartitionsExec(dataset=timeseries, shard=19, chunkMethod=TimeRangeChunkScan(6399000,7799000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1666791464],raw) + |------E~PromQlRemoteExec(PromQlQueryParams(test{job="app"}[1400s],7799,1,7799,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), 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),RoutingConfig(true,3 days,true,3000))) |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(7800,400,10000)) - |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#654367497],raw) + |--E~LocalPartitionReduceAggregateExec(aggrOp=Sum, aggrParams=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1666791464],raw) |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) |----T~PeriodicSamplesMapper(start=7800000, step=400000, end=10000000, window=Some(600000), functionId=Some(Rate), rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=3, chunkMethod=TimeRangeChunkScan(7200000,10000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#654367497],raw) + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=3, chunkMethod=TimeRangeChunkScan(7200000,10000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1666791464],raw) |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) |----T~PeriodicSamplesMapper(start=7800000, step=400000, end=10000000, window=Some(600000), functionId=Some(Rate), rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=19, chunkMethod=TimeRangeChunkScan(7200000,10000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#654367497],raw)""".stripMargin + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=19, chunkMethod=TimeRangeChunkScan(7200000,10000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1666791464],raw)""".stripMargin val engine1 = new MultiPartitionPlanner(partitionLocationProvider, localPlanner, "local", dataset, queryConfig.copy(routingConfig = queryConfig.routingConfig.copy( @@ -1743,6 +1747,8 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers with PlanValida val execPlan3 = engine1.materialize(lp5, QueryContext(origQueryParams = promQlQueryParam5, plannerParams = PlannerParams(processMultiPartition = true))) validatePlan(execPlan3, expectedPlanWithRemoteExportAndUncertainty) + + } it ("should give the correct routing keys") { @@ -1755,39 +1761,4 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers with PlanValida val expected = Set(Map("job" -> "abc"), Map("job" -> "def"), Map("job" -> "ghi")) mpp.getRoutingKeys(lp) shouldEqual expected } - - it ("Simulate namespace migration with histogram queries") { - val partitionLocationProvider = new PartitionLocationProvider { - override def getPartitions(routingKey: Map[String, String], timeRange: TimeRange): List[PartitionAssignment] = { - val mid = timeRange.startMs + (timeRange.endMs - timeRange.startMs) / 2 - List(PartitionAssignment("remote-1", "remote-url-1", TimeRange(timeRange.startMs, mid), Some("grpc-1")), - PartitionAssignment("remote-2", "remote-url-2", TimeRange(mid + 1, timeRange.endMs * 100), Some("grpc-2"))) - } - override def getMetadataPartitions(nonMetricShardKeyFilters: Seq[ColumnFilter], timeRange: TimeRange): List[PartitionAssignment] = - List(PartitionAssignment("local", "local-url", TimeRange(timeRange.startMs, timeRange.endMs))) - } - val histogramQuery = """histogram_quantile(0.5,sum(rate(mns_gmail_authenticate_request_ms{_ws_="acs-icloud",_ns_="mail-notifications",app="mail-notifications"}[2m])))""" - - val engine = new MultiPartitionPlanner(partitionLocationProvider, localPlanner, "local", dataset, queryConfig) - val lp = Parser.queryRangeToLogicalPlan(histogramQuery, TimeStepParams(1000, 100, 2000)) - val promQlQueryParams = PromQlQueryParams(histogramQuery, 1000, 100, 2000) - val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams, plannerParams = - PlannerParams(processMultiPartition = true))) - - // test the plan parameters - execPlan.isInstanceOf[StitchRvsExec] shouldEqual (true) - execPlan.children.length shouldEqual 3 - execPlan.children(0).isInstanceOf[PromQLGrpcRemoteExec] shouldEqual true - execPlan.children(1).isInstanceOf[LocalPartitionReduceAggregateExec] shouldEqual true - execPlan.children(2).isInstanceOf[PromQLGrpcRemoteExec] shouldEqual true - val innerRawExportPlan = execPlan.children(1).asInstanceOf[LocalPartitionReduceAggregateExec] - innerRawExportPlan.rangeVectorTransformers.length shouldEqual 2 - innerRawExportPlan.childAggregates.length shouldEqual 1 - innerRawExportPlan.childAggregates.head.isInstanceOf[StitchRvsExec] shouldEqual true - val innerStitchPlan = innerRawExportPlan.childAggregates.head.asInstanceOf[StitchRvsExec] - innerStitchPlan.rangeVectorTransformers.length shouldEqual 2 - innerStitchPlan.rangeVectorTransformers(0).isInstanceOf[PeriodicSamplesMapper] shouldEqual true - val psmT = innerStitchPlan.rangeVectorTransformers(0).asInstanceOf[PeriodicSamplesMapper] - psmT.rawSource shouldEqual true - } } diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala index 5417320851..3ebfbab5ba 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/PlannerHierarchySpec.scala @@ -2841,7 +2841,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(5001,3,6803)) |--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),RoutingConfig(true,3 days,true,3000),CachingConfig(true,2048))) |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |----T~PeriodicSamplesMapper(start=5001000, step=3000, end=6803000, window=Some(1800000), functionId=Some(Rate), rawSource=true, offsetMs=None) + |----T~PeriodicSamplesMapper(start=5001000, step=3000, end=6803000, window=Some(1800000), 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),RoutingConfig(true,3 days,true,3000),CachingConfig(true,2048))) |------E~PromQlRemoteExec(PromQlQueryParams(test{_ws_="demo",_ns_="localNs"}[3603s],6803,1,6803,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),RoutingConfig(true,3 days,true,3000),CachingConfig(true,2048))) |------E~PromQlRemoteExec(PromQlQueryParams(test{_ws_="demo",_ns_="localNs"}[3603s],6803,1,6803,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), 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),RoutingConfig(true,3 days,true,3000),CachingConfig(true,2048))) @@ -2852,7 +2852,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |-T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(6801,3,7403)) |--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),RoutingConfig(true,3 days,true,3000),CachingConfig(true,2048))) |---T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |----T~PeriodicSamplesMapper(start=6801000, step=3000, end=7403000, window=Some(600000), functionId=Some(Rate), rawSource=true, offsetMs=Some(1800000)) + |----T~PeriodicSamplesMapper(start=6801000, step=3000, end=7403000, window=Some(600000), functionId=Some(Rate), rawSource=false, offsetMs=Some(1800000)) |-----E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),RoutingConfig(true,3 days,true,3000),CachingConfig(true,2048))) |------E~PromQlRemoteExec(PromQlQueryParams(test{_ws_="demo",_ns_="localNs"}[3003s],7403,1,7403,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),RoutingConfig(true,3 days,true,3000),CachingConfig(true,2048))) |------E~PromQlRemoteExec(PromQlQueryParams(test{_ws_="demo",_ns_="localNs"}[3003s],7403,1,7403,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), 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),RoutingConfig(true,3 days,true,3000),CachingConfig(true,2048))) @@ -2864,14 +2864,14 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |--T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(5601,3,8003)) |---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),RoutingConfig(true,3 days,true,3000),CachingConfig(true,2048))) |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |-----T~PeriodicSamplesMapper(start=5601000, step=3000, end=8003000, window=Some(1800000), functionId=Some(Rate), rawSource=true, offsetMs=Some(1200000)) + |-----T~PeriodicSamplesMapper(start=5601000, step=3000, end=8003000, window=Some(1800000), functionId=Some(Rate), rawSource=false, offsetMs=Some(1200000)) |------E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),RoutingConfig(true,3 days,true,3000),CachingConfig(true,2048))) |-------E~PromQlRemoteExec(PromQlQueryParams(test{_ws_="demo",_ns_="localNs"}[5403s],8003,1,8003,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),RoutingConfig(true,3 days,true,3000),CachingConfig(true,2048))) |-------E~PromQlRemoteExec(PromQlQueryParams(test{_ws_="demo",_ns_="localNs"}[5403s],8003,1,8003,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), 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),RoutingConfig(true,3 days,true,3000),CachingConfig(true,2048))) |--T~AggregatePresenter(aggrOp=Sum, aggrParams=List(), rangeParams=RangeParams(5601,3,8003)) |---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),RoutingConfig(true,3 days,true,3000),CachingConfig(true,2048))) |----T~AggregateMapReduce(aggrOp=Sum, aggrParams=List(), without=List(), by=List()) - |-----T~PeriodicSamplesMapper(start=5601000, step=3000, end=8003000, window=Some(1800000), functionId=Some(Rate), rawSource=true, offsetMs=Some(600000)) + |-----T~PeriodicSamplesMapper(start=5601000, step=3000, end=8003000, window=Some(1800000), functionId=Some(Rate), rawSource=false, offsetMs=Some(600000)) |------E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),RoutingConfig(true,3 days,true,3000),CachingConfig(true,2048))) |-------E~PromQlRemoteExec(PromQlQueryParams(test{_ws_="demo",_ns_="localNs"}[4803s],8003,1,8003,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),RoutingConfig(true,3 days,true,3000),CachingConfig(true,2048))) |-------E~PromQlRemoteExec(PromQlQueryParams(test{_ws_="demo",_ns_="localNs"}[4803s],8003,1,8003,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), 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),RoutingConfig(true,3 days,true,3000),CachingConfig(true,2048))) @@ -2880,11 +2880,11 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS 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),RoutingConfig(true,3 days,true,3000),CachingConfig(true,2048))) |-E~PromQlRemoteExec(PromQlQueryParams(sum_over_time(test{_ws_ = "demo", _ns_ = "localNs"}[10m]) + sum_over_time(test{_ws_ = "demo", _ns_ = "localNs"}[30m] offset 10m),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,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),RoutingConfig(true,3 days,true,3000),CachingConfig(true,2048))) |-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),RoutingConfig(true,3 days,true,3000),CachingConfig(true,2048))) - |--T~PeriodicSamplesMapper(start=5001000, step=3000, end=7403000, window=Some(600000), functionId=Some(SumOverTime), rawSource=true, offsetMs=None) + |--T~PeriodicSamplesMapper(start=5001000, step=3000, end=7403000, window=Some(600000), functionId=Some(SumOverTime), 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),RoutingConfig(true,3 days,true,3000),CachingConfig(true,2048))) |----E~PromQlRemoteExec(PromQlQueryParams(test{_ws_="demo",_ns_="localNs"}[3003s],7403,1,7403,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),RoutingConfig(true,3 days,true,3000),CachingConfig(true,2048))) |----E~PromQlRemoteExec(PromQlQueryParams(test{_ws_="demo",_ns_="localNs"}[3003s],7403,1,7403,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), 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),RoutingConfig(true,3 days,true,3000),CachingConfig(true,2048))) - |--T~PeriodicSamplesMapper(start=5001000, step=3000, end=7403000, window=Some(1800000), functionId=Some(SumOverTime), rawSource=true, offsetMs=Some(600000)) + |--T~PeriodicSamplesMapper(start=5001000, step=3000, end=7403000, window=Some(1800000), functionId=Some(SumOverTime), rawSource=false, offsetMs=Some(600000)) |---E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),RoutingConfig(true,3 days,true,3000),CachingConfig(true,2048))) |----E~PromQlRemoteExec(PromQlQueryParams(test{_ws_="demo",_ns_="localNs"}[4803s],7403,1,7403,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),RoutingConfig(true,3 days,true,3000),CachingConfig(true,2048))) |----E~PromQlRemoteExec(PromQlQueryParams(test{_ws_="demo",_ns_="localNs"}[4803s],7403,1,7403,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), 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),RoutingConfig(true,3 days,true,3000),CachingConfig(true,2048))) @@ -2894,7 +2894,7 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |-E~PromQlRemoteExec(PromQlQueryParams(123 + sum_over_time(test{_ws_ = "demo", _ns_ = "localNs"}[10m] offset 10m),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,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),RoutingConfig(true,3 days,true,3000),CachingConfig(true,2048))) |-T~ScalarOperationMapper(operator=ADD, scalarOnLhs=true) |--FA1~StaticFuncArgs(123.0,RangeParams(5001,3,6203)) - |--T~PeriodicSamplesMapper(start=5001000, step=3000, end=6203000, window=Some(600000), functionId=Some(SumOverTime), rawSource=true, offsetMs=Some(600000)) + |--T~PeriodicSamplesMapper(start=5001000, step=3000, end=6203000, window=Some(600000), functionId=Some(SumOverTime), rawSource=false, offsetMs=Some(600000)) |---E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),RoutingConfig(true,3 days,true,3000),CachingConfig(true,2048))) |----E~PromQlRemoteExec(PromQlQueryParams(test{_ws_="demo",_ns_="localNs"}[2403s],6203,1,6203,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),RoutingConfig(true,3 days,true,3000),CachingConfig(true,2048))) |----E~PromQlRemoteExec(PromQlQueryParams(test{_ws_="demo",_ns_="localNs"}[2403s],6203,1,6203,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), 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),RoutingConfig(true,3 days,true,3000),CachingConfig(true,2048))) @@ -2905,11 +2905,11 @@ class PlannerHierarchySpec extends AnyFunSpec with Matchers with PlanValidationS |-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),RoutingConfig(true,3 days,true,3000),CachingConfig(true,2048))) |--T~ScalarOperationMapper(operator=ADD, scalarOnLhs=true) |---FA1~StaticFuncArgs(123.0,RangeParams(5001,3,7403)) - |---T~PeriodicSamplesMapper(start=5001000, step=3000, end=7403000, window=Some(600000), functionId=Some(SumOverTime), rawSource=true, offsetMs=Some(600000)) + |---T~PeriodicSamplesMapper(start=5001000, step=3000, end=7403000, window=Some(600000), functionId=Some(SumOverTime), rawSource=false, offsetMs=Some(600000)) |----E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),RoutingConfig(true,3 days,true,3000),CachingConfig(true,2048))) |-----E~PromQlRemoteExec(PromQlQueryParams(test{_ws_="demo",_ns_="localNs"}[3603s],7403,1,7403,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),RoutingConfig(true,3 days,true,3000),CachingConfig(true,2048))) |-----E~PromQlRemoteExec(PromQlQueryParams(test{_ws_="demo",_ns_="localNs"}[3603s],7403,1,7403,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), 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),RoutingConfig(true,3 days,true,3000),CachingConfig(true,2048))) - |--T~PeriodicSamplesMapper(start=5001000, step=3000, end=7403000, window=Some(1200000), functionId=Some(SumOverTime), rawSource=true, offsetMs=Some(1200000)) + |--T~PeriodicSamplesMapper(start=5001000, step=3000, end=7403000, window=Some(1200000), functionId=Some(SumOverTime), rawSource=false, offsetMs=Some(1200000)) |---E~StitchRvsExec() on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),RoutingConfig(true,3 days,true,3000),CachingConfig(true,2048))) |----E~PromQlRemoteExec(PromQlQueryParams(test{_ws_="demo",_ns_="localNs"}[4803s],7403,1,7403,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), queryEndpoint=remote0-url, requestTimeoutMs=10000) on InProcessPlanDispatcher(QueryConfig(10 seconds,300000,1,50,antlr,true,true,None,Some(10000),None,None,25,true,false,true,Set(),Some(plannerSelector),Map(filodb-query-exec-metadataexec -> 65536, filodb-query-exec-aggregate-large-container -> 65536),RoutingConfig(true,3 days,true,3000),CachingConfig(true,2048))) |----E~PromQlRemoteExec(PromQlQueryParams(test{_ws_="demo",_ns_="localNs"}[4803s],7403,1,7403,None,false), PlannerParams(filodb,None,None,None,None,60000,PerQueryLimits(1000000,18000000,100000,100000,300000000,1000000,200000000),PerQueryLimits(50000,15000000,50000,50000,150000000,500000,100000000),None,None,None,false,86400000,86400000,false,true,false,false,true,10,false,true,TreeSet(),LegacyFailoverMode,None,None,None,None), 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),RoutingConfig(true,3 days,true,3000),CachingConfig(true,2048))) diff --git a/core/src/main/scala/filodb.core/query/RangeVector.scala b/core/src/main/scala/filodb.core/query/RangeVector.scala index 212f4bbe71..a98ce2388e 100644 --- a/core/src/main/scala/filodb.core/query/RangeVector.scala +++ b/core/src/main/scala/filodb.core/query/RangeVector.scala @@ -296,11 +296,11 @@ object RepeatValueVector extends StrictLogging { val startNs = Utils.currentThreadCpuTimeNanos try { ChunkMap.validateNoSharedLocks(execPlan) - Using.resource(rv.rows()){ + Using(rv.rows()){ rows => val nextRow = if (rows.hasNext) Some(rows.next()) else None new RepeatValueVector(rv.key, startMs, stepMs, endMs, nextRow, schema) - } + }.get } finally { ChunkMap.releaseAllSharedLocks() queryStats.getCpuNanosCounter(Nil).addAndGet(Utils.currentThreadCpuTimeNanos - startNs) @@ -573,19 +573,20 @@ object SerializedRangeVector extends StrictLogging { val startRecordNo = oldContainerOpt.map(_.numRecords).getOrElse(0) try { ChunkMap.validateNoSharedLocks(execPlan) - Using.resource(rv.rows()) { - rows => while (rows.hasNext) { - val nextRow = rows.next() - // Don't encode empty / NaN data over the wire - if (!canRemoveEmptyRows(rv.outputRange, schema) || - schema.columns(1).colType == DoubleColumn && !java.lang.Double.isNaN(nextRow.getDouble(1)) || - schema.columns(1).colType == HistogramColumn && !nextRow.getHistogram(1).isEmpty) { - numRows += 1 - builder.addFromReader(nextRow, schema, 0) - } - } + val rows = rv.rows + while (rows.hasNext) { + val nextRow = rows.next() + // Don't encode empty / NaN data over the wire + if (!canRemoveEmptyRows(rv.outputRange, schema) || + schema.columns(1).colType == DoubleColumn && !java.lang.Double.isNaN(nextRow.getDouble(1)) || + schema.columns(1).colType == HistogramColumn && !nextRow.getHistogram(1).isEmpty) { + numRows += 1 + builder.addFromReader(nextRow, schema, 0) + } } } finally { + rv.rows().close() + // clear exec plan // When the query is done, clean up lingering shared locks caused by iterator limit. ChunkMap.releaseAllSharedLocks() } diff --git a/grpc/src/main/protobuf/query_service.proto b/grpc/src/main/protobuf/query_service.proto index 6f779b5092..3af700bcfa 100644 --- a/grpc/src/main/protobuf/query_service.proto +++ b/grpc/src/main/protobuf/query_service.proto @@ -653,7 +653,6 @@ enum MiscellaneousFunctionId { LABEL_REPLACE = 0; LABEL_JOIN = 1; HIST_TO_PROM_VECTORS = 2; - OPTIMIZE_WITH_AGG = 3; } enum BinaryOperator { diff --git a/http/src/main/scala/filodb/http/ClusterApiRoute.scala b/http/src/main/scala/filodb/http/ClusterApiRoute.scala index 1dd4b457d1..2b23e33401 100644 --- a/http/src/main/scala/filodb/http/ClusterApiRoute.scala +++ b/http/src/main/scala/filodb/http/ClusterApiRoute.scala @@ -6,7 +6,7 @@ import akka.http.scaladsl.server.Directives._ import com.typesafe.scalalogging.StrictLogging import de.heikoseeberger.akkahttpcirce.FailFastCirceSupport -import filodb.coordinator.{CurrentShardSnapshot, NodeClusterActor, ShardSnapshot} +import filodb.coordinator.{CurrentShardSnapshot, NodeClusterActor} import filodb.core.{DatasetRef, ErrorResponse, Success => SuccessResponse} import filodb.core.store.{AssignShardConfig, UnassignShardConfig} import filodb.http.apiv1.{HttpSchema, HttpShardDetails, HttpShardState, HttpShardStateByAddress} @@ -31,24 +31,9 @@ class ClusterApiRoute(clusterProxy: ActorRef) extends FiloRoute with StrictLoggi complete(httpList(statusList)) case DatasetUnknown(_) => complete(Codes.NotFound -> httpErr("DatasetUnknown", s"Dataset $dataset is not registered")) - case InternalServiceError(errorMessage) => - complete(Codes.InternalServerError -> httpErr("InternalServerError", errorMessage)) } } } ~ - // NOTE: statusV2 will only work with ClusteringV2 ShardAssignment strategy - path(Segment / "statusV2") { dataset => - get { - onSuccess(asyncAsk(clusterProxy, GetShardMapV2(DatasetRef.fromDotString(dataset)))) { - case ShardSnapshot(shardMapperV2) => - complete(httpList(Seq(shardMapperV2))) - case DatasetUnknown(_) => - complete(Codes.NotFound -> httpErr("DatasetUnknown", s"Dataset $dataset is not registered")) - case InternalServiceError(errorMessage) => - complete(Codes.InternalServerError -> httpErr("InternalServerError", errorMessage)) - } - } - } ~ // GET /api/v1/cluster//statusByAddress - shard health status grouped by node address // Sample output as follows: // {{{ diff --git a/query/src/main/scala/filodb/query/LogicalPlan.scala b/query/src/main/scala/filodb/query/LogicalPlan.scala index f60392a1d9..2e6c561494 100644 --- a/query/src/main/scala/filodb/query/LogicalPlan.scala +++ b/query/src/main/scala/filodb/query/LogicalPlan.scala @@ -192,11 +192,8 @@ case class RawSeries(rangeSelector: RangeSelector, // Check 1: Check if the leaf periodic series plan is allowed for raw series update HierarchicalQueryExperience.isLeafPeriodicSeriesPlanAllowedForRawSeriesUpdate(leafPeriodicPlan) match { case true => - // check if the metric used is raw metric and if the OptimizeWithAgg plan is used in the query - val isOptimizeWithAggLp = parentLogicalPlans - .contains(HierarchicalQueryExperience.logicalPlanForRawToAggMetric) - val updatedFilters = HierarchicalQueryExperience - .upsertMetricColumnFilterIfHigherLevelAggregationApplicable(params, filters, isOptimizeWithAggLp) + val updatedFilters = HierarchicalQueryExperience.upsertMetricColumnFilterIfHigherLevelAggregationApplicable( + params, filters) this.copy(filters = updatedFilters) case false => this } @@ -574,25 +571,13 @@ case class Aggregate(operator: AggregationOperator, override def useAggregatedMetricIfApplicable(params: HierarchicalQueryExperienceParams, parentLogicalPlans: Seq[String]): PeriodicSeriesPlan = { // Modify the map to retain all the AggRules which satisfies the current Aggregate clause labels. - // We should do this for both the maps. - val updatedAggRulesMap = params.aggRulesByAggregationSuffix.map { case (suffix, rules) => - val updatedRules = rules.filter(rule => HierarchicalQueryExperience - .checkAggregateQueryEligibleForHigherLevelAggregatedMetric(rule, operator, clauseOpt)) - suffix -> updatedRules - } - - val updatedRawMetricAggRulesMap = params.aggRulesByRawMetricName.map { case (metricName, rules) => - val updatedRules = rules.filter(rule => HierarchicalQueryExperience - .checkAggregateQueryEligibleForHigherLevelAggregatedMetric(rule, operator, clauseOpt)) - metricName -> updatedRules - } - - if (updatedAggRulesMap.isEmpty && updatedRawMetricAggRulesMap.isEmpty) { - // none of the aggregation rules matched with the aggregation clauses. No optimization possible + val updatedMap = params.aggRules.filter(x => HierarchicalQueryExperience + .checkAggregateQueryEligibleForHigherLevelAggregatedMetric(x._2, operator, clauseOpt)) + if (updatedMap.isEmpty) { + // none of the aggregation rules matched with the this } else { - val updatedParams = params.copy(aggRulesByAggregationSuffix = updatedAggRulesMap, - aggRulesByRawMetricName = updatedRawMetricAggRulesMap) + val updatedParams = params.copy(aggRules = updatedMap) this.copy(vectors = vectors.useAggregatedMetricIfApplicable( updatedParams, parentLogicalPlans :+ this.getClass.getSimpleName)) } @@ -741,7 +726,7 @@ case class ApplyMiscellaneousFunction(vectors: PeriodicSeriesPlan, override def useAggregatedMetricIfApplicable(params: HierarchicalQueryExperienceParams, parentLogicalPlans: Seq[String]): PeriodicSeriesPlan = { this.copy(vectors = vectors.useAggregatedMetricIfApplicable( - params, parentLogicalPlans :+ (this.getClass.getSimpleName + '-' + function.entryName))) + params, parentLogicalPlans :+ this.getClass.getSimpleName)) } } diff --git a/query/src/main/scala/filodb/query/PlanEnums.scala b/query/src/main/scala/filodb/query/PlanEnums.scala index c8fc1989f4..9333761b0b 100644 --- a/query/src/main/scala/filodb/query/PlanEnums.scala +++ b/query/src/main/scala/filodb/query/PlanEnums.scala @@ -177,7 +177,6 @@ object MiscellaneousFunctionId extends Enum[MiscellaneousFunctionId] { case object LabelReplace extends MiscellaneousFunctionId("label_replace") case object LabelJoin extends MiscellaneousFunctionId("label_join") case object HistToPromVectors extends MiscellaneousFunctionId("hist_to_prom_vectors") - case object OptimizeWithAgg extends MiscellaneousFunctionId("optimize_with_agg") } sealed abstract class SortFunctionId(override val entryName: String) extends EnumEntry diff --git a/query/src/main/scala/filodb/query/exec/aggregator/CountValuesRowAggregator.scala b/query/src/main/scala/filodb/query/exec/aggregator/CountValuesRowAggregator.scala index 7de538b8ab..d54c1042d1 100644 --- a/query/src/main/scala/filodb/query/exec/aggregator/CountValuesRowAggregator.scala +++ b/query/src/main/scala/filodb/query/exec/aggregator/CountValuesRowAggregator.scala @@ -1,7 +1,6 @@ package filodb.query.exec.aggregator import scala.collection.mutable -import scala.util.Using import filodb.core.Utils import filodb.core.binaryrecord2.RecordBuilder @@ -95,22 +94,21 @@ class CountValuesRowAggregator(label: String, limit: Int = 1000) extends RowAggr FiloSchedulers.assertThreadName(QuerySchedName) // aggRangeVector.rows.take below triggers the ChunkInfoIterator which requires lock/release ChunkMap.validateNoSharedLocks(s"CountValues-$label") - Using.resource(aggRangeVector.rows()) { - rows => rows.take(limit).foreach { row => - val rowMap = CountValuesSerDeser.deserialize(row.getBlobBase(1), - row.getBlobNumBytes(1), row.getBlobOffset(1)) - rowMap.foreach { (k, v) => - val rvk = CustomRangeVectorKey(aggRangeVector.key.labelValues + - (label.utf8 -> k.toString.utf8)) - val builder = resRvs.getOrElseUpdate(rvk, SerializedRangeVector.newBuilder()) - builder.startNewRecord(recSchema) - builder.addLong(row.getLong(0)) - builder.addDouble(v) - builder.endRecord() - } + aggRangeVector.rows.take(limit).foreach { row => + val rowMap = CountValuesSerDeser.deserialize(row.getBlobBase(1), + row.getBlobNumBytes(1), row.getBlobOffset(1)) + rowMap.foreach { (k, v) => + val rvk = CustomRangeVectorKey(aggRangeVector.key.labelValues + + (label.utf8 -> k.toString.utf8)) + val builder = resRvs.getOrElseUpdate(rvk, SerializedRangeVector.newBuilder()) + builder.startNewRecord(recSchema) + builder.addLong(row.getLong(0)) + builder.addDouble(v) + builder.endRecord() } - } + } } finally { + aggRangeVector.rows.close() ChunkMap.releaseAllSharedLocks() } resRvs.map { case (key, builder) => diff --git a/query/src/main/scala/filodb/query/exec/aggregator/TopBottomKRowAggregator.scala b/query/src/main/scala/filodb/query/exec/aggregator/TopBottomKRowAggregator.scala index 7dcc734bc8..fe45aa9dce 100644 --- a/query/src/main/scala/filodb/query/exec/aggregator/TopBottomKRowAggregator.scala +++ b/query/src/main/scala/filodb/query/exec/aggregator/TopBottomKRowAggregator.scala @@ -4,7 +4,6 @@ import java.util.concurrent.TimeUnit import scala.collection.{mutable, Iterator} import scala.collection.mutable.ListBuffer -import scala.util.Using import com.typesafe.scalalogging.StrictLogging @@ -126,35 +125,33 @@ class TopBottomKRowAggregator(k: Int, bottomK: Boolean) extends RowAggregator wi FiloSchedulers.assertThreadName(QuerySchedName) ChunkMap.validateNoSharedLocks(s"TopkQuery-$k-$bottomK") // We limit the results wherever it is materialized first. So it is done here. - Using.resource(aggRangeVector.rows()) { - rs => val rows = rs.take(limit) - val it = Iterator.from(0, rangeParams.stepSecs.toInt) - .takeWhile(_ <= (rangeParams.endSecs - rangeParams.startSecs)).map { t => - val timestamp = t + rangeParams.startSecs - val rvkSeen = new ListBuffer[RangeVectorKey] - if (rows.hasNext) { - val row = rows.next() - var i = 1 - while (row.notNull(i)) { - if (row.filoUTF8String(i) != CustomRangeVectorKey.emptyAsZcUtf8) { - val key = row.filoUTF8String(i) - val rvk = CustomRangeVectorKey.fromZcUtf8(key) - rvkSeen += rvk - val builder = resRvs.getOrElseUpdate(rvk, createBuilder(rangeParams, timestamp)) - addRecordToBuilder(builder, TimeUnit.SECONDS.toMillis(timestamp), row.getDouble(i + 1)) - } - i += 2 - } - resRvs.keySet.foreach { rvs => - if (!rvkSeen.contains(rvs)) addRecordToBuilder(resRvs(rvs), timestamp * 1000, Double.NaN) + val rows = aggRangeVector.rows.take(limit) + val it = Iterator.from(0, rangeParams.stepSecs.toInt) + .takeWhile(_ <= (rangeParams.endSecs - rangeParams.startSecs)).map { t => + val timestamp = t + rangeParams.startSecs + val rvkSeen = new ListBuffer[RangeVectorKey] + if (rows.hasNext) { + val row = rows.next() + var i = 1 + while (row.notNull(i)) { + if (row.filoUTF8String(i) != CustomRangeVectorKey.emptyAsZcUtf8) { + val key = row.filoUTF8String(i) + val rvk = CustomRangeVectorKey.fromZcUtf8(key) + rvkSeen += rvk + val builder = resRvs.getOrElseUpdate(rvk, createBuilder(rangeParams, timestamp)) + addRecordToBuilder(builder, TimeUnit.SECONDS.toMillis(timestamp), row.getDouble(i + 1)) } + i += 2 + } + resRvs.keySet.foreach { rvs => + if (!rvkSeen.contains(rvs)) addRecordToBuilder(resRvs(rvs), timestamp * 1000, Double.NaN) } } - // address step == 0 case - if (rangeParams.startSecs == rangeParams.endSecs || rangeParams.stepSecs == 0) - it.take(1).toList else it.toList } + // address step == 0 case + if (rangeParams.startSecs == rangeParams.endSecs || rangeParams.stepSecs == 0) it.take(1).toList else it.toList } finally { + aggRangeVector.rows().close() ChunkMap.releaseAllSharedLocks() } diff --git a/query/src/main/scala/filodb/query/util/HierarchicalQueryExperience.scala b/query/src/main/scala/filodb/query/util/HierarchicalQueryExperience.scala index 756fa763dd..453b003d96 100644 --- a/query/src/main/scala/filodb/query/util/HierarchicalQueryExperience.scala +++ b/query/src/main/scala/filodb/query/util/HierarchicalQueryExperience.scala @@ -8,47 +8,25 @@ import filodb.core.GlobalConfig import filodb.core.query.ColumnFilter import filodb.core.query.Filter.{Equals, EqualsRegex} import filodb.query.{AggregateClause, AggregationOperator, LogicalPlan, TsCardinalities} -import filodb.query.MiscellaneousFunctionId.OptimizeWithAgg /** * Aggregation rule definition. Contains the following information: - * - * @param metricDelimiter metricDelimiter to be matched and used to create the raw to agg metric - * @param aggRulesByAggregationSuffix map of: - * Current aggregation metric suffix -> Set of "NEXT" level aggregation's rules that can be matched - * Example - { - * "agg1": Set( - * AggRule{ metricSuffix = agg_2, tags = Set("tag1", "tag2")}, - * AggRule{ metricSuffix = agg_3, tags = Set("tag2")}, - * ) - * } - * @param aggRulesByRawMetricName map of: - * metric name -> Set of aggregation rules to be tested against the raw metric label filters. - * WHY is this separate map needed and why are we grouping by metric name ? - * A single promql query can have several metrics. Not all of them would qualify for raw to agg - * translation. Hence, we store the metrics which are eligible for the same in this map to avoid - * accidental updates. - * Example - { - * "metric_name1": Set( - * AggRule{ metricSuffix = agg, tags = Set("tag1", "tag2", "tag3")}, - * AggRule{ metricSuffix = agg_2, tags = Set("tag1", "tag2")}, - * AggRule{ metricSuffix = agg_3, tags = Set("tag2")}, - * ) - * } + * 1. aggregation metricDelimiter to be matched + * 2. map of current aggregation metric suffix -> nextLevelAggregation's AggRule to be used + * For example: agg -> AggRule { metricSuffix = agg_2, tags = Set("tag1", "tag2") } */ case class HierarchicalQueryExperienceParams(metricDelimiter: String, - aggRulesByAggregationSuffix: Map[String, Set[AggRule]], - aggRulesByRawMetricName: Map[String, Set[AggRule]] = Map.empty) { } + aggRules: Map[String, AggRule]) { } /** * Aggregation rule definition. Contains the following information: - * @param metricSuffix suffix for the given aggregation rule - * @param level level of the given aggregation rule - * @param tags include/exclude tags for the given aggregation rule + * 1. metric suffix for the given aggregation rule + * 2. include/exclude tags for the given aggregation rule */ sealed trait AggRule { + val metricSuffix: String - val level: String + val tags: Set[String] def isHigherLevelAggregationApplicable(shardKeyColumns: Set[String], filterTags: Seq[String]): Boolean } @@ -56,15 +34,14 @@ sealed trait AggRule { /** * @param metricSuffix - String - Metric suffix for the given aggregation rule * @param tags - Set[String] - Include tags as specified in the aggregation rule - * @param level - String - level of the aggregation rule */ -case class IncludeAggRule(metricSuffix: String, tags: Set[String], level: String = "1") extends AggRule { +case class IncludeAggRule(metricSuffix: String, tags: Set[String]) extends AggRule { /** * Checks if the higher level aggregation is applicable with IncludeTags. * * @param shardKeyColumns - Seq[String] - List of shard key columns. These columns are not part of check. This - * includes tags which are compulsory for the query like _metric_, _ws_, _ns_. + * include tags which are compulsory for the query like _metric_, _ws_, _ns_. * @param filterTags - Seq[String] - List of filter tags/labels in the query or in the aggregation clause * @return - Boolean */ @@ -76,9 +53,8 @@ case class IncludeAggRule(metricSuffix: String, tags: Set[String], level: String /** * @param metricSuffix - String - Metric suffix for the given aggregation rule * @param tags - Set[String] - Exclude tags as specified in the aggregation rule - * @param level - String - level of the aggregation rule */ -case class ExcludeAggRule(metricSuffix: String, tags: Set[String], level: String = "1") extends AggRule { +case class ExcludeAggRule(metricSuffix: String, tags: Set[String]) extends AggRule { /** * Checks if the higher level aggregation is applicable with ExcludeTags. Here we need to check if the column filter @@ -134,8 +110,6 @@ object HierarchicalQueryExperience extends StrictLogging { case None => None } - lazy val logicalPlanForRawToAggMetric = "ApplyMiscellaneousFunction-" + OptimizeWithAgg.entryName - /** * Helper function to get the ColumnFilter tag/label for the metric. This is needed to correctly update the filter. * @param filterTags - Seq[String] - List of ColumnFilter tags/labels @@ -175,18 +149,18 @@ object HierarchicalQueryExperience extends StrictLogging { } /** Checks if the higher level aggregation is applicable for the given Include/Exclude tags. - * @param aggRule - AggRule - Include or Exclude AggRule + * @param params - AggRule - Include or Exclude AggRule * @param filterTags - Seq[String] - List of filter tags/labels in the query or in the aggregation clause * @return - Boolean */ - def isHigherLevelAggregationApplicable(aggRule: AggRule, + def isHigherLevelAggregationApplicable(params: AggRule, filterTags: Seq[String]): Boolean = { shardKeyColumnsOption match { case None => logger.info("[HierarchicalQueryExperience] Dataset options config not found. Skipping optimization !") false case Some(shardKeyColumns) => - aggRule.isHigherLevelAggregationApplicable(shardKeyColumns, filterTags) + params.isHigherLevelAggregationApplicable(shardKeyColumns, filterTags) } } @@ -204,16 +178,6 @@ object HierarchicalQueryExperience extends StrictLogging { metricName.replaceFirst(metricDelimiter + ".*", metricDelimiter + metricSuffix) } - /** - * @param metricName raw metric name - * @param metricDelimiter metric delimiter pattern for aggregated metric - * @param metricSuffix suffix for the given aggregation rule - * @return the updated agregated metric name - */ - def getAggMetricNameForRawMetric(metricName : String, metricDelimiter: String, metricSuffix: String): String = { - metricName + metricDelimiter + metricSuffix - } - /** Gets the current metric name from the given metricColumnFilter and filters * * @param metricColumnFilter - String - Metric ColumnFilter tag/label @@ -286,57 +250,31 @@ object HierarchicalQueryExperience extends StrictLogging { } /** - * Updates the metric column filter if higher level aggregation is applicable. Two scenarios: - * 1. If the metric is aggregated metric - uses HierarchicalQueryExperienceParams.aggRulesByAggregationSuffix - * 2. If the metric is raw metric - uses HierarchicalQueryExperienceParams.aggRulesByRawMetricName map - * @param params - HierarchicalQueryExperienceParams - * Contains metricDelimiter, aggRulesByAggregationSuffix, and aggRulesByRawMetricName + * Updates the metric column filter if higher level aggregation is applicable + * @param params - HierarchicalQueryExperienceParams - Contains metricDelimiter and aggRules * @param filters - Seq[ColumnFilter] - label filters of the query/lp - * @param isOptimizeWithAggLp - Boolean - Flag to check if the query is using `optimize_with_agg` function * @return - Seq[ColumnFilter] - Updated filters */ def upsertMetricColumnFilterIfHigherLevelAggregationApplicable(params: HierarchicalQueryExperienceParams, - filters: Seq[ColumnFilter], - isOptimizeWithAggLp: Boolean): Seq[ColumnFilter] = { + filters: Seq[ColumnFilter]): Seq[ColumnFilter] = { val filterTags = getColumnsAfterFilteringOutDotStarRegexFilters(filters) val metricColumnFilter = getMetricColumnFilterTag(filterTags, GlobalConfig.datasetOptions.get.metricColumn) val currentMetricName = getMetricName(metricColumnFilter, filters) if (currentMetricName.isDefined) { - // Check if the metric name is part of the params.aggRulesByRawMetricName ( i.e. check if it is raw metric) - if (isOptimizeWithAggLp && params.aggRulesByRawMetricName.contains(currentMetricName.get)) { - // CASE 1: Check if the given raw metric can be optimized using an aggregated rule - val matchingRules = params.aggRulesByRawMetricName(currentMetricName.get) - .filter(x => isHigherLevelAggregationApplicable(x, filterTags)) - if (matchingRules.nonEmpty) { - val highestLevelAggRule = matchingRules.maxBy(x => x.level) - val updatedMetricName = getAggMetricNameForRawMetric( - currentMetricName.get, params.metricDelimiter, highestLevelAggRule.metricSuffix) - val updatedFilters = upsertFilters(filters, Seq(ColumnFilter(metricColumnFilter, Equals(updatedMetricName)))) - logger.info(s"[HierarchicalQueryExperience] Query optimized with filters: ${updatedFilters.toString()}") - incrementHierarchicalQueryOptimizedCounter(updatedFilters, true) - updatedFilters - } else { - filters - } - } else { - // CASE 2: Check if the given aggregated metric can be optimized using the NEXT level aggregation rules - params.aggRulesByAggregationSuffix.find(x => currentMetricName.get.endsWith(x._1)) match { - case Some(aggRulesSet) => - val matchingRules = aggRulesSet._2.filter( x => isHigherLevelAggregationApplicable(x, filterTags)) - if (matchingRules.nonEmpty) { - val highestLevelAggRule = matchingRules.maxBy(x => x.level) - val updatedMetricName = getNextLevelAggregatedMetricName( - currentMetricName.get, params.metricDelimiter, highestLevelAggRule.metricSuffix) - val updatedFilters = upsertFilters( - filters, Seq(ColumnFilter(metricColumnFilter, Equals(updatedMetricName)))) - logger.info(s"[HierarchicalQueryExperience] Query optimized with filters: ${updatedFilters.toString()}") - incrementHierarchicalQueryOptimizedCounter(updatedFilters, false) - updatedFilters - } else { - filters - } - case None => filters - } + params.aggRules.find( x => currentMetricName.get.endsWith(x._1)) match { + case Some(aggRule) => + if (isHigherLevelAggregationApplicable(aggRule._2, filterTags)) { + val updatedMetricName = getNextLevelAggregatedMetricName( + currentMetricName.get, params.metricDelimiter, aggRule._2.metricSuffix) + val updatedFilters = upsertFilters( + filters, Seq(ColumnFilter(metricColumnFilter, Equals(updatedMetricName)))) + logger.info(s"[HierarchicalQueryExperience] Query optimized with filters: ${updatedFilters.toString()}") + incrementHierarchicalQueryOptimizedCounter(updatedFilters) + updatedFilters + } else { + filters + } + case None => filters } } else { filters @@ -346,10 +284,8 @@ object HierarchicalQueryExperience extends StrictLogging { /** * Track the queries optimized by workspace and namespace * @param filters - Seq[ColumnFilter] - label filters of the query/lp - * @param optimizingRawMetric - Boolean - flag signifying if the metric getting optimized is a raw or agg metric */ - private def incrementHierarchicalQueryOptimizedCounter(filters: Seq[ColumnFilter], - optimizingRawMetric: Boolean): Unit = { + private def incrementHierarchicalQueryOptimizedCounter(filters: Seq[ColumnFilter]): Unit = { // track query optimized per workspace and namespace in the counter val metric_ws = LogicalPlan.getColumnValues(filters, TsCardinalities.LABEL_WORKSPACE) match { case Seq() => "" @@ -359,14 +295,9 @@ object HierarchicalQueryExperience extends StrictLogging { case Seq() => "" case ns => ns.head } - val metric_type = optimizingRawMetric match { - case true => "raw" - case false => "agg" - } hierarchicalQueryOptimizedCounter .withTag("metric_ws", metric_ws) .withTag("metric_ns", metric_ns) - .withTag("metric_type", metric_type) .increment() } @@ -407,10 +338,10 @@ object HierarchicalQueryExperience extends StrictLogging { // the includeTags. This requires the knowledge of all the tags/labels which are being published // for a metric. This info is not available during planning and hence we can't optimize this scenario. params match { - case IncludeAggRule( _, _, _) => + case IncludeAggRule( _, _) => // can't optimize this scenario as we don't have the required info at the planning stage false - case ExcludeAggRule(_, excludeTags, _) => + case ExcludeAggRule(_, excludeTags) => if (excludeTags.subsetOf(clause.labels.toSet)) { true } else { false } } diff --git a/query/src/test/scala/filodb/query/util/HierarchicalQueryExperienceSpec.scala b/query/src/test/scala/filodb/query/util/HierarchicalQueryExperienceSpec.scala index 6c19aa24e2..74e51bd72b 100644 --- a/query/src/test/scala/filodb/query/util/HierarchicalQueryExperienceSpec.scala +++ b/query/src/test/scala/filodb/query/util/HierarchicalQueryExperienceSpec.scala @@ -18,7 +18,7 @@ class HierarchicalQueryExperienceSpec extends AnyFunSpec with Matchers { it("getNextLevelAggregatedMetricName should return expected metric name") { - val params = IncludeAggRule("agg_2", Set("job", "instance"), "2") + val params = IncludeAggRule("agg_2", Set("job", "instance")) // Case 1: Should not update if metric doesn't have the aggregated metric identifier HierarchicalQueryExperience.getNextLevelAggregatedMetricName( @@ -29,13 +29,6 @@ class HierarchicalQueryExperienceSpec extends AnyFunSpec with Matchers { "metric1:::agg", ":::", params.metricSuffix) shouldEqual "metric1:::agg_2" } - it("getAggMetricNameForRawMetric should return expected metric name") { - val params = IncludeAggRule("agg_2", Set("job", "instance"), "2") - // Case 1: Should not update if metric doesn't have the aggregated metric identifier - HierarchicalQueryExperience.getAggMetricNameForRawMetric( - "metric1", ":::", params.metricSuffix) shouldEqual "metric1:::agg_2" - } - it("isParentPeriodicSeriesPlanAllowedForRawSeriesUpdateForHigherLevelAggregatedMetric return expected values") { HierarchicalQueryExperience.isParentPeriodicSeriesPlanAllowed( Seq("BinaryJoin", "Aggregate", "ScalarOperation")) shouldEqual true @@ -66,27 +59,27 @@ class HierarchicalQueryExperienceSpec extends AnyFunSpec with Matchers { it("should check if higher level aggregation is applicable with IncludeTags") { HierarchicalQueryExperience.isHigherLevelAggregationApplicable( - IncludeAggRule("agg_2", Set("tag1", "tag2"), "2"), Seq("tag1", "tag2", "_ws_", "_ns_", "_metric_")) shouldEqual true + IncludeAggRule("agg_2", Set("tag1", "tag2")), Seq("tag1", "tag2", "_ws_", "_ns_", "_metric_")) shouldEqual true HierarchicalQueryExperience.isHigherLevelAggregationApplicable( - IncludeAggRule("agg_2", Set("tag1", "tag2", "tag3"), "2"), Seq("tag1", "tag2", "_ws_", "_ns_", "__name__")) shouldEqual true + IncludeAggRule("agg_2", Set("tag1", "tag2", "tag3")), Seq("tag1", "tag2", "_ws_", "_ns_", "__name__")) shouldEqual true HierarchicalQueryExperience.isHigherLevelAggregationApplicable( - IncludeAggRule("agg_2", Set("tag1", "tag2", "tag3"), "2"), Seq("tag3", "tag4", "_ws_", "_ns_", "__name__")) shouldEqual false + IncludeAggRule("agg_2", Set("tag1", "tag2", "tag3")), Seq("tag3", "tag4", "_ws_", "_ns_", "__name__")) shouldEqual false } it("should check if higher level aggregation is applicable with ExcludeTags") { HierarchicalQueryExperience.isHigherLevelAggregationApplicable( - ExcludeAggRule("agg_2", Set("tag1", "tag2"), "2"),Seq("tag1", "tag2", "_ws_", "_ns_", "_metric_")) shouldEqual false + ExcludeAggRule("agg_2", Set("tag1", "tag2")),Seq("tag1", "tag2", "_ws_", "_ns_", "_metric_")) shouldEqual false HierarchicalQueryExperience.isHigherLevelAggregationApplicable( - ExcludeAggRule("agg_2", Set("tag1", "tag3"), "2"),Seq("tag1", "tag2", "_ws_", "_ns_", "_metric_")) shouldEqual false + ExcludeAggRule("agg_2", Set("tag1", "tag3")),Seq("tag1", "tag2", "_ws_", "_ns_", "_metric_")) shouldEqual false HierarchicalQueryExperience.isHigherLevelAggregationApplicable( - ExcludeAggRule("agg_2", Set("tag1", "tag2"), "2"),Seq("tag1", "tag2", "_ws_", "_ns_", "_metric_")) shouldEqual false + ExcludeAggRule("agg_2", Set("tag1", "tag2")),Seq("tag1", "tag2", "_ws_", "_ns_", "_metric_")) shouldEqual false HierarchicalQueryExperience.isHigherLevelAggregationApplicable( - ExcludeAggRule("agg_2", Set("tag3", "tag4"), "2"), Seq("tag1", "tag2", "_ws_", "_ns_", "_metric_")) shouldEqual true + ExcludeAggRule("agg_2", Set("tag3", "tag4")), Seq("tag1", "tag2", "_ws_", "_ns_", "_metric_")) shouldEqual true } it("getColumnsAfterFilteringOutDotStarRegexFilters should return as expected") { @@ -120,8 +113,8 @@ class HierarchicalQueryExperienceSpec extends AnyFunSpec with Matchers { } it("checkAggregateQueryEligibleForHigherLevelAggregatedMetric should increment counter if metric updated") { - val excludeRule = ExcludeAggRule("agg_2", Set("notAggTag1", "notAggTag2"), "2") - val params = HierarchicalQueryExperienceParams(":::", Map("agg" -> Set(excludeRule)), Map("metric1" -> Set(excludeRule))) + val excludeRule = ExcludeAggRule("agg_2", Set("notAggTag1", "notAggTag2")) + val params = HierarchicalQueryExperienceParams(":::", Map("agg" -> excludeRule)) Kamon.init() var counter = Kamon.counter("hierarchical-query-plans-optimized") @@ -132,10 +125,11 @@ class HierarchicalQueryExperienceSpec extends AnyFunSpec with Matchers { ColumnFilter("__name__", Equals("metric1:::agg")), ColumnFilter("_ws_", Equals("testws")), ColumnFilter("_ns_", Equals("testns")), - ColumnFilter("aggTag", Equals("value"))), false) + ColumnFilter("aggTag", Equals("value")))) updatedFilters.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] .shouldEqual("metric1:::agg_2") - counter.withTag("metric_ws", "testws").withTag("metric_ns", "testns").withTag("metric_type", "agg").value shouldEqual 1 + counter.withTag("metric_ws", "testws").withTag("metric_ns", "testns").value shouldEqual 1 + // CASE 2: Should not update if metric doesn't have the aggregated metric identifier // reset the counter @@ -146,23 +140,12 @@ class HierarchicalQueryExperienceSpec extends AnyFunSpec with Matchers { ColumnFilter("__name__", Equals("metric1:::agg")), ColumnFilter("_ws_", Equals("testws")), ColumnFilter("_ns_", Equals("testns")), - ColumnFilter("notAggTag1", Equals("value"))), false) // using exclude tag, so should not optimize + ColumnFilter("notAggTag1", Equals("value")))) // using exclude tag, so should not optimize updatedFilters.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] .shouldEqual("metric1:::agg") // count should not increment - counter.withTag("metric_ws", "testws").withTag("metric_ns", "testns").withTag("metric_type", "").value shouldEqual 0 + counter.withTag("metric_ws", "testws").withTag("metric_ns", "testns").value shouldEqual 0 - // CASE 3: Should update for raw metric optimization - counter = Kamon.counter("hierarchical-query-plans-optimized") - updatedFilters = HierarchicalQueryExperience.upsertMetricColumnFilterIfHigherLevelAggregationApplicable( - params, Seq( - ColumnFilter("__name__", Equals("metric1")), - ColumnFilter("_ws_", Equals("testws")), - ColumnFilter("_ns_", Equals("testns")), - ColumnFilter("aggTag", Equals("value"))), true) - updatedFilters.filter(x => x.column == "__name__").head.filter.valuesStrings.head.asInstanceOf[String] - .shouldEqual("metric1:::agg_2") - counter.withTag("metric_ws", "testws").withTag("metric_ns", "testns").withTag("metric_type", "raw").value shouldEqual 1 Kamon.stop() } }