From a5c0cc83b935a05ee2f23c92b1416c2633881886 Mon Sep 17 00:00:00 2001 From: TanviBhavsar Date: Tue, 16 Jun 2020 17:14:12 -0700 Subject: [PATCH 01/22] feat(query, coordinator): support querying across multiple clusters (#763) --- cli/src/main/scala/filodb.cli/CliMain.scala | 7 +- .../client/FiloKryoSerializers.scala | 26 +- .../client/Serializer.scala | 1 - .../HighAvailabilityPlanner.scala | 22 +- .../queryplanner/LogicalPlanUtils.scala | 76 ++-- .../queryplanner/LongTimeRangePlanner.scala | 8 +- .../queryplanner/MultiPartitionPlanner.scala | 185 +++++++++ .../queryplanner/SingleClusterPlanner.scala | 2 +- .../client/SerializationSpec.scala | 2 +- .../queryplanner/FailureProviderSpec.scala | 14 +- .../HighAvailabilityPlannerSpec.scala | 63 +-- .../MultiPartitionPlannerSpec.scala | 359 ++++++++++++++++++ .../queryplanner/ScalarQueriesSpec.scala | 6 +- .../SingleClusterPlannerSpec.scala | 2 +- .../SinglePartitionPlannerSpec.scala | 11 +- .../DownsampledTimeSeriesShard.scala | 10 +- .../DownsampledTimeSeriesStore.scala | 2 +- .../scala/filodb.core/memstore/MemStore.scala | 2 +- .../memstore/TimeSeriesMemStore.scala | 2 +- .../memstore/TimeSeriesShard.scala | 23 +- .../filodb.core/query/QueryContext.scala | 7 +- .../TimeSeriesMemStoreForMetadataSpec.scala | 26 +- .../filodb/http/PrometheusApiRoute.scala | 7 +- .../main/scala/filodb/query/LogicalPlan.scala | 24 +- .../scala/filodb/query/PromCirceSupport.scala | 1 + .../filodb/query/PromQueryResponse.scala | 7 +- .../filodb/query/exec/MetadataExecPlan.scala | 100 ++--- .../query/exec/MetadataRemoteExec.scala | 55 +++ .../scala/filodb/query/exec/PromQlExec.scala | 149 -------- .../filodb/query/exec/PromQlRemoteExec.scala | 193 ++++++++++ .../scala/filodb/query/LogicalPlanSpec.scala | 14 +- .../filodb/query/exec/MetadataExecSpec.scala | 4 +- ...cSpec.scala => PromQlRemoteExecSpec.scala} | 34 +- 33 files changed, 1046 insertions(+), 398 deletions(-) create mode 100644 coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala create mode 100644 coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala create mode 100644 query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala delete mode 100644 query/src/main/scala/filodb/query/exec/PromQlExec.scala create mode 100644 query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala rename query/src/test/scala/filodb/query/exec/{PromQlExecSpec.scala => PromQlRemoteExecSpec.scala} (54%) diff --git a/cli/src/main/scala/filodb.cli/CliMain.scala b/cli/src/main/scala/filodb.cli/CliMain.scala index c4aa69ca0f..f979073735 100644 --- a/cli/src/main/scala/filodb.cli/CliMain.scala +++ b/cli/src/main/scala/filodb.cli/CliMain.scala @@ -9,7 +9,6 @@ import scala.util.Try import com.opencsv.CSVWriter import com.quantifind.sumac.{ArgMain, FieldArgs} -import com.typesafe.config.ConfigFactory import monix.reactive.Observable import org.scalactic._ @@ -266,7 +265,7 @@ object CliMain extends ArgMain[Arguments] with FilodbClusterNode { def parseLabelValuesQuery(client: LocalClient, labelNames: Seq[String], constraints: Map[String, String], dataset: String, timeParams: TimeRangeParams, options: QOptions): Unit = { - val logicalPlan = LabelValues(labelNames, constraints, 3.days.toMillis) + val logicalPlan = LabelValues(labelNames, constraints, timeParams.start * 1000, timeParams.end * 1000) executeQuery2(client, dataset, logicalPlan, options, UnavailablePromQlQueryParams) } @@ -275,8 +274,8 @@ object CliMain extends ArgMain[Arguments] with FilodbClusterNode { options: QOptions): Unit = { val logicalPlan = Parser.queryRangeToLogicalPlan(query, timeParams) // Routing is not supported with CLI - executeQuery2(client, dataset, logicalPlan, options, PromQlQueryParams(ConfigFactory.empty, query,timeParams.start, timeParams.step, - timeParams.end)) + executeQuery2(client, dataset, logicalPlan, options, + PromQlQueryParams(query,timeParams.start, timeParams.step, timeParams.end)) } def promFilterToPartKeyBr(query: String, schemaName: String): Unit = { diff --git a/coordinator/src/main/scala/filodb.coordinator/client/FiloKryoSerializers.scala b/coordinator/src/main/scala/filodb.coordinator/client/FiloKryoSerializers.scala index 8f20fd3e6f..e6813ed027 100644 --- a/coordinator/src/main/scala/filodb.coordinator/client/FiloKryoSerializers.scala +++ b/coordinator/src/main/scala/filodb.coordinator/client/FiloKryoSerializers.scala @@ -2,11 +2,10 @@ package com.esotericsoftware.kryo.io import com.esotericsoftware.kryo.{Serializer => KryoSerializer} import com.esotericsoftware.kryo.Kryo -import com.typesafe.config.{ ConfigFactory, ConfigRenderOptions} import com.typesafe.scalalogging.StrictLogging import filodb.core.binaryrecord2.{RecordSchema => RecordSchema2} -import filodb.core.query.{ColumnInfo, PartitionInfo, PartitionRangeVectorKey, PromQlQueryParams} +import filodb.core.query.{ColumnInfo, PartitionInfo, PartitionRangeVectorKey} import filodb.memory.format._ // NOTE: This file has to be in the kryo namespace so we can use the require() method @@ -79,26 +78,3 @@ class PartitionInfoSerializer extends KryoSerializer[PartitionInfo] { output.writeInt(info.shardNo) } } - -class PromQlQueryParamsSerializer extends KryoSerializer[PromQlQueryParams] { - override def read(kryo: Kryo, input: Input, typ: Class[PromQlQueryParams]): PromQlQueryParams = { - val config = ConfigFactory.parseString(input.readString()) - val promQl = input.readString() - val start = input.readLong() - val step = input.readLong() - val end = input.readLong() - val spreadInt = input.readInt() - val spread = if (spreadInt == -1) None else Some(spreadInt) - val procFailure = input.readBoolean() - PromQlQueryParams(config, promQl, start, step, end, spread, procFailure) - } - override def write(kryo: Kryo, output: Output, promParam: PromQlQueryParams): Unit = { - output.writeString(promParam.config.root().render(ConfigRenderOptions.concise())) - output.writeString(promParam.promQl) - output.writeLong(promParam.startSecs) - output.writeLong(promParam.stepSecs) - output.writeLong(promParam.endSecs) - output.writeInt(promParam.spread.getOrElse(-1)) - output.writeBoolean(promParam.processFailure) - } -} \ No newline at end of file diff --git a/coordinator/src/main/scala/filodb.coordinator/client/Serializer.scala b/coordinator/src/main/scala/filodb.coordinator/client/Serializer.scala index 3dbfdde856..6733884c91 100644 --- a/coordinator/src/main/scala/filodb.coordinator/client/Serializer.scala +++ b/coordinator/src/main/scala/filodb.coordinator/client/Serializer.scala @@ -98,7 +98,6 @@ class KryoInit { kryo.register(classOf[QueryCommands.BadQuery]) kryo.register(classOf[QueryContext]) kryo.register(classOf[QueryCommands.FilteredPartitionQuery]) - kryo.register(classOf[PromQlQueryParams], new PromQlQueryParamsSerializer) } } diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/HighAvailabilityPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/HighAvailabilityPlanner.scala index faa3b0074a..aefbc8cefc 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/HighAvailabilityPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/HighAvailabilityPlanner.scala @@ -5,7 +5,7 @@ import com.typesafe.scalalogging.StrictLogging import filodb.core.DatasetRef import filodb.core.query.{PromQlQueryParams, QueryConfig, QueryContext} import filodb.query.LogicalPlan -import filodb.query.exec.{ExecPlan, InProcessPlanDispatcher, PromQlExec, StitchRvsExec} +import filodb.query.exec.{ExecPlan, InProcessPlanDispatcher, PromQlRemoteExec, StitchRvsExec} /** * HighAvailabilityPlanner responsible for using underlying local planner and FailureProvider @@ -23,9 +23,15 @@ class HighAvailabilityPlanner(dsRef: DatasetRef, failureProvider: FailureProvider, queryConfig: QueryConfig) extends QueryPlanner with StrictLogging { + import net.ceedubs.ficus.Ficus._ import LogicalPlanUtils._ import QueryFailureRoutingStrategy._ + val remoteHttpEndpoint: String = queryConfig.routingConfig.getString("remote.http.endpoint") + + val remoteHttpTimeoutMs: Long = + queryConfig.routingConfig.config.as[Option[Long]]("remote.http.timeout").getOrElse(60000) + /** * Converts Route objects returned by FailureProvider to ExecPlan */ @@ -43,24 +49,25 @@ class HighAvailabilityPlanner(dsRef: DatasetRef, // Offset logic is handled in ExecPlan localPlanner.materialize( copyWithUpdatedTimeRange(rootLogicalPlan, TimeRange(timeRange.startMs + offsetMs, - timeRange.endMs + offsetMs) , lookBackTime), qContext) + timeRange.endMs + offsetMs)), qContext) } case route: RemoteRoute => val timeRange = route.timeRange.get val queryParams = qContext.origQueryParams.asInstanceOf[PromQlQueryParams] // Divide by 1000 to convert millis to seconds. PromQL params are in seconds. - val promQlParams = PromQlQueryParams(queryConfig.routingConfig, queryParams.promQl, + val promQlParams = PromQlQueryParams(queryParams.promQl, (timeRange.startMs + offsetMs) / 1000, queryParams.stepSecs, (timeRange.endMs + offsetMs) / 1000, queryParams.spread, processFailure = false) logger.debug("PromQlExec params:" + promQlParams) - PromQlExec(qContext, InProcessPlanDispatcher, dsRef, promQlParams) + PromQlRemoteExec(remoteHttpEndpoint, remoteHttpTimeoutMs, + qContext, InProcessPlanDispatcher, dsRef, promQlParams) } } if (execPlans.size == 1) execPlans.head else StitchRvsExec(qContext, InProcessPlanDispatcher, - execPlans.sortWith((x, y) => !x.isInstanceOf[PromQlExec])) + execPlans.sortWith((x, y) => !x.isInstanceOf[PromQlRemoteExec])) // ^^ Stitch RemoteExec plan results with local using InProcessPlanDispatcher // Sort to move RemoteExec in end as it does not have schema @@ -70,7 +77,7 @@ class HighAvailabilityPlanner(dsRef: DatasetRef, // lazy because we want to fetch failures only if needed lazy val offsetMillis = LogicalPlanUtils.getOffsetMillis(logicalPlan) - lazy val periodicSeriesTime = getPeriodicSeriesTimeFromLogicalPlan(logicalPlan) + lazy val periodicSeriesTime = getTimeFromLogicalPlan(logicalPlan) lazy val periodicSeriesTimeWithOffset = TimeRange(periodicSeriesTime.startMs - offsetMillis, periodicSeriesTime.endMs - offsetMillis) lazy val lookBackTime = getLookBackMillis(logicalPlan) @@ -84,7 +91,8 @@ class HighAvailabilityPlanner(dsRef: DatasetRef, if (!logicalPlan.isRoutable || !tsdbQueryParams.isInstanceOf[PromQlQueryParams] || // We don't know the promql issued (unusual) (tsdbQueryParams.isInstanceOf[PromQlQueryParams] - && !tsdbQueryParams.asInstanceOf[PromQlQueryParams].processFailure) || // This is a query that was part of + && !tsdbQueryParams.asInstanceOf[PromQlQueryParams].processFailure) || // This is a query that was + // part of failure routing !hasSingleTimeRange(logicalPlan) || // Sub queries have different time ranges (unusual) failures.isEmpty) { // no failures in query time range localPlanner.materialize(logicalPlan, qContext) diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala index 92b2cd9642..4482404058 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala @@ -12,8 +12,8 @@ object LogicalPlanUtils { def hasSingleTimeRange(logicalPlan: LogicalPlan): Boolean = { logicalPlan match { case binaryJoin: BinaryJoin => - val lhsTime = getPeriodicSeriesTimeFromLogicalPlan(binaryJoin.lhs) - val rhsTime = getPeriodicSeriesTimeFromLogicalPlan(binaryJoin.rhs) + val lhsTime = getTimeFromLogicalPlan(binaryJoin.lhs) + val rhsTime = getTimeFromLogicalPlan(binaryJoin.rhs) (lhsTime.startMs == rhsTime.startMs) && (lhsTime.endMs == rhsTime.endMs) case _ => true } @@ -21,24 +21,40 @@ object LogicalPlanUtils { /** * Retrieve start and end time from LogicalPlan - * NOTE: Plan should be PeriodicSeriesPlan */ - def getPeriodicSeriesTimeFromLogicalPlan(logicalPlan: LogicalPlan): TimeRange = { + def getTimeFromLogicalPlan(logicalPlan: LogicalPlan): TimeRange = { logicalPlan match { case lp: PeriodicSeries => TimeRange(lp.startMs, lp.endMs) case lp: PeriodicSeriesWithWindowing => TimeRange(lp.startMs, lp.endMs) - case lp: ApplyInstantFunction => getPeriodicSeriesTimeFromLogicalPlan(lp.vectors) - case lp: Aggregate => getPeriodicSeriesTimeFromLogicalPlan(lp.vectors) + case lp: ApplyInstantFunction => getTimeFromLogicalPlan(lp.vectors) + case lp: Aggregate => getTimeFromLogicalPlan(lp.vectors) case lp: BinaryJoin => // can assume lhs & rhs have same time - getPeriodicSeriesTimeFromLogicalPlan(lp.lhs) - case lp: ScalarVectorBinaryOperation => getPeriodicSeriesTimeFromLogicalPlan(lp.vector) - case lp: ApplyMiscellaneousFunction => getPeriodicSeriesTimeFromLogicalPlan(lp.vectors) - case lp: ApplySortFunction => getPeriodicSeriesTimeFromLogicalPlan(lp.vectors) - case lp: ScalarVaryingDoublePlan => getPeriodicSeriesTimeFromLogicalPlan(lp.vectors) + getTimeFromLogicalPlan(lp.lhs) + case lp: ScalarVectorBinaryOperation => getTimeFromLogicalPlan(lp.vector) + case lp: ApplyMiscellaneousFunction => getTimeFromLogicalPlan(lp.vectors) + case lp: ApplySortFunction => getTimeFromLogicalPlan(lp.vectors) + case lp: ScalarVaryingDoublePlan => getTimeFromLogicalPlan(lp.vectors) case lp: ScalarTimeBasedPlan => TimeRange(lp.rangeParams.startSecs, lp.rangeParams.endSecs) - case lp: VectorPlan => getPeriodicSeriesTimeFromLogicalPlan(lp.scalars) - case lp: ApplyAbsentFunction => getPeriodicSeriesTimeFromLogicalPlan(lp.vectors) - case _ => throw new BadQueryException(s"Invalid logical plan") + case lp: VectorPlan => getTimeFromLogicalPlan(lp.scalars) + case lp: ApplyAbsentFunction => getTimeFromLogicalPlan(lp.vectors) + case lp: RawSeries => lp.rangeSelector match { + case i: IntervalSelector => TimeRange(i.from, i.to) + case _ => throw new BadQueryException(s"Invalid logical plan") + } + case _ => throw new BadQueryException(s"Invalid logical plan ${logicalPlan}") + } + } + + /** + * Used to change start and end time(TimeRange) of LogicalPlan + * NOTE: Plan should be PeriodicSeriesPlan + */ + def copyLogicalPlanWithUpdatedTimeRange(logicalPlan: LogicalPlan, + timeRange: TimeRange): LogicalPlan = { + logicalPlan match { + case lp: PeriodicSeriesPlan => copyWithUpdatedTimeRange(lp, timeRange) + case lp: RawSeriesLikePlan => copyNonPeriodicWithUpdatedTimeRange(lp, timeRange) + case _ => throw new UnsupportedOperationException("Logical plan not supported for copy") } } @@ -47,30 +63,29 @@ object LogicalPlanUtils { * NOTE: Plan should be PeriodicSeriesPlan */ def copyWithUpdatedTimeRange(logicalPlan: LogicalPlan, - timeRange: TimeRange, - lookBackTime: Long): PeriodicSeriesPlan = { + timeRange: TimeRange): PeriodicSeriesPlan = { logicalPlan match { case lp: PeriodicSeries => lp.copy(startMs = timeRange.startMs, endMs = timeRange.endMs, - rawSeries = copyNonPeriodicWithUpdatedTimeRange(lp.rawSeries, timeRange, - lookBackTime).asInstanceOf[RawSeries]) + rawSeries = copyNonPeriodicWithUpdatedTimeRange(lp.rawSeries, timeRange) + .asInstanceOf[RawSeries]) case lp: PeriodicSeriesWithWindowing => lp.copy(startMs = timeRange.startMs, endMs = timeRange.endMs, - series = copyNonPeriodicWithUpdatedTimeRange(lp.series, timeRange, - lookBackTime)) - case lp: ApplyInstantFunction => lp.copy(vectors = copyWithUpdatedTimeRange(lp.vectors, timeRange, lookBackTime)) + series = copyNonPeriodicWithUpdatedTimeRange(lp.series, + timeRange)) + case lp: ApplyInstantFunction => lp.copy(vectors = copyWithUpdatedTimeRange(lp.vectors, timeRange)) - case lp: Aggregate => lp.copy(vectors = copyWithUpdatedTimeRange(lp.vectors, timeRange, lookBackTime)) + case lp: Aggregate => lp.copy(vectors = copyWithUpdatedTimeRange(lp.vectors, timeRange)) - case lp: BinaryJoin => lp.copy(lhs = copyWithUpdatedTimeRange(lp.lhs, timeRange, lookBackTime), - rhs = copyWithUpdatedTimeRange(lp.rhs, timeRange, lookBackTime)) + case lp: BinaryJoin => lp.copy(lhs = copyWithUpdatedTimeRange(lp.lhs, timeRange), + rhs = copyWithUpdatedTimeRange(lp.rhs, timeRange)) case lp: ScalarVectorBinaryOperation => - lp.copy(vector = copyWithUpdatedTimeRange(lp.vector, timeRange, lookBackTime)) + lp.copy(vector = copyWithUpdatedTimeRange(lp.vector, timeRange)) case lp: ApplyMiscellaneousFunction => - lp.copy(vectors = copyWithUpdatedTimeRange(lp.vectors, timeRange, lookBackTime)) + lp.copy(vectors = copyWithUpdatedTimeRange(lp.vectors, timeRange)) - case lp: ApplySortFunction => lp.copy(vectors = copyWithUpdatedTimeRange(lp.vectors, timeRange, lookBackTime)) + case lp: ApplySortFunction => lp.copy(vectors = copyWithUpdatedTimeRange(lp.vectors, timeRange)) case _ => throw new UnsupportedOperationException("Logical plan not supported for copy") } @@ -79,16 +94,15 @@ object LogicalPlanUtils { /** * Used to change rangeSelector of RawSeriesLikePlan */ - private def copyNonPeriodicWithUpdatedTimeRange(plan: RawSeriesLikePlan, - timeRange: TimeRange, - lookBackTime: Long): RawSeriesLikePlan = { + private def copyNonPeriodicWithUpdatedTimeRange(plan: LogicalPlan, + timeRange: TimeRange): RawSeriesLikePlan = { plan match { case rs: RawSeries => rs.rangeSelector match { case is: IntervalSelector => rs.copy(rangeSelector = is.copy(timeRange.startMs, timeRange.endMs)) case _ => throw new UnsupportedOperationException("Copy supported only for IntervalSelector") } case p: ApplyInstantFunctionRaw => - p.copy(vectors = copyNonPeriodicWithUpdatedTimeRange(p.vectors, timeRange, lookBackTime) + p.copy(vectors = copyNonPeriodicWithUpdatedTimeRange(p.vectors, timeRange) .asInstanceOf[RawSeries]) case _ => throw new UnsupportedOperationException("Copy supported only for RawSeries") } diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LongTimeRangePlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LongTimeRangePlanner.scala index 8d34a14146..e7a4492786 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LongTimeRangePlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LongTimeRangePlanner.scala @@ -48,7 +48,7 @@ class LongTimeRangePlanner(rawClusterPlanner: QueryPlanner, logicalPlan } else { copyWithUpdatedTimeRange(logicalPlan, - TimeRange(p.startMs, latestDownsampleTimestampFn + offsetMillis), lookbackMs) + TimeRange(p.startMs, latestDownsampleTimestampFn + offsetMillis)) } downsampleClusterPlanner.materialize(downsampleLp, qContext) } else { @@ -58,14 +58,14 @@ class LongTimeRangePlanner(rawClusterPlanner: QueryPlanner, val firstInstantInRaw = lastDownsampleInstant + p.stepMs val downsampleLp = copyWithUpdatedTimeRange(logicalPlan, - TimeRange(p.startMs, lastDownsampleInstant), - lookbackMs) + TimeRange(p.startMs, lastDownsampleInstant)) val downsampleEp = downsampleClusterPlanner.materialize(downsampleLp, qContext) - val rawLp = copyWithUpdatedTimeRange(logicalPlan, TimeRange(firstInstantInRaw, p.endMs), lookbackMs) + val rawLp = copyWithUpdatedTimeRange(logicalPlan, TimeRange(firstInstantInRaw, p.endMs)) val rawEp = rawClusterPlanner.materialize(rawLp, qContext) StitchRvsExec(qContext, stitchDispatcher, Seq(rawEp, downsampleEp)) } + // Metadata query not supported for downsample cluster case _ => rawClusterPlanner.materialize(logicalPlan, qContext) } } diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala new file mode 100644 index 0000000000..56536b730b --- /dev/null +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala @@ -0,0 +1,185 @@ +package filodb.coordinator.queryplanner + +import filodb.coordinator.queryplanner.LogicalPlanUtils._ +import filodb.core.metadata.Dataset +import filodb.core.query.{PromQlQueryParams, QueryConfig, QueryContext} +import filodb.query.{BinaryJoin, LabelValues, LogicalPlan, RawSeriesLikePlan, SeriesKeysByFilters} +import filodb.query.exec._ + +case class PartitionAssignment(partitionName: String, endPoint: String, timeRange: TimeRange) + +trait PartitionLocationProvider { + + def getPartitions(routingKey: Map[String, String], timeRange: TimeRange): List[PartitionAssignment] + def getAuthorizedPartitions(timeRange: TimeRange): List[PartitionAssignment] +} + +class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider, + localPartitionPlanner: QueryPlanner, + localPartitionName: String, + dataset: Dataset, + queryConfig: QueryConfig) extends QueryPlanner { + + import net.ceedubs.ficus.Ficus._ + + val remoteHttpTimeoutMs: Long = + queryConfig.routingConfig.config.as[Option[Long]]("remote.http.timeout").getOrElse(60000) + + override def materialize(logicalPlan: LogicalPlan, qContext: QueryContext): ExecPlan = { + + val tsdbQueryParams = qContext.origQueryParams + + if(!tsdbQueryParams.isInstanceOf[PromQlQueryParams] || // We don't know the promql issued (unusual) + (tsdbQueryParams.isInstanceOf[PromQlQueryParams] + && !tsdbQueryParams.asInstanceOf[PromQlQueryParams].processMultiPartition)) // Query was part of routing + localPartitionPlanner.materialize(logicalPlan, qContext) + + else logicalPlan match { + case lp: BinaryJoin => materializeBinaryJoin(lp, qContext) + case lp: LabelValues => materializeLabelValues(lp, qContext) + case lp: SeriesKeysByFilters => materializeSeriesKeysFilters(lp, qContext) + case _ => materializeSimpleQuery(logicalPlan, qContext) + + } + } + + private def getRoutingKeys(logicalPlan: LogicalPlan) = dataset.options.nonMetricShardColumns + .map(x => (x, LogicalPlan.getLabelValueFromLogicalPlan(logicalPlan, x))) + + private def generateRemoteExecParams(queryParams: PromQlQueryParams, startMs: Long, endMs: Long) = { + PromQlQueryParams(queryParams.promQl, startMs / 1000, queryParams.stepSecs, endMs / 1000, queryParams.spread, + queryParams.remoteQueryPath, queryParams.processFailure, processMultiPartition = false, queryParams.verbose) + } + + /** + * + * @param routingKeys Non Metric ShardColumns of dataset and value in logicalPlan + * @param queryParams PromQlQueryParams having query details + * @param logicalPlan Logical plan + */ + private def partitionUtil(routingKeys: Seq[(String, Option[scala.Seq[String]])], queryParams: PromQlQueryParams, + logicalPlan: LogicalPlan) = { + val routingKeyMap = routingKeys.map(x => (x._1, x._2.get.head)).toMap + val offsetMs = LogicalPlanUtils.getOffsetMillis(logicalPlan) + val periodicSeriesTimeWithOffset = TimeRange((queryParams.startSecs * 1000) - offsetMs, + (queryParams.endSecs * 1000) - offsetMs) + val lookBackMs = getLookBackMillis(logicalPlan) + + // Time at which raw data would be retrieved which is used to get partition assignments. + // It should have time with offset and lookback as we need raw data at time including offset and lookback. + val queryTimeRange = TimeRange(periodicSeriesTimeWithOffset.startMs - lookBackMs, + periodicSeriesTimeWithOffset.endMs) + + val partitions = partitionLocationProvider.getPartitions(routingKeyMap, queryTimeRange). + sortBy(_.timeRange.startMs) + + (partitions, lookBackMs, offsetMs) + } + + /** + * Materialize all queries except Binary Join and Metadata + */ + def materializeSimpleQuery(logicalPlan: LogicalPlan, qContext: QueryContext): ExecPlan = { + + val routingKeys = getRoutingKeys(logicalPlan) + if (routingKeys.forall(_._2.isEmpty)) localPartitionPlanner.materialize(logicalPlan, qContext) + else { + val queryParams = qContext.origQueryParams.asInstanceOf[PromQlQueryParams] + val stepMs = queryParams.stepSecs * 1000 + val isInstantQuery: Boolean = if (queryParams.startSecs == queryParams.endSecs) true else false + + val (partitions, lookBackMs, offsetMs) = partitionUtil(routingKeys, queryParams, logicalPlan) + var prevPartitionStart = queryParams.startSecs * 1000 + val execPlans = partitions.zipWithIndex.map { case (p, i) => + // First partition should start from query start time + // No need to calculate time according to step for instant queries + val startMs = if (i == 0 || isInstantQuery) queryParams.startSecs * 1000 + else { + // Lookback not supported across partitions + val numStepsInPrevPartition = (p.timeRange.startMs - prevPartitionStart + lookBackMs) / stepMs + val lastPartitionInstant = prevPartitionStart + numStepsInPrevPartition * stepMs + lastPartitionInstant + stepMs + } + prevPartitionStart = startMs + val endMs = if (isInstantQuery) queryParams.endSecs * 1000 else p.timeRange.endMs + offsetMs + if (p.partitionName.equals(localPartitionName)) + localPartitionPlanner.materialize( + copyLogicalPlanWithUpdatedTimeRange(logicalPlan, TimeRange(startMs, endMs)), qContext) + else { + val httpEndpoint = p.endPoint + queryParams.remoteQueryPath.getOrElse("") + PromQlRemoteExec(httpEndpoint, remoteHttpTimeoutMs, qContext, InProcessPlanDispatcher, dataset.ref, + generateRemoteExecParams(queryParams, startMs, endMs), logicalPlan.isInstanceOf[RawSeriesLikePlan]) + } + } + if (execPlans.size == 1) execPlans.head + else StitchRvsExec(qContext, InProcessPlanDispatcher, + execPlans.sortWith((x, y) => !x.isInstanceOf[PromQlRemoteExec])) + // ^^ Stitch RemoteExec plan results with local using InProcessPlanDispatcher + // Sort to move RemoteExec in end as it does not have schema + } + } + + def materializeBinaryJoin(logicalPlan: LogicalPlan, qContext: QueryContext): ExecPlan = { + + val routingKeys = getRoutingKeys(logicalPlan) + if (routingKeys.forall(_._2.isEmpty)) localPartitionPlanner.materialize(logicalPlan, qContext) + else { + val queryParams = qContext.origQueryParams.asInstanceOf[PromQlQueryParams] + val partitions = partitionUtil(routingKeys, queryParams, logicalPlan)._1 + val partitionName = partitions.head.partitionName + + // Binary Join supported only for single partition now + if (partitions.forall(_.partitionName.equals((partitionName)))) { + if (partitionName.equals(localPartitionName)) localPartitionPlanner.materialize(logicalPlan, qContext) + else { + val httpEndpoint = partitions.head.endPoint + queryParams.remoteQueryPath.getOrElse("") + PromQlRemoteExec(httpEndpoint, remoteHttpTimeoutMs, qContext, InProcessPlanDispatcher, dataset.ref, + generateRemoteExecParams(queryParams, queryParams.startSecs * 1000, queryParams.endSecs * 1000), + logicalPlan.isInstanceOf[RawSeriesLikePlan]) + } + } + else throw new UnsupportedOperationException("Binary Join across multiple partitions not supported") + } + } + + def materializeSeriesKeysFilters(lp: SeriesKeysByFilters, qContext: QueryContext): ExecPlan = { + val queryParams = qContext.origQueryParams.asInstanceOf[PromQlQueryParams] + val partitions = partitionLocationProvider.getAuthorizedPartitions( + TimeRange(queryParams.startSecs * 1000, queryParams.endSecs * 1000)) + val execPlans = partitions.map { p => + if (p.partitionName.equals(localPartitionName)) + localPartitionPlanner.materialize(lp.copy(startMs = p.timeRange.startMs, endMs = p.timeRange.endMs), qContext) + else + createMetadataRemoteExec(qContext, queryParams, p, Map("match[]" -> queryParams.promQl)) + } + if (execPlans.size == 1) execPlans.head + else PartKeysDistConcatExec(qContext, InProcessPlanDispatcher, + execPlans.sortWith((x, y) => !x.isInstanceOf[MetadataRemoteExec])) + } + + def materializeLabelValues(lp: LabelValues, qContext: QueryContext): ExecPlan = { + val queryParams = qContext.origQueryParams.asInstanceOf[PromQlQueryParams] + val partitions = partitionLocationProvider.getAuthorizedPartitions( + TimeRange(queryParams.startSecs * 1000, queryParams.endSecs * 1000)) + val execPlans = partitions.map { p => + if (p.partitionName.equals(localPartitionName)) + localPartitionPlanner.materialize(lp.copy(startMs = p.timeRange.startMs, endMs = p.timeRange.endMs), qContext) + else + createMetadataRemoteExec(qContext, queryParams, p, + Map("filter" -> lp.labelConstraints.map{case (k, v) => k + "=" + v}.mkString(","), + "labels" -> lp.labelNames.mkString(","))) + } + if (execPlans.size == 1) execPlans.head + else LabelValuesDistConcatExec(qContext, InProcessPlanDispatcher, + execPlans.sortWith((x, y) => !x.isInstanceOf[MetadataRemoteExec])) + } + + private def createMetadataRemoteExec(qContext: QueryContext, queryParams: PromQlQueryParams, + partitionAssignment: PartitionAssignment, urlParams: Map[String, String]) = { + val finalQueryParams = generateRemoteExecParams( + queryParams, partitionAssignment.timeRange.startMs, partitionAssignment.timeRange.endMs) + val httpEndpoint = partitionAssignment.endPoint + finalQueryParams.remoteQueryPath.getOrElse("") + MetadataRemoteExec(httpEndpoint, remoteHttpTimeoutMs, + urlParams, qContext, InProcessPlanDispatcher, dataset.ref, finalQueryParams) + } +} diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala index b75bbaaaae..7d83002b1d 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala @@ -393,7 +393,7 @@ class SingleClusterPlanner(dsRef: DatasetRef, } val metaExec = shardsToHit.map { shard => val dispatcher = dispatcherForShard(shard) - exec.LabelValuesExec(qContext, dispatcher, dsRef, shard, filters, labelNames, lp.lookbackTimeMs) + exec.LabelValuesExec(qContext, dispatcher, dsRef, shard, filters, labelNames, lp.startMs, lp.endMs) } PlanResult(metaExec, false) } diff --git a/coordinator/src/test/scala/filodb.coordinator/client/SerializationSpec.scala b/coordinator/src/test/scala/filodb.coordinator/client/SerializationSpec.scala index 952bc3142a..05df4f7e2a 100644 --- a/coordinator/src/test/scala/filodb.coordinator/client/SerializationSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/client/SerializationSpec.scala @@ -330,7 +330,7 @@ class SerializationSpec extends ActorTest(SerializationSpecConfig.getNewSystem) val logicalPlan = Parser.queryRangeToLogicalPlan( s"""http_request_duration_seconds_bucket{job="prometheus",$shardKeyStr}""", qParams) - val param = PromQlQueryParams(ConfigFactory.empty(), "test", 1000, 200, 5000) + val param = PromQlQueryParams("test", 1000, 200, 5000) val execPlan = engine.materialize(logicalPlan, QueryContext(origQueryParams = param, spreadOverride = Some(new StaticSpreadProvider(SpreadChange(0, 0))))) roundTrip(execPlan) shouldEqual execPlan diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/FailureProviderSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/FailureProviderSpec.scala index 6f17868cc8..5e8fb53e3d 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/FailureProviderSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/FailureProviderSpec.scala @@ -45,7 +45,7 @@ class FailureProviderSpec extends FunSpec with Matchers { it("should extract time from logical plan") { hasSingleTimeRange(summed1) shouldEqual (true) - val timeRange = getPeriodicSeriesTimeFromLogicalPlan(summed1) + val timeRange = getTimeFromLogicalPlan(summed1) timeRange.startMs shouldEqual (100000) timeRange.endMs shouldEqual (150000) @@ -54,10 +54,10 @@ class FailureProviderSpec extends FunSpec with Matchers { it("should update time in logical plan") { val expectedRaw = RawSeries(rangeSelector = IntervalSelector(20000, 30000), filters = f1, columns = Seq("value")) - val updatedTimeLogicalPlan = copyWithUpdatedTimeRange(summed1, TimeRange(20000, 30000), 0) + val updatedTimeLogicalPlan = copyWithUpdatedTimeRange(summed1, TimeRange(20000, 30000)) - getPeriodicSeriesTimeFromLogicalPlan(updatedTimeLogicalPlan).startMs shouldEqual (20000) - getPeriodicSeriesTimeFromLogicalPlan(updatedTimeLogicalPlan).endMs shouldEqual (30000) + getTimeFromLogicalPlan(updatedTimeLogicalPlan).startMs shouldEqual (20000) + getTimeFromLogicalPlan(updatedTimeLogicalPlan).endMs shouldEqual (30000) updatedTimeLogicalPlan.isInstanceOf[Aggregate] shouldEqual (true) val aggregate = updatedTimeLogicalPlan.asInstanceOf[Aggregate] @@ -160,10 +160,10 @@ class FailureProviderSpec extends FunSpec with Matchers { val expectedRaw = RawSeries(rangeSelector = IntervalSelector(20000, 30000), filters = f1, columns = Seq("value"), Some(100), None) - val updatedTimeLogicalPlan = copyWithUpdatedTimeRange(summed, TimeRange(20000, 30000), 100) + val updatedTimeLogicalPlan = copyWithUpdatedTimeRange(summed, TimeRange(20000, 30000)) - getPeriodicSeriesTimeFromLogicalPlan(updatedTimeLogicalPlan).startMs shouldEqual (20000) - getPeriodicSeriesTimeFromLogicalPlan(updatedTimeLogicalPlan).endMs shouldEqual (30000) + getTimeFromLogicalPlan(updatedTimeLogicalPlan).startMs shouldEqual (20000) + getTimeFromLogicalPlan(updatedTimeLogicalPlan).endMs shouldEqual (30000) updatedTimeLogicalPlan.isInstanceOf[Aggregate] shouldEqual (true) val aggregate = updatedTimeLogicalPlan.asInstanceOf[Aggregate] diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/HighAvailabilityPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/HighAvailabilityPlannerSpec.scala index cd88071a23..dad6d9c46a 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/HighAvailabilityPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/HighAvailabilityPlannerSpec.scala @@ -29,7 +29,8 @@ class HighAvailabilityPlannerSpec extends FunSpec with Matchers { private val dsRef = dataset.ref private val schemas = Schemas(dataset.schema) - private val routingConfigString = "routing {\n buddy {\n http {\n timeout = 10.seconds\n }\n }\n}" + private val routingConfigString = "routing {\n remote {\n " + + " http {\n endpoint = localhost\n timeout = 10000\n }\n }\n}" private val routingConfig = ConfigFactory.parseString(routingConfigString) @@ -47,7 +48,7 @@ class HighAvailabilityPlannerSpec extends FunSpec with Matchers { ColumnFilter("job", Filter.Equals("myService")), ColumnFilter("le", Filter.Equals("0.3"))) - private val promQlQueryParams = PromQlQueryParams(ConfigFactory.empty, "sum(heap_usage)", 100, 1, 1000, None) + private val promQlQueryParams = PromQlQueryParams("sum(heap_usage)", 100, 1, 1000) val localPlanner = new SingleClusterPlanner(dsRef, schemas, mapperRef, earliestRetainedTimestampFn = 0, queryConfig) @@ -109,19 +110,19 @@ class HighAvailabilityPlannerSpec extends FunSpec with Matchers { val execPlan = engine.materialize(summed, QueryContext(origQueryParams = promQlQueryParams)) - execPlan.isInstanceOf[PromQlExec] shouldEqual (true) - execPlan.asInstanceOf[PromQlExec].params.startSecs shouldEqual(from/1000) - execPlan.asInstanceOf[PromQlExec].params.endSecs shouldEqual(to/1000) + execPlan.isInstanceOf[PromQlRemoteExec] shouldEqual (true) + execPlan.asInstanceOf[PromQlRemoteExec].params.startSecs shouldEqual(from/1000) + execPlan.asInstanceOf[PromQlRemoteExec].params.endSecs shouldEqual(to/1000) } - it("should generate RemotExecPlan with RawSeries time according to lookBack") { + it("should generate RemoteExecPlan with RawSeries time according to lookBack") { val to = 2000000 val from = 1000000 val intervalSelector = IntervalSelector(from, to) // Lookback of 50000 val raw = RawSeries(rangeSelector = intervalSelector, filters = f1, columns = Seq("value"), Some(50000)) val windowed = PeriodicSeriesWithWindowing(raw, from, 100, to, 5000, RangeFunctionId.Rate) val summed = Aggregate(AggregationOperator.Sum, windowed, Nil, Seq("job")) - val promQlQueryParams = PromQlQueryParams(ConfigFactory.empty, "", from/1000, 1, to/1000, None) + val promQlQueryParams = PromQlQueryParams("", from/1000, 1, to/1000) val failureProvider = new FailureProvider { override def getFailures(datasetRef: DatasetRef, queryTimeRange: TimeRange): Seq[FailureTimeRange] = { @@ -140,10 +141,10 @@ class HighAvailabilityPlannerSpec extends FunSpec with Matchers { val stitchRvsExec = execPlan.asInstanceOf[StitchRvsExec] stitchRvsExec.children.size shouldEqual (2) stitchRvsExec.children(0).isInstanceOf[ReduceAggregateExec] shouldEqual (true) - stitchRvsExec.children(1).isInstanceOf[PromQlExec] shouldEqual (true) + stitchRvsExec.children(1).isInstanceOf[PromQlRemoteExec] shouldEqual (true) val child1 = stitchRvsExec.children(0).asInstanceOf[ReduceAggregateExec] - val child2 = stitchRvsExec.children(1).asInstanceOf[PromQlExec] + val child2 = stitchRvsExec.children(1).asInstanceOf[PromQlRemoteExec] child1.children.length shouldEqual (2) //default spread is 1 so 2 shards @@ -184,9 +185,9 @@ class HighAvailabilityPlannerSpec extends FunSpec with Matchers { val execPlan = engine.materialize(summed, QueryContext(origQueryParams = promQlQueryParams)) - execPlan.isInstanceOf[PromQlExec] shouldEqual (true) - execPlan.asInstanceOf[PromQlExec].params.startSecs shouldEqual(from/1000) - execPlan.asInstanceOf[PromQlExec].params.endSecs shouldEqual(to/1000) + execPlan.isInstanceOf[PromQlRemoteExec] shouldEqual (true) + execPlan.asInstanceOf[PromQlRemoteExec].params.startSecs shouldEqual(from/1000) + execPlan.asInstanceOf[PromQlRemoteExec].params.endSecs shouldEqual(to/1000) } it("should generate only PromQlExec when local failure timerange coincide with query time range") { @@ -208,9 +209,9 @@ class HighAvailabilityPlannerSpec extends FunSpec with Matchers { val execPlan = engine.materialize(summed, QueryContext(origQueryParams = promQlQueryParams)) - execPlan.isInstanceOf[PromQlExec] shouldEqual (true) - execPlan.asInstanceOf[PromQlExec].params.startSecs shouldEqual(from/1000) - execPlan.asInstanceOf[PromQlExec].params.endSecs shouldEqual(to/1000) + execPlan.isInstanceOf[PromQlRemoteExec] shouldEqual (true) + execPlan.asInstanceOf[PromQlRemoteExec].params.startSecs shouldEqual(from/1000) + execPlan.asInstanceOf[PromQlRemoteExec].params.endSecs shouldEqual(to/1000) } it("should generate only PromQlExec when local failure starts before query end time and ends after query end time") { @@ -232,9 +233,9 @@ class HighAvailabilityPlannerSpec extends FunSpec with Matchers { val execPlan = engine.materialize(summed, QueryContext(origQueryParams = promQlQueryParams)) - execPlan.isInstanceOf[PromQlExec] shouldEqual (true) - execPlan.asInstanceOf[PromQlExec].params.startSecs shouldEqual(from/1000) - execPlan.asInstanceOf[PromQlExec].params.endSecs shouldEqual(to/1000) + execPlan.isInstanceOf[PromQlRemoteExec] shouldEqual (true) + execPlan.asInstanceOf[PromQlRemoteExec].params.startSecs shouldEqual(from/1000) + execPlan.asInstanceOf[PromQlRemoteExec].params.endSecs shouldEqual(to/1000) } it("should generate PromQlExecPlan and LocalPlan with RawSeries time according to lookBack and step") { @@ -246,7 +247,7 @@ class HighAvailabilityPlannerSpec extends FunSpec with Matchers { val raw = RawSeries(rangeSelector = intervalSelector, filters = f1, columns = Seq("value")) val windowed = PeriodicSeriesWithWindowing(raw, from * 1000, step * 1000, to * 1000, 5000, RangeFunctionId.Rate) val summed = Aggregate(AggregationOperator.Sum, windowed, Nil, Seq("job")) - val promQlQueryParams = PromQlQueryParams(ConfigFactory.empty, "dummy query", from, step, to, None) + val promQlQueryParams = PromQlQueryParams("dummy query", from, step, to) val failureProvider = new FailureProvider { override def getFailures(datasetRef: DatasetRef, queryTimeRange: TimeRange): Seq[FailureTimeRange] = { @@ -266,10 +267,10 @@ class HighAvailabilityPlannerSpec extends FunSpec with Matchers { val stitchRvsExec = execPlan.asInstanceOf[StitchRvsExec] stitchRvsExec.children.size shouldEqual 2 stitchRvsExec.children(0).isInstanceOf[ReduceAggregateExec] shouldEqual (true) - stitchRvsExec.children(1).isInstanceOf[PromQlExec] shouldEqual (true) + stitchRvsExec.children(1).isInstanceOf[PromQlRemoteExec] shouldEqual (true) val child1 = stitchRvsExec.children(0).asInstanceOf[ReduceAggregateExec] - val child2 = stitchRvsExec.children(1).asInstanceOf[PromQlExec] + val child2 = stitchRvsExec.children(1).asInstanceOf[PromQlRemoteExec] child1.children.length shouldEqual 2 //default spread is 1 so 2 shards @@ -301,7 +302,7 @@ class HighAvailabilityPlannerSpec extends FunSpec with Matchers { val raw = RawSeries(rangeSelector = intervalSelector, filters = f1, columns = Seq("value")) val windowed = PeriodicSeriesWithWindowing(raw, from * 1000, step * 1000, to * 1000, 5000, RangeFunctionId.Rate) val summed = Aggregate(AggregationOperator.Sum, windowed, Nil, Seq("job")) - val promQlQueryParams = PromQlQueryParams(ConfigFactory.empty, "dummy query", from, step, to, None) + val promQlQueryParams = PromQlQueryParams("dummy query", from, step, to) val failureProvider = new FailureProvider { override def getFailures(datasetRef: DatasetRef, queryTimeRange: TimeRange): Seq[FailureTimeRange] = { @@ -315,9 +316,9 @@ class HighAvailabilityPlannerSpec extends FunSpec with Matchers { val execPlan = engine.materialize(summed, QueryContext(origQueryParams = promQlQueryParams)) - execPlan.isInstanceOf[PromQlExec] shouldEqual true + execPlan.isInstanceOf[PromQlRemoteExec] shouldEqual true - val child = execPlan.asInstanceOf[PromQlExec] + val child = execPlan.asInstanceOf[PromQlRemoteExec] child.params.startSecs shouldEqual 900 child.params.endSecs shouldEqual 1980 @@ -334,7 +335,7 @@ class HighAvailabilityPlannerSpec extends FunSpec with Matchers { val raw = RawSeries(rangeSelector = intervalSelector, filters = f1, columns = Seq("value")) val windowed = PeriodicSeriesWithWindowing(raw, from * 1000, step * 1000, to * 1000, 5000, RangeFunctionId.Rate) val summed = Aggregate(AggregationOperator.Sum, windowed, Nil, Seq("job")) - val promQlQueryParams = PromQlQueryParams(ConfigFactory.empty, "dummy query", from, step, to, None) + val promQlQueryParams = PromQlQueryParams("dummy query", from, step, to) val failureProvider = new FailureProvider { override def getFailures(datasetRef: DatasetRef, queryTimeRange: TimeRange): Seq[FailureTimeRange] = { @@ -373,7 +374,7 @@ class HighAvailabilityPlannerSpec extends FunSpec with Matchers { val raw = RawSeries(rangeSelector = intervalSelector, filters = f1, columns = Seq("value")) val windowed = PeriodicSeriesWithWindowing(raw, from * 1000, step * 1000, to * 1000, 5000, RangeFunctionId.Rate) val summed = Aggregate(AggregationOperator.Sum, windowed, Nil, Seq("job")) - val promQlQueryParams = PromQlQueryParams(ConfigFactory.empty, "dummy query", from, step, to, None) + val promQlQueryParams = PromQlQueryParams("dummy query", from, step, to) val failureProvider = new FailureProvider { override def getFailures(datasetRef: DatasetRef, queryTimeRange: TimeRange): Seq[FailureTimeRange] = { @@ -387,9 +388,9 @@ class HighAvailabilityPlannerSpec extends FunSpec with Matchers { val execPlan = engine.materialize(summed, QueryContext(origQueryParams = promQlQueryParams)) - execPlan.isInstanceOf[PromQlExec] shouldEqual true + execPlan.isInstanceOf[PromQlRemoteExec] shouldEqual true - val child = execPlan.asInstanceOf[PromQlExec] + val child = execPlan.asInstanceOf[PromQlRemoteExec] child.params.startSecs shouldEqual from child.params.endSecs shouldEqual to child.params.stepSecs shouldEqual step @@ -416,8 +417,8 @@ class HighAvailabilityPlannerSpec extends FunSpec with Matchers { // Because of offset starts time would be (700 - 600) = 100 seconds where there is failure // So PromQlExec is generated instead of local DistConcatExec. PromQlExec will have original query and start time // Start time with offset will be calculated by buddy pod - execPlan2.isInstanceOf[PromQlExec] shouldEqual (true) - execPlan2.asInstanceOf[PromQlExec].params.startSecs shouldEqual(700) - execPlan2.asInstanceOf[PromQlExec].params.endSecs shouldEqual(10000) + execPlan2.isInstanceOf[PromQlRemoteExec] shouldEqual (true) + execPlan2.asInstanceOf[PromQlRemoteExec].params.startSecs shouldEqual(700) + execPlan2.asInstanceOf[PromQlRemoteExec].params.endSecs shouldEqual(10000) } } diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala new file mode 100644 index 0000000000..f3c19c8924 --- /dev/null +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala @@ -0,0 +1,359 @@ +package filodb.coordinator.queryplanner + +import akka.actor.ActorSystem +import akka.testkit.TestProbe +import com.typesafe.config.ConfigFactory +import org.scalatest.{FunSpec, Matchers} +import filodb.coordinator.ShardMapper +import filodb.core.MetricsTestData +import filodb.core.metadata.Schemas +import filodb.core.query.{PromQlQueryParams, QueryConfig, QueryContext} +import filodb.core.store.TimeRangeChunkScan +import filodb.prometheus.ast.TimeStepParams +import filodb.prometheus.parse.Parser +import filodb.query.exec._ + +class MultiPartitionPlannerSpec extends FunSpec with Matchers { + private implicit val system = ActorSystem() + private val node = TestProbe().ref + + private val mapper = new ShardMapper(32) + for { i <- 0 until 32 } mapper.registerNode(Seq(i), node) + + private def mapperRef = mapper + + private val dataset = MetricsTestData.timeseriesDataset + private val schemas = Schemas(dataset.schema) + + private val routingConfigString = "routing {\n remote {\n http {\n timeout = 10000\n }\n }\n}" + private val routingConfig = ConfigFactory.parseString(routingConfigString) + + private val config = ConfigFactory.load("application_test.conf") + .getConfig("filodb.query").withFallback(routingConfig) + private val queryConfig = new QueryConfig(config) + + val localPlanner = new SingleClusterPlanner(dataset.ref, schemas, mapperRef, earliestRetainedTimestampFn = 0, + queryConfig) + + val startSeconds = 1000 + val endSeconds = 10000 + val localPartitionStart = 3000 + val lookbackMs = 300000 + val step = 100 + + def partitions(timeRange: TimeRange): List[PartitionAssignment] = List(PartitionAssignment("local", "local-url", + TimeRange(timeRange.startMs, timeRange.endMs))) + + it ("should not generate PromQlExec plan when partitions are local") { + val partitionLocationProvider = new PartitionLocationProvider { + override def getPartitions(routingKey: Map[String, String], timeRange: TimeRange): List[PartitionAssignment] = + List(PartitionAssignment("local", "local-url", TimeRange(timeRange.startMs, timeRange.endMs))) + + override def getAuthorizedPartitions(timeRange: TimeRange): List[PartitionAssignment] = + List(PartitionAssignment("local", "local-url", TimeRange(timeRange.startMs, timeRange.endMs))) + } + + val engine = new MultiPartitionPlanner(partitionLocationProvider, localPlanner, "local", dataset, queryConfig) + val lp = Parser.queryRangeToLogicalPlan("test{job = \"app\"}", TimeStepParams(1000, 100, 2000)) + + val promQlQueryParams = PromQlQueryParams("test{job = \"app\"}", 1000, 100, 2000, processMultiPartition = true) + + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + + execPlan.isInstanceOf[DistConcatExec] shouldEqual (true) + execPlan.children.length shouldEqual 2 + execPlan.children.head.isInstanceOf[MultiSchemaPartitionsExec] shouldEqual true + execPlan.children.head.rangeVectorTransformers.head.isInstanceOf[PeriodicSamplesMapper] shouldEqual true + } + + it ("should generate local & PromQlRemoteExec plan") { + + def twoPartitions(timeRange: TimeRange): List[PartitionAssignment] = List( + PartitionAssignment("remote", "remote-url", TimeRange(startSeconds * 1000 - lookbackMs, + localPartitionStart * 1000 - 1)), PartitionAssignment("local", "local-url", + TimeRange(localPartitionStart * 1000, endSeconds * 1000))) + + val partitionLocationProvider = new PartitionLocationProvider { + override def getPartitions(routingKey: Map[String, String], timeRange: TimeRange): List[PartitionAssignment] = { + if (routingKey.equals(Map("job" -> "app"))) twoPartitions(timeRange) + else Nil + } + + override def getAuthorizedPartitions(timeRange: TimeRange): List[PartitionAssignment] = twoPartitions(timeRange) + + } + val engine = new MultiPartitionPlanner(partitionLocationProvider, localPlanner, "local", dataset, queryConfig) + val lp = Parser.queryRangeToLogicalPlan("test{job = \"app\"}", TimeStepParams(startSeconds, step, endSeconds)) + + val promQlQueryParams = PromQlQueryParams("test{job = \"app\"}", startSeconds, step, endSeconds, + processMultiPartition = true) + + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + + val stitchRvsExec = execPlan.asInstanceOf[StitchRvsExec] + stitchRvsExec.children.size shouldEqual (2) + stitchRvsExec.children(0).isInstanceOf[DistConcatExec] shouldEqual (true) + stitchRvsExec.children(1).isInstanceOf[PromQlRemoteExec] shouldEqual (true) + + + val remoteExec = stitchRvsExec.children(1).asInstanceOf[PromQlRemoteExec] + remoteExec.params.startSecs shouldEqual startSeconds + remoteExec.params.endSecs shouldEqual (localPartitionStart - 1) + remoteExec.params.stepSecs shouldEqual step + remoteExec.params.processFailure shouldEqual true + remoteExec.params.processMultiPartition shouldEqual false + remoteExec.queryEndpoint shouldEqual "remote-url" + + val localExec = stitchRvsExec.children(0).asInstanceOf[DistConcatExec] + localExec.isInstanceOf[DistConcatExec] shouldEqual (true) + localExec.children.length shouldEqual 2 + localExec.children.head.isInstanceOf[MultiSchemaPartitionsExec] shouldEqual true + + val expectedStartMs = ((startSeconds*1000) to (endSeconds*1000) by (step*1000)).find { instant => + instant - lookbackMs > (localPartitionStart * 1000) + }.get + + localExec.children.head.asInstanceOf[MultiSchemaPartitionsExec]. + chunkMethod.asInstanceOf[TimeRangeChunkScan].startTime shouldEqual (expectedStartMs - lookbackMs) + localExec.children.head.asInstanceOf[MultiSchemaPartitionsExec]. + chunkMethod.asInstanceOf[TimeRangeChunkScan].endTime shouldEqual (endSeconds * 1000) + localExec.children.head.rangeVectorTransformers.head.isInstanceOf[PeriodicSamplesMapper] shouldEqual true + localExec.children.head.rangeVectorTransformers.head.asInstanceOf[PeriodicSamplesMapper].start shouldEqual + (expectedStartMs) + localExec.children.head.rangeVectorTransformers.head.asInstanceOf[PeriodicSamplesMapper].end shouldEqual + (endSeconds * 1000) + + } + + it ("should generate only local exec for fixed scalar queries") { + val partitionLocationProvider = new PartitionLocationProvider { + override def getPartitions(routingKey: Map[String, String], timeRange: TimeRange): List[PartitionAssignment] = + partitions(timeRange) + + override def getAuthorizedPartitions(timeRange: TimeRange): List[PartitionAssignment] = + partitions(timeRange) + } + + val engine = new MultiPartitionPlanner(partitionLocationProvider, localPlanner, "local", dataset, queryConfig) + val lp = Parser.queryRangeToLogicalPlan("time()", TimeStepParams(1000, 100, 2000)) + + val promQlQueryParams = PromQlQueryParams("time()", 1000, 100, 2000, processMultiPartition = true) + + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + + execPlan.isInstanceOf[TimeScalarGeneratorExec] shouldEqual (true) + } + + it ("should generate BinaryJoinExec plan when lhs and rhs are in local partition") { + val partitionLocationProvider = new PartitionLocationProvider { + override def getPartitions(routingKey: Map[String, String], timeRange: TimeRange): List[PartitionAssignment] = + partitions(timeRange) + + override def getAuthorizedPartitions(timeRange: TimeRange): List[PartitionAssignment] = + partitions(timeRange) + } + + val engine = new MultiPartitionPlanner(partitionLocationProvider, localPlanner, "local", dataset, queryConfig) + val lp = Parser.queryRangeToLogicalPlan("test1{job = \"app\"} + test2{job = \"app\"}", + TimeStepParams(1000, 100, 2000)) + + val promQlQueryParams = PromQlQueryParams("test1{job = \"app\"} + test2{job = \"app\"}", 1000, 100, 2000, + processMultiPartition = true) + + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + + execPlan.isInstanceOf[BinaryJoinExec] shouldEqual (true) + } + + it ("should generate PromQlRemoteExec plan for BinaryJoin when lhs and rhs are in same remote partition") { + def partitions(timeRange: TimeRange): List[PartitionAssignment] = List(PartitionAssignment("remote", "remote-url", + TimeRange(timeRange.startMs, timeRange.endMs))) + + val partitionLocationProvider = new PartitionLocationProvider { + override def getPartitions(routingKey: Map[String, String], timeRange: TimeRange): List[PartitionAssignment] = + partitions(timeRange) + + override def getAuthorizedPartitions(timeRange: TimeRange): List[PartitionAssignment] = + partitions(timeRange) + } + + val engine = new MultiPartitionPlanner(partitionLocationProvider, localPlanner, "local", dataset, queryConfig) + val lp = Parser.queryRangeToLogicalPlan("test1{job = \"app\"} + test2{job = \"app\"}", + TimeStepParams(1000, 100, 10000)) + + val promQlQueryParams = PromQlQueryParams("test1{job = \"app\"} + test2{job = \"app\"}", 1000, 100, 10000, + processMultiPartition = true) + + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + + execPlan.isInstanceOf[PromQlRemoteExec] shouldEqual (true) + execPlan.asInstanceOf[PromQlRemoteExec].params.startSecs shouldEqual 1000 + execPlan.asInstanceOf[PromQlRemoteExec].params.endSecs shouldEqual 10000 + } + + it ("should generate Exec plan for Metadata query") { + def partitions(timeRange: TimeRange): List[PartitionAssignment] = + List(PartitionAssignment("remote", "remote-url", + TimeRange(startSeconds * 1000, localPartitionStart * 1000 - 1)), + PartitionAssignment("local", "local-url", TimeRange(localPartitionStart * 1000, endSeconds * 1000))) + + val partitionLocationProvider = new PartitionLocationProvider { + override def getPartitions(routingKey: Map[String, String], timeRange: TimeRange): List[PartitionAssignment] = + partitions(timeRange) + + override def getAuthorizedPartitions(timeRange: TimeRange): List[PartitionAssignment] = + partitions(timeRange) + } + + val engine = new MultiPartitionPlanner(partitionLocationProvider, localPlanner, "local", dataset, queryConfig) + val lp = Parser.metadataQueryToLogicalPlan("http_requests_total{job=\"prometheus\", method=\"GET\"}", + TimeStepParams(startSeconds, step, endSeconds)) + + val promQlQueryParams = PromQlQueryParams( + "http_requests_total{job=\"prometheus\", method=\"GET\"}", startSeconds, step, endSeconds, + processMultiPartition = true) + + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + + execPlan.isInstanceOf[PartKeysDistConcatExec] shouldEqual (true) + execPlan.children(0).isInstanceOf[PartKeysDistConcatExec] shouldEqual(true) + execPlan.children(1).isInstanceOf[MetadataRemoteExec] shouldEqual(true) + + execPlan.children(1).asInstanceOf[MetadataRemoteExec].params.startSecs shouldEqual(startSeconds) + execPlan.children(1).asInstanceOf[MetadataRemoteExec].params.endSecs shouldEqual(localPartitionStart - 1) + execPlan.children(0).asInstanceOf[PartKeysDistConcatExec].children(0).asInstanceOf[PartKeysExec].start shouldEqual + (localPartitionStart * 1000) + execPlan.children(0).asInstanceOf[PartKeysDistConcatExec].children(0).asInstanceOf[PartKeysExec].end shouldEqual + (endSeconds * 1000) + } + + it ("should generate local & PromQlRemoteExec from 3 assignments") { + val startSeconds = 1000 + val endSeconds = 10000 + val secondPartitionStart = 4000 + val thirdPartitionStart = 7000 + val lookbackMs = 300000 + val step = 100 + val partitionLocationProvider = new PartitionLocationProvider { + override def getPartitions(routingKey: Map[String, String], timeRange: TimeRange): List[PartitionAssignment] = { + if (routingKey.equals(Map("job" -> "app"))) + List(PartitionAssignment("remote1", "remote-url1", TimeRange(startSeconds * 1000 - lookbackMs, + secondPartitionStart * 1000 - 1)), + PartitionAssignment("remote2", "remote-url2", TimeRange(secondPartitionStart * 1000, + thirdPartitionStart * 1000 - 1)), + PartitionAssignment("local", "local-url", TimeRange(thirdPartitionStart * 1000, endSeconds * 1000))) + else Nil + } + override def getAuthorizedPartitions(timeRange: TimeRange): List[PartitionAssignment] = + List(PartitionAssignment("remote1", "remote-url1", TimeRange(startSeconds * 1000 - lookbackMs, + secondPartitionStart * 1000 - 1)), + PartitionAssignment("remote2", "remote-url2", TimeRange(secondPartitionStart * 1000, + thirdPartitionStart * 1000 - 1)), + PartitionAssignment("local", "local-url", TimeRange(thirdPartitionStart * 1000, endSeconds * 1000))) + } + val engine = new MultiPartitionPlanner(partitionLocationProvider, localPlanner, "local", dataset, queryConfig) + val lp = Parser.queryRangeToLogicalPlan("test{job = \"app\"}", TimeStepParams(startSeconds, step, endSeconds)) + val promQlQueryParams = PromQlQueryParams("test{job = \"app\"}", startSeconds, step, endSeconds, + processMultiPartition = true) + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + val stitchRvsExec = execPlan.asInstanceOf[StitchRvsExec] + stitchRvsExec.children.size shouldEqual (3) + stitchRvsExec.children(0).isInstanceOf[DistConcatExec] shouldEqual (true) + stitchRvsExec.children(1).isInstanceOf[PromQlRemoteExec] shouldEqual (true) + stitchRvsExec.children(2).isInstanceOf[PromQlRemoteExec] shouldEqual (true) + val remoteExec1 = stitchRvsExec.children(1).asInstanceOf[PromQlRemoteExec] + remoteExec1.params.startSecs shouldEqual startSeconds + remoteExec1.params.endSecs shouldEqual 3999 + remoteExec1.params.stepSecs shouldEqual step + remoteExec1.params.processFailure shouldEqual true + remoteExec1.params.processMultiPartition shouldEqual false + remoteExec1.queryEndpoint shouldEqual "remote-url1" + val remoteExec2 = stitchRvsExec.children(2).asInstanceOf[PromQlRemoteExec] + + val expectedStartMs1 = ((startSeconds*1000) to (endSeconds*1000) by (step*1000)).find { instant => + instant - lookbackMs > (secondPartitionStart * 1000) + }.get + + val expectedStartMs2 = ((startSeconds*1000) to (endSeconds*1000) by (step*1000)).find { instant => + instant - lookbackMs > (thirdPartitionStart * 1000) + }.get + + remoteExec2.params.startSecs shouldEqual expectedStartMs1 / 1000 + remoteExec2.params.endSecs shouldEqual 6999 + remoteExec2.params.stepSecs shouldEqual step + remoteExec2.params.processFailure shouldEqual true + remoteExec2.params.processMultiPartition shouldEqual false + remoteExec2.queryEndpoint shouldEqual "remote-url2" + val localExec = stitchRvsExec.children(0).asInstanceOf[DistConcatExec] + localExec.isInstanceOf[DistConcatExec] shouldEqual (true) + localExec.children.length shouldEqual 2 + localExec.children.head.isInstanceOf[MultiSchemaPartitionsExec] shouldEqual true + + localExec.children.head.rangeVectorTransformers.head.asInstanceOf[PeriodicSamplesMapper].start shouldEqual + expectedStartMs2 + localExec.children.head.asInstanceOf[MultiSchemaPartitionsExec]. + chunkMethod.asInstanceOf[TimeRangeChunkScan].startTime shouldEqual expectedStartMs2 - lookbackMs + localExec.children.head.asInstanceOf[MultiSchemaPartitionsExec]. + chunkMethod.asInstanceOf[TimeRangeChunkScan].endTime shouldEqual (endSeconds * 1000) + localExec.children.head.rangeVectorTransformers.head.isInstanceOf[PeriodicSamplesMapper] shouldEqual true + + localExec.children.head.rangeVectorTransformers.head.asInstanceOf[PeriodicSamplesMapper].end shouldEqual + (endSeconds * 1000) + } + + it ("should generate local & PromQlRemoteExec plan for instant queries") { + val startSeconds = 1000 + val endSeconds = 1000 + val localPartitionStartSec= 950 + val lookbackMs = 100000 + val step = 1 + + val partitionLocationProvider = new PartitionLocationProvider { + override def getPartitions(routingKey: Map[String, String], timeRange: TimeRange): List[PartitionAssignment] = { + if (routingKey.equals(Map("job" -> "app"))) + List(PartitionAssignment("remote", "remote-url", TimeRange(startSeconds * 1000 - lookbackMs, + localPartitionStartSec * 1000 - 1)), PartitionAssignment("local", "local-url", + TimeRange(localPartitionStartSec * 1000, endSeconds * 1000))) + else Nil + } + + override def getAuthorizedPartitions(timeRange: TimeRange): List[PartitionAssignment] = + List(PartitionAssignment("remote", "remote-url", TimeRange(startSeconds * 1000 - lookbackMs, + localPartitionStartSec * 1000 - 1)), PartitionAssignment("local", "local-url", + TimeRange(localPartitionStartSec * 1000, endSeconds * 1000))) + } + val engine = new MultiPartitionPlanner(partitionLocationProvider, localPlanner, "local", dataset, queryConfig) + val lp = Parser.queryRangeToLogicalPlan("test{job = \"app\"}[100s]", TimeStepParams(startSeconds, step, endSeconds)) + + val promQlQueryParams = PromQlQueryParams("test{job = \"app\"}", startSeconds, step, endSeconds, + processMultiPartition = true) + + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + + val stitchRvsExec = execPlan.asInstanceOf[StitchRvsExec] + stitchRvsExec.children.size shouldEqual (2) + stitchRvsExec.children(0).isInstanceOf[DistConcatExec] shouldEqual (true) + stitchRvsExec.children(1).isInstanceOf[PromQlRemoteExec] shouldEqual (true) + + + // Instant/Raw queries will have same start and end point in all partitions as we want to fetch raw data + val remoteExec = stitchRvsExec.children(1).asInstanceOf[PromQlRemoteExec] + remoteExec.params.startSecs shouldEqual startSeconds + remoteExec.params.endSecs shouldEqual endSeconds + remoteExec.params.stepSecs shouldEqual step + remoteExec.params.processFailure shouldEqual true + remoteExec.params.processMultiPartition shouldEqual false + remoteExec.queryEndpoint shouldEqual "remote-url" + + val localExec = stitchRvsExec.children(0).asInstanceOf[DistConcatExec] + localExec.isInstanceOf[DistConcatExec] shouldEqual (true) + localExec.children.length shouldEqual 2 + localExec.children.head.isInstanceOf[MultiSchemaPartitionsExec] shouldEqual true + + localExec.children.head.asInstanceOf[MultiSchemaPartitionsExec]. + chunkMethod.asInstanceOf[TimeRangeChunkScan].startTime shouldEqual (startSeconds * 1000 - lookbackMs) + localExec.children.head.asInstanceOf[MultiSchemaPartitionsExec]. + chunkMethod.asInstanceOf[TimeRangeChunkScan].endTime shouldEqual (endSeconds * 1000) + + } +} diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/ScalarQueriesSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/ScalarQueriesSpec.scala index 609bbc1186..f86028278f 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/ScalarQueriesSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/ScalarQueriesSpec.scala @@ -33,11 +33,7 @@ class ScalarQueriesSpec extends FunSpec with Matchers { val engine = new SingleClusterPlanner(dsRef, schemas, mapperRef, earliestRetainedTimestampFn = 0, queryConfig) - val queryEngineConfigString = "routing {\n buddy {\n http {\n timeout = 10.seconds\n }\n }\n}" - - val queryEngineConfig = ConfigFactory.parseString(queryEngineConfigString) - val promQlQueryParams = PromQlQueryParams(ConfigFactory.empty, "sum(heap_usage)", 100, 1, 1000, None) - + val promQlQueryParams = PromQlQueryParams("sum(heap_usage)", 100, 1, 1000) val f1 = Seq(ColumnFilter("__name__", Filter.Equals("http_request_duration_seconds_bucket")), ColumnFilter("job", Filter.Equals("myService")), diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/SingleClusterPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/SingleClusterPlannerSpec.scala index 6d0e4eca14..d994995d8c 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/SingleClusterPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/SingleClusterPlannerSpec.scala @@ -64,7 +64,7 @@ class SingleClusterPlannerSpec extends FunSpec with Matchers with ScalaFutures { val raw2 = RawSeries(rangeSelector = intervalSelector, filters= f2, columns = Seq("value")) val windowed2 = PeriodicSeriesWithWindowing(raw2, from, 1000, to, 5000, RangeFunctionId.Rate) val summed2 = Aggregate(AggregationOperator.Sum, windowed2, Nil, Seq("job")) - val promQlQueryParams = PromQlQueryParams(ConfigFactory.empty, "sum(heap_usage)", 100, 1, 1000, None) + val promQlQueryParams = PromQlQueryParams("sum(heap_usage)", 100, 1, 1000) it ("should generate ExecPlan for LogicalPlan") { // final logical plan diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/SinglePartitionPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/SinglePartitionPlannerSpec.scala index 35e0c7d9d4..f643ce41b6 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/SinglePartitionPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/SinglePartitionPlannerSpec.scala @@ -30,13 +30,15 @@ class SinglePartitionPlannerSpec extends FunSpec with Matchers{ private val dsRef = dataset.ref private val schemas = Schemas(dataset.schema) - private val routingConfigString = "routing {\n buddy {\n http {\n timeout = 10.seconds\n }\n }\n}" + private val routingConfigString = "routing {\n remote {\n " + + " http {\n endpoint = localhost\n timeout = 10000\n }\n }\n}" + private val routingConfig = ConfigFactory.parseString(routingConfigString) private val config = ConfigFactory.load("application_test.conf").getConfig("filodb.query"). withFallback(routingConfig) private val queryConfig = new QueryConfig(config) - private val promQlQueryParams = PromQlQueryParams(ConfigFactory.empty, "sum(heap_usage)", 100, 1, 1000, None) + private val promQlQueryParams = PromQlQueryParams("sum(heap_usage)", 100, 1, 1000) val localPlanner = new SingleClusterPlanner(dsRef, schemas, localMapper, earliestRetainedTimestampFn = 0, queryConfig) val remotePlanner = new SingleClusterPlanner(dsRef, schemas, remoteMapper, earliestRetainedTimestampFn = 0, @@ -139,12 +141,13 @@ class SinglePartitionPlannerSpec extends FunSpec with Matchers{ TimeStepParams(1000, 10, 2000)) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) - execPlan.printTree() execPlan.isInstanceOf[PartKeysDistConcatExec] shouldEqual (true) execPlan.asInstanceOf[PartKeysDistConcatExec].children.length shouldEqual(3) - // For Raw and Downsample + // For Raw execPlan.asInstanceOf[PartKeysDistConcatExec].children(0).isInstanceOf[PartKeysDistConcatExec] shouldEqual true + execPlan.asInstanceOf[PartKeysDistConcatExec].children(0).asInstanceOf[PartKeysDistConcatExec].children. + forall(_.isInstanceOf[PartKeysExec]) shouldEqual true execPlan.asInstanceOf[PartKeysDistConcatExec].children(1).asInstanceOf[MockExecPlan].name shouldEqual ("rules1") execPlan.asInstanceOf[PartKeysDistConcatExec].children(2).asInstanceOf[MockExecPlan].name shouldEqual ("rules2") diff --git a/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesShard.scala b/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesShard.scala index 9f32528a9e..19aae40be2 100644 --- a/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesShard.scala @@ -21,6 +21,7 @@ import filodb.core.metadata.Schemas import filodb.core.query.{ColumnFilter, QuerySession} import filodb.core.store._ import filodb.memory.format.{UnsafeUtils, ZeroCopyUTF8String} +import filodb.memory.format.ZeroCopyUTF8String._ class DownsampledTimeSeriesShardStats(dataset: DatasetRef, shardNum: Int) { val tags = Map("shard" -> shardNum.toString, "dataset" -> dataset.toString) @@ -90,9 +91,13 @@ class DownsampledTimeSeriesShard(rawDatasetRef: DatasetRef, fetchFirstLastSampleTimes: Boolean, endTime: Long, startTime: Long, - limit: Int): Iterator[PartKeyWithTimes] = { + limit: Int): Iterator[Map[ZeroCopyUTF8String, ZeroCopyUTF8String]] = { partKeyIndex.partKeyRecordsFromFilters(filter, startTime, endTime).iterator.take(limit).map { pk => - PartKeyWithTimes(pk.partKey, UnsafeUtils.arayOffset, pk.startTime, pk.endTime) + val partKey = PartKeyWithTimes(pk.partKey, UnsafeUtils.arayOffset, pk.startTime, pk.endTime) + schemas.part.binSchema.toStringPairs(partKey.base, partKey.offset).map(pair => { + pair._1.utf8 -> pair._2.utf8 + }).toMap ++ + Map("_type_".utf8 -> Schemas.global.schemaName(RecordSchema.schemaID(partKey.base, partKey.offset)).utf8) } } @@ -325,7 +330,6 @@ class DownsampledTimeSeriesShard(rawDatasetRef: DatasetRef, while(partIndex < partIds.length && numResultsReturned < limit && !foundValue) { val partId = partIds(partIndex) - import ZeroCopyUTF8String._ //retrieve PartKey either from In-memory map or from PartKeyIndex val nextPart = partKeyFromPartId(partId) diff --git a/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesStore.scala b/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesStore.scala index 0c546965b6..628f67100b 100644 --- a/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesStore.scala +++ b/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesStore.scala @@ -82,7 +82,7 @@ extends MemStore with StrictLogging { def partKeysWithFilters(dataset: DatasetRef, shard: Int, filters: Seq[ColumnFilter], fetchFirstLastSampleTimes: Boolean, end: Long, start: Long, - limit: Int): Iterator[PartKeyWithTimes] = + limit: Int): Iterator[Map[ZeroCopyUTF8String, ZeroCopyUTF8String]] = getShard(dataset, shard).map(_.partKeysWithFilters(filters, fetchFirstLastSampleTimes, end, start, limit)).getOrElse(Iterator.empty) diff --git a/core/src/main/scala/filodb.core/memstore/MemStore.scala b/core/src/main/scala/filodb.core/memstore/MemStore.scala index 0e111332b0..e24bcc6b6b 100644 --- a/core/src/main/scala/filodb.core/memstore/MemStore.scala +++ b/core/src/main/scala/filodb.core/memstore/MemStore.scala @@ -170,7 +170,7 @@ trait MemStore extends ChunkSource { */ def partKeysWithFilters(dataset: DatasetRef, shard: Int, filters: Seq[ColumnFilter], fetchFirstLastSampleTimes: Boolean, end: Long, start: Long, - limit: Int): Iterator[PartKeyWithTimes] + limit: Int): Iterator[Map[ZeroCopyUTF8String, ZeroCopyUTF8String]] /** * Returns the number of partitions being maintained in the memtable for a given shard diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala index c9166de1da..e2199977b6 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala @@ -187,7 +187,7 @@ extends MemStore with StrictLogging { def partKeysWithFilters(dataset: DatasetRef, shard: Int, filters: Seq[ColumnFilter], fetchFirstLastSampleTimes: Boolean, end: Long, start: Long, - limit: Int): Iterator[PartKeyWithTimes] = + limit: Int): Iterator[Map[ZeroCopyUTF8String, ZeroCopyUTF8String]] = getShard(dataset, shard).map(_.partKeysWithFilters(filters, fetchFirstLastSampleTimes, end, start, limit)).getOrElse(Iterator.empty) diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala index aad7a5fe2d..d9fc677d14 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala @@ -673,20 +673,33 @@ class TimeSeriesShard(val ref: DatasetRef, fetchFirstLastSampleTimes: Boolean, endTime: Long, startTime: Long, - limit: Int): Iterator[PartKeyWithTimes] = { + limit: Int): Iterator[Map[ZeroCopyUTF8String, ZeroCopyUTF8String]] = { if (fetchFirstLastSampleTimes) { partKeyIndex.partKeyRecordsFromFilters(filter, startTime, endTime).iterator.map { pk => - PartKeyWithTimes(pk.partKey, UnsafeUtils.arayOffset, pk.startTime, pk.endTime) - } + val partKeyMap = convertPartKeyWithTimesToMap( + PartKeyWithTimes(pk.partKey, UnsafeUtils.arayOffset, pk.startTime, pk.endTime)) + partKeyMap ++ Map( + ("_firstSampleTime_".utf8, pk.startTime.toString.utf8), + ("_lastSampleTime_".utf8, pk.endTime.toString.utf8)) + } take(limit) } else { val partIds = partKeyIndex.partIdsFromFilters(filter, startTime, endTime) val inMem = InMemPartitionIterator2(partIds) - val inMemPartKeys = inMem.map { p => PartKeyWithTimes(p.partKeyBase, p.partKeyOffset, -1, -1) } - val skippedPartKeys = inMem.skippedPartIDs.iterator().map(partKeyFromPartId) + val inMemPartKeys = inMem.map { p => + convertPartKeyWithTimesToMap(PartKeyWithTimes(p.partKeyBase, p.partKeyOffset, -1, -1))} + val skippedPartKeys = inMem.skippedPartIDs.iterator().map(partId => { + convertPartKeyWithTimesToMap(partKeyFromPartId(partId))}) (inMemPartKeys ++ skippedPartKeys).take(limit) } } + private def convertPartKeyWithTimesToMap(partKey: PartKeyWithTimes): Map[ZeroCopyUTF8String, ZeroCopyUTF8String] = { + schemas.part.binSchema.toStringPairs(partKey.base, partKey.offset).map(pair => { + pair._1.utf8 -> pair._2.utf8 + }).toMap ++ + Map("_type_".utf8 -> Schemas.global.schemaName(RecordSchema.schemaID(partKey.base, partKey.offset)).utf8) + } + /** * retrieve partKey for a given PartId */ diff --git a/core/src/main/scala/filodb.core/query/QueryContext.scala b/core/src/main/scala/filodb.core/query/QueryContext.scala index e0a0192e4a..bc2dc5ebd2 100644 --- a/core/src/main/scala/filodb.core/query/QueryContext.scala +++ b/core/src/main/scala/filodb.core/query/QueryContext.scala @@ -3,8 +3,6 @@ package filodb.core.query import java.util.UUID import java.util.concurrent.locks.Lock -import com.typesafe.config.Config - import filodb.core.{SpreadChange, SpreadProvider} trait TsdbQueryParams @@ -13,8 +11,9 @@ trait TsdbQueryParams * This class provides PromQl query parameters * Config has routing parameters */ -case class PromQlQueryParams(config: Config, promQl: String, startSecs: Long, stepSecs: Long, endSecs: Long, - spread: Option[Int] = None, processFailure: Boolean = true) extends TsdbQueryParams +case class PromQlQueryParams(promQl: String, startSecs: Long, stepSecs: Long, endSecs: Long, spread: Option[Int] = None, + remoteQueryPath: Option[String] = None, processFailure: Boolean = true, + processMultiPartition: Boolean = false, verbose: Boolean = false) extends TsdbQueryParams case object UnavailablePromQlQueryParams extends TsdbQueryParams /** diff --git a/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreForMetadataSpec.scala b/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreForMetadataSpec.scala index f44f643bfa..05ae77ebc4 100644 --- a/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreForMetadataSpec.scala +++ b/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreForMetadataSpec.scala @@ -1,7 +1,5 @@ package filodb.core.memstore -import scala.collection.mutable.ArrayBuffer - import com.typesafe.config.ConfigFactory import monix.execution.Scheduler.Implicits.global import org.scalatest.{BeforeAndAfterAll, FunSpec, Matchers} @@ -11,7 +9,7 @@ import org.scalatest.time.{Millis, Seconds, Span} import filodb.core.MetricsTestData.{builder, timeseriesDataset, timeseriesSchema} import filodb.core.TestData import filodb.core.metadata.Schemas -import filodb.core.query.{ColumnFilter, Filter, SeqMapConsumer} +import filodb.core.query.{ColumnFilter, Filter} import filodb.core.store.{InMemoryMetaStore, NullColumnStore} import filodb.core.binaryrecord2.RecordContainer import filodb.memory.format.{SeqRowReader, ZeroCopyUTF8String} @@ -27,10 +25,12 @@ class TimeSeriesMemStoreForMetadataSpec extends FunSpec with Matchers with Scala val metadataKeyLabelValues = Map("ignore" -> "ignore") val jobQueryResult1 = Map(("job".utf8, "myCoolService".utf8)) - val jobQueryResult2 = ArrayBuffer(("__name__".utf8, "http_req_total".utf8), + val jobQueryResult2 = Map(("job".utf8, "myCoolService".utf8), ("id".utf8, "0".utf8), - ("instance".utf8, "someHost:8787".utf8), - ("job".utf8, "myCoolService".utf8)) + ("__name__".utf8, "http_req_total".utf8), + ("_type_".utf8 -> "schemaID:35859".utf8), + ("instance".utf8, "someHost:8787".utf8) + ) val now = System.currentTimeMillis() val numRawSamples = 1000 @@ -60,12 +60,8 @@ class TimeSeriesMemStoreForMetadataSpec extends FunSpec with Matchers with Scala ColumnFilter("job", Filter.Equals("myCoolService".utf8)), ColumnFilter("id", Filter.Equals("0".utf8))) val metadata = memStore.partKeysWithFilters(timeseriesDataset.ref, 0, filters, false, now, now - 5000, 10) - val seqMapConsumer = new SeqMapConsumer() val tsPartData = metadata.next() - timeseriesDataset.partKeySchema.consumeMapItems(tsPartData.base, tsPartData.offset, 0, seqMapConsumer) - tsPartData.startTime shouldEqual -1 // since fetchFirstLastSampleTimes is false - tsPartData.endTime shouldEqual -1 // since fetchFirstLastSampleTimes is false - seqMapConsumer.pairs shouldEqual jobQueryResult2 + tsPartData shouldEqual jobQueryResult2 } it("should search the metadata of evicted partitions") { @@ -88,12 +84,10 @@ class TimeSeriesMemStoreForMetadataSpec extends FunSpec with Matchers with Scala ColumnFilter("job", Filter.Equals("myCoolService".utf8)), ColumnFilter("id", Filter.Equals("0".utf8))) val metadata = memStore.partKeysWithFilters(timeseriesDataset.ref, 0, filters, true, endTime, endTime - 5000, 10) - val seqMapConsumer = new SeqMapConsumer() val tsPartData = metadata.next() - timeseriesDataset.partKeySchema.consumeMapItems(tsPartData.base, tsPartData.offset, 0, seqMapConsumer) - tsPartData.startTime shouldEqual startTime - tsPartData.endTime shouldEqual endTime - seqMapConsumer.pairs shouldEqual jobQueryResult2 + val jobQueryResult = jobQueryResult2 ++ + Map(("_firstSampleTime_".utf8, startTime.toString.utf8), ("_lastSampleTime_".utf8, endTime.toString.utf8)) + tsPartData shouldEqual jobQueryResult } it ("should read the metadata label values for instance") { diff --git a/http/src/main/scala/filodb/http/PrometheusApiRoute.scala b/http/src/main/scala/filodb/http/PrometheusApiRoute.scala index d2ee28fbe5..19e1b34cdf 100644 --- a/http/src/main/scala/filodb/http/PrometheusApiRoute.scala +++ b/http/src/main/scala/filodb/http/PrometheusApiRoute.scala @@ -7,7 +7,6 @@ import akka.http.scaladsl.model.{HttpEntity, HttpResponse, MediaTypes, StatusCod import akka.http.scaladsl.server.Directives._ import akka.stream.ActorMaterializer import akka.util.ByteString -import com.typesafe.config.ConfigFactory import com.typesafe.scalalogging.StrictLogging import de.heikoseeberger.akkahttpcirce.FailFastCirceSupport import org.xerial.snappy.Snappy @@ -49,8 +48,7 @@ class PrometheusApiRoute(nodeCoord: ActorRef, settings: HttpSettings)(implicit a // No cross-cluster failure routing in this API, hence we pass empty config askQueryAndRespond(dataset, logicalPlan, explainOnly.getOrElse(false), verbose.getOrElse(false), - spread, PromQlQueryParams(ConfigFactory.empty, query, start.toLong, step.toLong, end.toLong, spread), - histMap.getOrElse(false)) + spread, PromQlQueryParams(query, start.toLong, step.toLong, end.toLong, spread), histMap.getOrElse(false)) } } } ~ @@ -65,8 +63,7 @@ class PrometheusApiRoute(nodeCoord: ActorRef, settings: HttpSettings)(implicit a { (query, time, explainOnly, verbose, spread, histMap) => val logicalPlan = Parser.queryToLogicalPlan(query, time.toLong) askQueryAndRespond(dataset, logicalPlan, explainOnly.getOrElse(false), - verbose.getOrElse(false), spread, PromQlQueryParams(ConfigFactory.empty, query, time.toLong, 1000, - time.toLong, spread), + verbose.getOrElse(false), spread, PromQlQueryParams(query, time.toLong, 1000, time.toLong, spread), histMap.getOrElse(false)) } } diff --git a/query/src/main/scala/filodb/query/LogicalPlan.scala b/query/src/main/scala/filodb/query/LogicalPlan.scala index f449552d3e..c12409fcbb 100644 --- a/query/src/main/scala/filodb/query/LogicalPlan.scala +++ b/query/src/main/scala/filodb/query/LogicalPlan.scala @@ -77,7 +77,8 @@ case class RawSeries(rangeSelector: RangeSelector, case class LabelValues(labelNames: Seq[String], labelConstraints: Map[String, String], - lookbackTimeMs: Long) extends MetadataQueryPlan + startMs: Long, + endMs: Long) extends MetadataQueryPlan case class SeriesKeysByFilters(filters: Seq[ColumnFilter], fetchFirstLastSampleTimes: Boolean, @@ -353,26 +354,29 @@ object LogicalPlan { } def getLabelValueFromLogicalPlan(logicalPlan: LogicalPlan, labelName: String): Option[Seq[String]] = { - getLabelValueFromLogicalPlan(getLabelValueOperatorsFromLogicalPlan(logicalPlan), labelName) + getAllLabelValueFromGroups(getLabelValueOperatorsFromLogicalPlan(logicalPlan), labelName) } - def getLabelValueFromLogicalPlan(labelValues: Option[Seq[LabelValueOperatorGroup]], + def getAllLabelValueFromGroups(labelValues: Option[Seq[LabelValueOperatorGroup]], labelName: String): Option[Seq[String]] = { labelValues match { case None => None - case _ => labelValues.get.flatMap(group => - group.labelValueOperators.flatMap(lops => { - lops.columnName.equals(labelName) match { - case true => lops.value - case false => Seq() - } - })).distinct match { + case _ => labelValues.get.flatMap(group => getLabelValueFromGroup(labelName, group)).distinct match { case Nil => None case lVFilters: Seq[String] => Some(lVFilters) } } } + def getLabelValueFromGroup(labelName: String, group: LabelValueOperatorGroup): Seq[String] = { + group.labelValueOperators.flatMap(lops => { + lops.columnName.equals(labelName) match { + case true => lops.value + case false => Seq() + } + }) + } + private def getLabelValueOpsFromFilters(filters: Seq[ColumnFilter]): Option[LabelValueOperatorGroup] = { Some(LabelValueOperatorGroup(filters.map(cf => LabelValueOperator(cf.column, cf.filter.valuesStrings.map(_.toString).toSeq.sorted, cf.filter.operatorString)))) diff --git a/query/src/main/scala/filodb/query/PromCirceSupport.scala b/query/src/main/scala/filodb/query/PromCirceSupport.scala index 254fa0b424..ed6f47d3ae 100644 --- a/query/src/main/scala/filodb/query/PromCirceSupport.scala +++ b/query/src/main/scala/filodb/query/PromCirceSupport.scala @@ -11,6 +11,7 @@ object PromCirceSupport { implicit val encodeSampl: Encoder[DataSampl] = Encoder.instance { case s @ Sampl(t, v) => Json.fromValues(Seq(t.asJson, v.toString.asJson)) case h @ HistSampl(t, b) => Json.fromValues(Seq(t.asJson, b.asJson)) + case m @ MetadataSampl(v) => Json.fromValues(Seq(v.asJson)) } implicit val decodeFoo: Decoder[DataSampl] = new Decoder[DataSampl] { diff --git a/query/src/main/scala/filodb/query/PromQueryResponse.scala b/query/src/main/scala/filodb/query/PromQueryResponse.scala index bbaf9e0d32..5b3ad39aca 100644 --- a/query/src/main/scala/filodb/query/PromQueryResponse.scala +++ b/query/src/main/scala/filodb/query/PromQueryResponse.scala @@ -12,6 +12,9 @@ final case class ExplainPlanResponse(debugInfo: Seq[String], status: String = "s final case class Data(resultType: String, result: Seq[Result]) +final case class MetadataSuccessResponse(data: Seq[Map[String, String]], + status: String = "success") extends PromQueryResponse + final case class Result(metric: Map[String, String], values: Option[Seq[DataSampl]], value: Option[DataSampl] = None) sealed trait DataSampl @@ -23,4 +26,6 @@ sealed trait DataSampl */ final case class Sampl(timestamp: Long, value: Double) extends DataSampl -final case class HistSampl(timestamp: Long, buckets: Map[String, Double]) extends DataSampl \ No newline at end of file +final case class HistSampl(timestamp: Long, buckets: Map[String, Double]) extends DataSampl + +final case class MetadataSampl(values: Map[String, String]) extends DataSampl diff --git a/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala b/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala index b5d75b4fd6..eed647cfe8 100644 --- a/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala +++ b/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala @@ -7,7 +7,7 @@ import monix.reactive.Observable import filodb.core.DatasetRef import filodb.core.binaryrecord2.BinaryRecordRowReader -import filodb.core.memstore.{MemStore, PartKeyRowReader} +import filodb.core.memstore.MemStore import filodb.core.metadata.Column.ColumnType import filodb.core.metadata.PartitionSchema import filodb.core.query._ @@ -17,50 +17,16 @@ import filodb.memory.format.ZeroCopyUTF8String._ import filodb.query._ import filodb.query.Query.qLogger -final case class PartKeysDistConcatExec(queryContext: QueryContext, - dispatcher: PlanDispatcher, - children: Seq[ExecPlan]) extends NonLeafExecPlan { +trait MetadataDistConcatExec extends NonLeafExecPlan { require(!children.isEmpty) override def enforceLimit: Boolean = false /** - * Args to use for the ExecPlan for printTree purposes only. - * DO NOT change to a val. Increases heap usage. - */ - override protected def args: String = "" - - /** - * Compose the sub-query/leaf results here. - */ - protected def compose(childResponses: Observable[(QueryResponse, Int)], - firstSchema: Task[ResultSchema], - querySession: QuerySession): Observable[RangeVector] = { - qLogger.debug(s"NonLeafMetadataExecPlan: Concatenating results") - val taskOfResults = childResponses.map { - case (QueryResult(_, _, result), _) => result - case (QueryError(_, ex), _) => throw ex - }.toListL.map { resp => - IteratorBackedRangeVector(new CustomRangeVectorKey(Map.empty), rowIterAccumulator(resp)) - } - Observable.fromTask(taskOfResults) - } - -} - -final case class LabelValuesDistConcatExec(queryContext: QueryContext, - dispatcher: PlanDispatcher, - children: Seq[ExecPlan]) extends NonLeafExecPlan { - - require(!children.isEmpty) - - override def enforceLimit: Boolean = false - - /** - * Args to use for the ExecPlan for printTree purposes only. - * DO NOT change to a val. Increases heap usage. - */ + * Args to use for the ExecPlan for printTree purposes only. + * DO NOT change to a val. Increases heap usage. + */ override protected def args: String = "" /** @@ -76,25 +42,31 @@ final case class LabelValuesDistConcatExec(queryContext: QueryContext, }.toListL.map { resp => var metadataResult = scala.collection.mutable.Set.empty[Map[ZeroCopyUTF8String, ZeroCopyUTF8String]] resp.foreach { rv => - metadataResult ++= rv.head.rows.map { rowReader => - val binaryRowReader = rowReader.asInstanceOf[BinaryRecordRowReader] - rv.head match { - case srv: SerializedRangeVector => - srv.schema.toStringPairs (binaryRowReader.recordBase, binaryRowReader.recordOffset) - .map (pair => pair._1.utf8 -> pair._2.utf8).toMap - case _ => throw new UnsupportedOperationException("Metadata query currently needs SRV results") - } + metadataResult ++= rv.head.rows.map { rowReader => + val binaryRowReader = rowReader.asInstanceOf[BinaryRecordRowReader] + rv.head match { + case srv: SerializedRangeVector => + srv.schema.toStringPairs (binaryRowReader.recordBase, binaryRowReader.recordOffset) + .map (pair => pair._1.utf8 -> pair._2.utf8).toMap + case _ => throw new UnsupportedOperationException("Metadata query currently needs SRV results") } + } } - //distinct -> result may have duplicates in case of labelValues IteratorBackedRangeVector(new CustomRangeVectorKey(Map.empty), new UTF8MapIteratorRowReader(metadataResult.toIterator)) } Observable.fromTask(taskOfResults) } - } +final case class PartKeysDistConcatExec(queryContext: QueryContext, + dispatcher: PlanDispatcher, + children: Seq[ExecPlan]) extends MetadataDistConcatExec + +final case class LabelValuesDistConcatExec(queryContext: QueryContext, + dispatcher: PlanDispatcher, + children: Seq[ExecPlan]) extends MetadataDistConcatExec + final case class PartKeysExec(queryContext: QueryContext, dispatcher: PlanDispatcher, dataset: DatasetRef, @@ -114,28 +86,27 @@ final case class PartKeysExec(queryContext: QueryContext, case memStore: MemStore => val response = memStore.partKeysWithFilters(dataset, shard, filters, fetchFirstLastSampleTimes, end, start, queryContext.sampleLimit) - Observable.now(IteratorBackedRangeVector(new CustomRangeVectorKey(Map.empty), PartKeyRowReader(response))) + Observable.now(IteratorBackedRangeVector( + new CustomRangeVectorKey(Map.empty), UTF8MapIteratorRowReader(response))) case other => Observable.empty } Kamon.currentSpan().mark("creating-resultschema") - val sch = new ResultSchema(Seq(ColumnInfo("TimeSeries", ColumnType.BinaryRecordColumn), - ColumnInfo("_firstSampleTime_", ColumnType.LongColumn), - ColumnInfo("_lastSampleTime_", ColumnType.LongColumn)), 3, - Map(0 -> partSchema.binSchema)) + val sch = new ResultSchema(Seq(ColumnInfo("Labels", ColumnType.MapColumn)), 1) ExecResult(rvs, Task.eval(sch)) } def args: String = s"shard=$shard, filters=$filters, limit=${queryContext.sampleLimit}" } -final case class LabelValuesExec(queryContext: QueryContext, - dispatcher: PlanDispatcher, - dataset: DatasetRef, - shard: Int, - filters: Seq[ColumnFilter], - columns: Seq[String], - lookBackInMillis: Long) extends LeafExecPlan { +final case class LabelValuesExec(queryContext: QueryContext, + dispatcher: PlanDispatcher, + dataset: DatasetRef, + shard: Int, + filters: Seq[ColumnFilter], + columns: Seq[String], + startMs: Long, + endMs: Long) extends LeafExecPlan { override def enforceLimit: Boolean = false @@ -145,9 +116,6 @@ final case class LabelValuesExec(queryContext: QueryContext, val parentSpan = Kamon.currentSpan() val rvs = if (source.isInstanceOf[MemStore]) { val memStore = source.asInstanceOf[MemStore] - val curr = System.currentTimeMillis() - val end = curr - curr % 1000 // round to the floor second - val start = end - lookBackInMillis val response = filters.isEmpty match { // retrieves label values for a single label - no column filter case true if (columns.size == 1) => memStore.labelValues(dataset, shard, columns.head, queryContext.sampleLimit) @@ -155,7 +123,7 @@ final case class LabelValuesExec(queryContext: QueryContext, .toIterator case true => throw new BadQueryException("either label name is missing " + "or there are multiple label names without filter") - case false => memStore.labelValuesWithFilters(dataset, shard, filters, columns, end, start, + case false => memStore.labelValuesWithFilters(dataset, shard, filters, columns, endMs, startMs, queryContext.sampleLimit) } Observable.now(IteratorBackedRangeVector(new CustomRangeVectorKey(Map.empty), @@ -169,5 +137,5 @@ final case class LabelValuesExec(queryContext: QueryContext, } def args: String = s"shard=$shard, filters=$filters, col=$columns, limit=${queryContext.sampleLimit}, " + - s"lookBackInMillis=$lookBackInMillis" + s"startMs=$startMs, endMs=$endMs" } diff --git a/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala b/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala new file mode 100644 index 0000000000..de3adb06c0 --- /dev/null +++ b/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala @@ -0,0 +1,55 @@ +package filodb.query.exec + +import kamon.Kamon +import kamon.trace.Span +import monix.execution.Scheduler +import scala.concurrent.Future + +import filodb.core.DatasetRef +import filodb.core.metadata.Column.ColumnType +import filodb.core.query._ +import filodb.memory.format.UTF8MapIteratorRowReader +import filodb.memory.format.ZeroCopyUTF8String._ +import filodb.query._ + +case class MetadataRemoteExec(queryEndpoint: String, + requestTimeoutMs: Long, + urlParams: Map[String, Any], + queryContext: QueryContext, + dispatcher: PlanDispatcher, + dataset: DatasetRef, + params: PromQlQueryParams) extends RemoteExec { + + private val columns = Seq(ColumnInfo("Labels", ColumnType.MapColumn)) + private val resultSchema = ResultSchema(columns, 1) + private val recordSchema = SerializedRangeVector.toSchema(columns) + private val builder = SerializedRangeVector.newBuilder() + + override def sendHttpRequest(execPlan2Span: Span, httpEndpoint: String, httpTimeoutMs: Long) + (implicit sched: Scheduler): Future[QueryResponse] = { + PromRemoteExec.httpMetadataGet(httpEndpoint, httpTimeoutMs, queryContext.submitTime, getUrlParams()) + .map { response => + response.unsafeBody match { + case Left(error) => QueryError(queryContext.queryId, error.error) + case Right(successResponse) => toQueryResponse(successResponse.data, queryContext.queryId, execPlan2Span) + } + } + } + + def toQueryResponse(data: Seq[Map[String, String]], id: String, parentSpan: kamon.trace.Span): QueryResponse = { + val span = Kamon.spanBuilder(s"create-queryresponse-${getClass.getSimpleName}") + .asChildOf(parentSpan) + .tag("query-id", id) + .start() + + val iteratorMap = data.map { r => r.map { v => (v._1.utf8, v._2.utf8) }} + + val rangeVector = IteratorBackedRangeVector(new CustomRangeVectorKey(Map.empty), + new UTF8MapIteratorRowReader(iteratorMap.toIterator)) + + val srvSeq = Seq(SerializedRangeVector(rangeVector, builder, recordSchema, printTree(false))) + + span.finish() + QueryResult(id, resultSchema, srvSeq) + } +} diff --git a/query/src/main/scala/filodb/query/exec/PromQlExec.scala b/query/src/main/scala/filodb/query/exec/PromQlExec.scala deleted file mode 100644 index af2a120ddd..0000000000 --- a/query/src/main/scala/filodb/query/exec/PromQlExec.scala +++ /dev/null @@ -1,149 +0,0 @@ -package filodb.query.exec - -import java.util.concurrent.TimeUnit - -import com.softwaremill.sttp.asynchttpclient.future.AsyncHttpClientFutureBackend -import com.softwaremill.sttp.circe._ -import com.typesafe.scalalogging.StrictLogging -import kamon.Kamon -import monix.eval.Task -import monix.execution.Scheduler -import scala.concurrent.Future -import scala.concurrent.duration._ -import scala.sys.ShutdownHookThread - -import filodb.core.DatasetRef -import filodb.core.metadata.Column.ColumnType -import filodb.core.query._ -import filodb.core.store.ChunkSource -import filodb.memory.format.RowReader -import filodb.memory.format.ZeroCopyUTF8String._ -import filodb.query._ - -case class PromQlExec(queryContext: QueryContext, - dispatcher: PlanDispatcher, - dataset: DatasetRef, - params: PromQlQueryParams) extends LeafExecPlan { - - protected def args: String = params.toString - import PromQlExec._ - - val builder = SerializedRangeVector.newBuilder() - - def limit: Int = ??? - - /** - * Sub classes should override this method to provide a concrete - * implementation of the operation represented by this exec plan - * node - */ - def doExecute(source: ChunkSource, - querySession: QuerySession) - (implicit sched: Scheduler): ExecResult = ??? - - override def execute(source: ChunkSource, - querySession: QuerySession) - (implicit sched: Scheduler): Task[QueryResponse] = { - val execPlan2Span = Kamon.spanBuilder(s"execute-${getClass.getSimpleName}") - .asChildOf(Kamon.currentSpan()) - .tag("query-id", queryContext.queryId) - .start() - - val queryResponse = PromQlExec.httpGet(params, queryContext.submitTime). - map { response => - response.unsafeBody match { - case Left(error) => QueryError(queryContext.queryId, error.error) - case Right(successResponse) => toQueryResponse(successResponse.data, queryContext.queryId, execPlan2Span) - } - } - // Please note that the following needs to be wrapped inside `runWithSpan` so that the context will be propagated - // across threads. Note that task/observable will not run on the thread where span is present since - // kamon uses thread-locals. - Kamon.runWithSpan(execPlan2Span, true) { - Task.fromFuture(queryResponse) - } - } - - // TODO: Set histogramMap=true and parse histogram maps. The problem is that code below assumes normal double - // schema. Would need to detect ahead of time to use TransientHistRow(), so we'd need to add schema to output, - // and detect it in execute() above. Need to discuss compatibility issues with Prometheus. - def toQueryResponse(data: Data, id: String, parentSpan: kamon.trace.Span): QueryResponse = { - val span = Kamon.spanBuilder(s"create-queryresponse-${getClass.getSimpleName}") - .asChildOf(parentSpan) - .tag("query-id", id) - .start() - val rangeVectors = data.result.map { r => - val samples = r.values.getOrElse(Seq(r.value.get)) - - val rv = new RangeVector { - val row = new TransientRow() - - override def key: RangeVectorKey = CustomRangeVectorKey(r.metric.map (m => m._1.utf8 -> m._2.utf8)) - - override def rows: Iterator[RowReader] = { - samples.iterator.collect { case v: Sampl => - row.setLong(0, v.timestamp * 1000) - row.setDouble(1, v.value) - row - } - } - - override def numRows: Option[Int] = Option(samples.size) - - } - SerializedRangeVector(rv, builder, recSchema, printTree(useNewline = false)) - } - span.finish() - QueryResult(id, resultSchema, rangeVectors) - } - -} - -object PromQlExec extends StrictLogging { - - import com.softwaremill.sttp._ - import io.circe.generic.auto._ - import net.ceedubs.ficus.Ficus._ - - val columns: Seq[ColumnInfo] = Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), - ColumnInfo("value", ColumnType.DoubleColumn)) - val recSchema = SerializedRangeVector.toSchema(columns) - val resultSchema = ResultSchema(columns, 1) - - // DO NOT REMOVE PromCirceSupport import below assuming it is unused - Intellij removes it in auto-imports :( . - // Needed to override Sampl case class Encoder. - import PromCirceSupport._ - implicit val backend = AsyncHttpClientFutureBackend() - - ShutdownHookThread(shutdown()) - - def httpGet(promQlQueryParams: PromQlQueryParams, submitTime: Long)(implicit scheduler: Scheduler): - Future[Response[scala.Either[DeserializationError[io.circe.Error], SuccessResponse]]] = { - val endpoint = promQlQueryParams.config.as[Option[String]]("buddy.http.endpoint").get - val queryTimeElapsed = System.currentTimeMillis() - submitTime - val buddyHttpTimeout = promQlQueryParams.config.as[Option[FiniteDuration]]("buddy.http.timeout"). - getOrElse(60000.millis) - val readTimeout = FiniteDuration(buddyHttpTimeout.toMillis - queryTimeElapsed, TimeUnit.MILLISECONDS) - var urlParams = Map("query" -> promQlQueryParams.promQl, - "start" -> promQlQueryParams.startSecs, - "end" -> promQlQueryParams.endSecs, - "step" -> promQlQueryParams.stepSecs, - "processFailure" -> promQlQueryParams.processFailure) - if (promQlQueryParams.spread.isDefined) urlParams = urlParams + ("spread" -> promQlQueryParams.spread.get) - - val url = uri"$endpoint?$urlParams" - logger.debug("promqlexec url is {}", url) - sttp - .get(url) - .readTimeout(readTimeout) - .response(asJson[SuccessResponse]) - .send() - } - - def shutdown(): Unit = - { - logger.info("Shutting PromQlExec http") - backend.close() - } -} - diff --git a/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala b/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala new file mode 100644 index 0000000000..04ad3ae302 --- /dev/null +++ b/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala @@ -0,0 +1,193 @@ +package filodb.query.exec + +import java.util.concurrent.TimeUnit + +import com.softwaremill.sttp.asynchttpclient.future.AsyncHttpClientFutureBackend +import com.softwaremill.sttp.circe._ +import com.typesafe.scalalogging.StrictLogging +import kamon.Kamon +import kamon.trace.Span +import monix.eval.Task +import monix.execution.Scheduler +import scala.concurrent.Future +import scala.concurrent.duration._ +import scala.sys.ShutdownHookThread + +import filodb.core.DatasetRef +import filodb.core.metadata.Column.ColumnType +import filodb.core.query._ +import filodb.core.store.ChunkSource +import filodb.memory.format.RowReader +import filodb.memory.format.ZeroCopyUTF8String._ +import filodb.query._ + +trait RemoteExec extends LeafExecPlan { + + val params: PromQlQueryParams + + val queryEndpoint: String + + val requestTimeoutMs: Long + + val urlParams: Map[String, Any] + + def args: String = s"${params.toString}, queryEndpoint=$queryEndpoint, " + + s"requestTimeoutMs=$requestTimeoutMs, limit=${queryContext.sampleLimit}" + + def limit: Int = ??? + + /** + * Since execute is already overrided here, doExecute() can be empty. + */ + def doExecute(source: ChunkSource, + querySession: QuerySession) + (implicit sched: Scheduler): ExecResult = ??? + + override def execute(source: ChunkSource, + querySession: QuerySession) + (implicit sched: Scheduler): Task[QueryResponse] = { + val execPlan2Span = Kamon.spanBuilder(s"execute-${getClass.getSimpleName}") + .asChildOf(Kamon.currentSpan()) + .tag("query-id", queryContext.queryId) + .start() + + if (queryEndpoint == null) { + throw new BadQueryException("Remote Query endpoint can not be null in RemoteExec.") + } + + // Please note that the following needs to be wrapped inside `runWithSpan` so that the context will be propagated + // across threads. Note that task/observable will not run on the thread where span is present since + // kamon uses thread-locals. + Kamon.runWithSpan(execPlan2Span, true) { + Task.fromFuture(sendHttpRequest(execPlan2Span, queryEndpoint, requestTimeoutMs)) + } + } + + def sendHttpRequest(execPlan2Span: Span, httpEndpoint: String, httpTimeoutMs: Long) + (implicit sched: Scheduler): Future[QueryResponse] + + def getUrlParams(): Map[String, Any] = { + var finalUrlParams = urlParams ++ + Map("start" -> params.startSecs, + "end" -> params.endSecs, + "time" -> params.endSecs, + "step" -> params.stepSecs, + "processFailure" -> params.processFailure, + "processMultiPartition" -> params.processMultiPartition, + "verbose" -> params.verbose) + if (params.spread.isDefined) finalUrlParams = finalUrlParams + ("spread" -> params.spread.get) + finalUrlParams + } + +} + +case class PromQlRemoteExec(queryEndpoint: String, + requestTimeoutMs: Long, + queryContext: QueryContext, + dispatcher: PlanDispatcher, + dataset: DatasetRef, + params: PromQlQueryParams, + numberColumnRequired: Boolean = false) extends RemoteExec { + + private val columns = Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), + ColumnInfo(if (numberColumnRequired) "number" else "value", ColumnType.DoubleColumn)) + private val recSchema = SerializedRangeVector.toSchema(columns) + private val resultSchema = ResultSchema(columns, 1) + private val builder = SerializedRangeVector.newBuilder() + + override val urlParams = Map("query" -> params.promQl) + + override def sendHttpRequest(execPlan2Span: Span, httpEndpoint: String, httpTimeoutMs: Long) + (implicit sched: Scheduler): Future[QueryResponse] = { + PromRemoteExec.httpGet(queryEndpoint, requestTimeoutMs, queryContext.submitTime, getUrlParams()) + .map { response => + response.unsafeBody match { + case Left(error) => QueryError(queryContext.queryId, error.error) + case Right(successResponse) => toQueryResponse(successResponse.data, queryContext.queryId, execPlan2Span) + } + } + } + + // TODO: Set histogramMap=true and parse histogram maps. The problem is that code below assumes normal double + // schema. Would need to detect ahead of time to use TransientHistRow(), so we'd need to add schema to output, + // and detect it in execute() above. Need to discuss compatibility issues with Prometheus. + def toQueryResponse(data: Data, id: String, parentSpan: kamon.trace.Span): QueryResponse = { + val span = Kamon.spanBuilder(s"create-queryresponse-${getClass.getSimpleName}") + .asChildOf(parentSpan) + .tag("query-id", id) + .start() + val rangeVectors = data.result.map { r => + val samples = r.values.getOrElse(Seq(r.value.get)) + + val rv = new RangeVector { + val row = new TransientRow() + + override def key: RangeVectorKey = CustomRangeVectorKey(r.metric.map (m => m._1.utf8 -> m._2.utf8)) + + override def rows: Iterator[RowReader] = { + samples.iterator.collect { case v: Sampl => + row.setLong(0, v.timestamp * 1000) + row.setDouble(1, v.value) + row + } + } + + override def numRows: Option[Int] = Option(samples.size) + + } + SerializedRangeVector(rv, builder, recSchema, printTree(useNewline = false)) + // TODO: Handle stitching with verbose flag + } + span.finish() + QueryResult(id, resultSchema, rangeVectors) + } + +} + +object PromRemoteExec extends StrictLogging { + + import com.softwaremill.sttp._ + import io.circe.generic.auto._ + + // DO NOT REMOVE PromCirceSupport import below assuming it is unused - Intellij removes it in auto-imports :( . + // Needed to override Sampl case class Encoder. + import PromCirceSupport._ + implicit val backend = AsyncHttpClientFutureBackend() + + ShutdownHookThread(shutdown()) + + def httpGet(httpEndpoint: String, httpTimeoutMs: Long, submitTime: Long, urlParams: Map[String, Any]) + (implicit scheduler: Scheduler): + Future[Response[scala.Either[DeserializationError[io.circe.Error], SuccessResponse]]] = { + val queryTimeElapsed = System.currentTimeMillis() - submitTime + val readTimeout = FiniteDuration(httpTimeoutMs - queryTimeElapsed, TimeUnit.MILLISECONDS) + val url = uri"$httpEndpoint?$urlParams" + logger.debug("promQlExec url={}", url) + sttp + .get(url) + .readTimeout(readTimeout) + .response(asJson[SuccessResponse]) + .send() + } + + def httpMetadataGet(httpEndpoint: String, httpTimeoutMs: Long, submitTime: Long, urlParams: Map[String, Any]) + (implicit scheduler: Scheduler): + Future[Response[scala.Either[DeserializationError[io.circe.Error], MetadataSuccessResponse]]] = { + val queryTimeElapsed = System.currentTimeMillis() - submitTime + val readTimeout = FiniteDuration(httpTimeoutMs - queryTimeElapsed, TimeUnit.MILLISECONDS) + val url = uri"$httpEndpoint?$urlParams" + logger.debug("promMetadataExec url={}", url) + sttp + .get(url) + .readTimeout(readTimeout) + .response(asJson[MetadataSuccessResponse]) + .send() + } + + def shutdown(): Unit = + { + logger.info("Shutting PromQlExec http") + backend.close() + } +} + diff --git a/query/src/test/scala/filodb/query/LogicalPlanSpec.scala b/query/src/test/scala/filodb/query/LogicalPlanSpec.scala index 57e9fd0f6f..90333341d3 100644 --- a/query/src/test/scala/filodb/query/LogicalPlanSpec.scala +++ b/query/src/test/scala/filodb/query/LogicalPlanSpec.scala @@ -12,7 +12,7 @@ class LogicalPlanSpec extends FunSpec with Matchers { it("should get labelValueOps from logicalPlan") { val rawSeries = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("_name_", Equals("MetricName")), - ColumnFilter("instance", NotEquals("Inst-0"))), Seq("_name_", "instance"), Some(300000), None) + ColumnFilter("instance", NotEquals("Inst-0"))), Seq("job", "instance"), Some(300000), None) val periodicSeriesWithWindowing = PeriodicSeriesWithWindowing(rawSeries, 1000, 500, 5000, 100, SumOverTime) val res = LogicalPlan.getLabelValueOperatorsFromLogicalPlan(periodicSeriesWithWindowing) @@ -29,7 +29,7 @@ class LogicalPlanSpec extends FunSpec with Matchers { it("should get labelValueOps from logicalPlan with filter In") { val rawSeries = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("_name_", Equals("MetricName")), - ColumnFilter("instance", In(Set("Inst-1", "Inst-0")))), Seq("_name_", "instance"), Some(300000), None) + ColumnFilter("instance", In(Set("Inst-1", "Inst-0")))), Seq("job", "instance"), Some(300000), None) val periodicSeriesWithWindowing = PeriodicSeriesWithWindowing(rawSeries, 1000, 500, 5000, 100, SumOverTime) val res = LogicalPlan.getLabelValueOperatorsFromLogicalPlan(periodicSeriesWithWindowing) @@ -46,7 +46,7 @@ class LogicalPlanSpec extends FunSpec with Matchers { it("should get labelValueOps from BinaryJoin LogicalPlan") { val rawSeriesLhs = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("_name_", Equals("MetricName1")), - ColumnFilter("instance", EqualsRegex("Inst-0"))), Seq("_name_", "instance"), Some(300000), None) + ColumnFilter("instance", EqualsRegex("Inst-0"))), Seq("job", "instance"), Some(300000), None) val lhs = PeriodicSeries(rawSeriesLhs, 1000, 500, 50000) val rawSeriesRhs = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("job", Equals("MetricName2")), @@ -90,7 +90,7 @@ class LogicalPlanSpec extends FunSpec with Matchers { it("should get MetricName from logicalPlan") { val rawSeries = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("_name_", Equals("MetricName")), - ColumnFilter("instance", Equals("Inst-0"))), Seq("_name_", "instance"), Some(300000), None) + ColumnFilter("instance", Equals("Inst-0"))), Seq("job", "instance"), Some(300000), None) val periodicSeriesWithWindowing = PeriodicSeriesWithWindowing(rawSeries, 1000, 500, 5000, 100, SumOverTime) val res = LogicalPlan.getLabelValueFromLogicalPlan(periodicSeriesWithWindowing, "_name_") @@ -100,7 +100,7 @@ class LogicalPlanSpec extends FunSpec with Matchers { it("should get LabelName from logicalPlan with filter In") { val rawSeries = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("_name_", Equals("MetricName")), - ColumnFilter("instance", In(Set("Inst-0", "Inst-1")))), Seq("_name_", "instance"), Some(300000), None) + ColumnFilter("instance", In(Set("Inst-0", "Inst-1")))), Seq("job", "instance"), Some(300000), None) val periodicSeriesWithWindowing = PeriodicSeriesWithWindowing(rawSeries, 1000, 500, 5000, 100, SumOverTime) val res = LogicalPlan.getLabelValueFromLogicalPlan(periodicSeriesWithWindowing, "instance") @@ -126,7 +126,7 @@ class LogicalPlanSpec extends FunSpec with Matchers { it("should return None if label value is not present in logicalPlan") { val rawSeries = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("_name_", Equals("MetricName")), - ColumnFilter("instance", Equals("Inst-0"))), Seq("_name_", "instance"), Some(300000), None) + ColumnFilter("instance", Equals("Inst-0"))), Seq("job", "instance"), Some(300000), None) val periodicSeriesWithWindowing = PeriodicSeriesWithWindowing(rawSeries, 1000, 500, 5000, 100, SumOverTime) val res = LogicalPlan.getLabelValueFromLogicalPlan(periodicSeriesWithWindowing, "_name") @@ -136,7 +136,7 @@ class LogicalPlanSpec extends FunSpec with Matchers { it("should concatenate results from lhs and rhs for BinaryJoin LogicalPlan") { val rawSeriesLhs = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("_name_", Equals("MetricName1")), - ColumnFilter("instance", Equals("Inst-0"))), Seq("_name_", "instance"), Some(300000), None) + ColumnFilter("instance", Equals("Inst-0"))), Seq("job", "instance"), Some(300000), None) val lhs = PeriodicSeries(rawSeriesLhs, 1000, 500, 50000) val rawSeriesRhs = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("job", Equals("MetricName2")), diff --git a/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala b/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala index a360807b1e..c9f7a5295f 100644 --- a/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala @@ -38,7 +38,7 @@ class MetadataExecSpec extends FunSpec with Matchers with ScalaFutures with Befo ("http_resp_time", Map("instance"->"someHost:8787", "job"->"myCoolService")) ) - val addlLabels = Map("_type_" -> "prom-counter", "_firstSampleTime_" -> "-1", "_lastSampleTime_" -> "-1") + val addlLabels = Map("_type_" -> "prom-counter") val expectedLabelValues = partKeyLabelValues.map { case (metric, tags) => tags + ("_metric_" -> metric) ++ addlLabels } @@ -86,7 +86,7 @@ class MetadataExecSpec extends FunSpec with Matchers with ScalaFutures with Befo ColumnFilter("job", Filter.Equals("myCoolService".utf8))) val execPlan = LabelValuesExec(QueryContext(), dummyDispatcher, - timeseriesDataset.ref, 0, filters, Seq("job"), 10) + timeseriesDataset.ref, 0, filters, Seq("job"), now-5000, now) val resp = execPlan.execute(memStore, querySession).runAsync.futureValue val result = resp match { diff --git a/query/src/test/scala/filodb/query/exec/PromQlExecSpec.scala b/query/src/test/scala/filodb/query/exec/PromQlRemoteExecSpec.scala similarity index 54% rename from query/src/test/scala/filodb/query/exec/PromQlExecSpec.scala rename to query/src/test/scala/filodb/query/exec/PromQlRemoteExecSpec.scala index 955711d834..8fa494822e 100644 --- a/query/src/test/scala/filodb/query/exec/PromQlExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/PromQlRemoteExecSpec.scala @@ -1,17 +1,17 @@ package filodb.query.exec -import com.typesafe.config.ConfigFactory import kamon.Kamon import monix.eval.Task import monix.execution.Scheduler import org.scalatest.{FunSpec, Matchers} import org.scalatest.concurrent.ScalaFutures + import filodb.core.metadata.{Dataset, DatasetOptions} import filodb.core.query.{PromQlQueryParams, QueryContext} import filodb.query import filodb.query.{Data, QueryResponse, QueryResult, Sampl} -class PromQlExecSpec extends FunSpec with Matchers with ScalaFutures { +class PromQlRemoteExecSpec extends FunSpec with Matchers with ScalaFutures { val timeseriesDataset = Dataset.make("timeseries", Seq("tags:map"), Seq("timestamp:ts", "value:double:detectDrops=true"), @@ -23,10 +23,10 @@ class PromQlExecSpec extends FunSpec with Matchers with ScalaFutures { } val queryContext = QueryContext() - val params = PromQlQueryParams(ConfigFactory.empty(), "", 0, 0 , 0) + val params = PromQlQueryParams("", 0, 0 , 0) it ("should convert matrix Data to QueryResponse ") { val expectedResult = List((1000000, 1.0), (2000000, 2.0), (3000000, 3.0)) - val exec = PromQlExec(queryContext, dummyDispatcher, timeseriesDataset.ref, params) + val exec = PromQlRemoteExec("", 60000, queryContext, dummyDispatcher, timeseriesDataset.ref, params) val result = query.Result (Map("instance" ->"inst1"), Some(Seq(Sampl(1000, 1), Sampl(2000, 2), Sampl(3000, 3))), None) val res = exec.toQueryResponse(Data("vector", Seq(result)), "id", Kamon.currentSpan()) res.isInstanceOf[QueryResult] shouldEqual true @@ -39,7 +39,7 @@ class PromQlExecSpec extends FunSpec with Matchers with ScalaFutures { it ("should convert vector Data to QueryResponse ") { val expectedResult = List((1000000, 1.0)) - val exec = PromQlExec(queryContext, dummyDispatcher, timeseriesDataset.ref, params) + val exec = PromQlRemoteExec("", 60000, queryContext, dummyDispatcher, timeseriesDataset.ref, params) val result = query.Result (Map("instance" ->"inst1"), None, Some(Sampl(1000, 1))) val res = exec.toQueryResponse(Data("vector", Seq(result)), "id", Kamon.currentSpan()) res.isInstanceOf[QueryResult] shouldEqual true @@ -50,4 +50,28 @@ class PromQlExecSpec extends FunSpec with Matchers with ScalaFutures { } + it ("should convert vector Data to QueryResponse for MetadataQuery") { + val exec = MetadataRemoteExec("", 60000, Map.empty, + queryContext, dummyDispatcher, timeseriesDataset.ref, params) + val map1 = Map("instance" -> "inst-1", "last-sample" -> "6377838" ) + val map2 = Map("instance" -> "inst-2", "last-sample" -> "6377834" ) + val res = exec.toQueryResponse(Seq(map1, map2), "id", Kamon.currentSpan()) + res.isInstanceOf[QueryResult] shouldEqual true + val queryResult = res.asInstanceOf[QueryResult] + val data = queryResult.result.flatMap(x=>x.rows.map{ r => r.getAny(0) }.toList) + data(0) shouldEqual(map1) + data(1) shouldEqual(map2) + } + + it ("should convert vector Data to QueryResponse for Metadata series query") { + val exec = MetadataRemoteExec("", 60000, Map.empty, queryContext, dummyDispatcher, timeseriesDataset.ref, params) + val map1 = Map("instance" -> "inst-1", "last-sample" -> "6377838" ) + val map2 = Map("instance" -> "inst-2", "last-sample" -> "6377834" ) + val res = exec.toQueryResponse(Seq(map1, map2), "id", Kamon.currentSpan()) + res.isInstanceOf[QueryResult] shouldEqual true + val queryResult = res.asInstanceOf[QueryResult] + val data = queryResult.result.flatMap(x=>x.rows.map{ r => r.getAny(0) }.toList) + data(0) shouldEqual(map1) + data(1) shouldEqual(map2) + } } From a6d3f385954c39899059d71bf55b5ae66070346a Mon Sep 17 00:00:00 2001 From: "Brian S. O'Neill" Date: Wed, 17 Jun 2020 10:25:56 -0700 Subject: [PATCH 02/22] fix(memory): Improve exclusive timeout accuracy, and fix a potential lost unpark of an exclusive waiter. (#791) fix(memory): Improve exclusive timeout accuracy, and fix a potential lost unpark of an exclusive waiter. --- memory/src/main/java/filodb/memory/Latch.java | 36 ++++++++++++++++--- 1 file changed, 31 insertions(+), 5 deletions(-) diff --git a/memory/src/main/java/filodb/memory/Latch.java b/memory/src/main/java/filodb/memory/Latch.java index 296854c422..6a6b2df24c 100644 --- a/memory/src/main/java/filodb/memory/Latch.java +++ b/memory/src/main/java/filodb/memory/Latch.java @@ -739,13 +739,22 @@ private boolean acquire(final WaitNode node) { return true; } - if (!(node instanceof Shared) && mLatchState >= 0) { - // Unpark any shared waiters that queued behind this exclusive request. + int state = mLatchState; + if (state >= 0) { + // Unpark any waiters that queued behind this request. WaitNode wnode = node; - while ((wnode = wnode.mNext) instanceof Shared) { + while ((wnode = wnode.mNext) != null) { Object waiter = wnode.mWaiter; if (waiter instanceof Thread) { - LockSupport.unpark((Thread) waiter); + if (wnode instanceof Shared) { + LockSupport.unpark((Thread) waiter); + } else { + if (state == 0) { + LockSupport.unpark((Thread) waiter); + } + // No need to iterate past an exclusive waiter. + break; + } } } } @@ -1014,7 +1023,7 @@ int tryAcquire(Latch latch) { } Thread.onSpinWait(); } - if (++trials >= SPIN_LIMIT >> 1) { + if (++trials >= SPIN_LIMIT >> 1 || timedOut()) { return -1; } // Yield to avoid parking. @@ -1022,6 +1031,10 @@ int tryAcquire(Latch latch) { } } + protected boolean timedOut() { + return false; + } + @Override public String toString() { var b = new StringBuilder(); @@ -1058,6 +1071,19 @@ final boolean parkNow(Latch latch) { return (mNanosTimeout = mEndNanos - System.nanoTime()) <= 0; } } + + @Override + protected boolean timedOut() { + if (mNanosTimeout >= 0) { + long timeout = mEndNanos - System.nanoTime(); + if (timeout <= 0) { + mNanosTimeout = 0; + return true; + } + mNanosTimeout = timeout; + } + return false; + } } static class Shared extends WaitNode { From 8c0f7e5a0da89497d2442ee459a68b98f7e0b9ea Mon Sep 17 00:00:00 2001 From: Jackson Jeyapaul Date: Thu, 18 Jun 2020 12:35:38 -0700 Subject: [PATCH 03/22] misc(query): update logicalPlan utility methods (#794) --- .../queryplanner/LogicalPlanUtils.scala | 7 +- .../queryplanner/MultiPartitionPlanner.scala | 11 +- .../queryplanner/SingleClusterPlanner.scala | 2 +- .../queryplanner/SinglePartitionPlanner.scala | 5 +- .../MultiPartitionPlannerSpec.scala | 21 ++ .../main/scala/filodb/query/LogicalPlan.scala | 65 ++--- .../scala/filodb/query/LogicalPlanSpec.scala | 246 +++++++++++++----- 7 files changed, 251 insertions(+), 106 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala index 4482404058..7be1d4a0f2 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala @@ -137,9 +137,10 @@ object LogicalPlanUtils { } } - def getMetricName(logicalPlan: LogicalPlan, datasetMetricColumn: String): Option[Seq[String]] = { - val metricName = LogicalPlan.getLabelValueFromLogicalPlan(logicalPlan, PromMetricLabel) - if (metricName.isEmpty) LogicalPlan.getLabelValueFromLogicalPlan(logicalPlan, datasetMetricColumn) + def getMetricName(logicalPlan: LogicalPlan, datasetMetricColumn: String): Set[String] = { + val columnFilterGroup = LogicalPlan.getColumnFilterGroup(logicalPlan) + val metricName = LogicalPlan.getColumnValues(columnFilterGroup, PromMetricLabel) + if (metricName.isEmpty) LogicalPlan.getColumnValues(columnFilterGroup, datasetMetricColumn) else metricName } diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala index 56536b730b..b25ae286a8 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala @@ -43,8 +43,11 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider } } - private def getRoutingKeys(logicalPlan: LogicalPlan) = dataset.options.nonMetricShardColumns - .map(x => (x, LogicalPlan.getLabelValueFromLogicalPlan(logicalPlan, x))) + private def getRoutingKeys(logicalPlan: LogicalPlan) = { + val columnFilterGroup = LogicalPlan.getColumnFilterGroup(logicalPlan) + dataset.options.nonMetricShardColumns + .map(x => (x, LogicalPlan.getColumnValues(columnFilterGroup, x))) + } private def generateRemoteExecParams(queryParams: PromQlQueryParams, startMs: Long, endMs: Long) = { PromQlQueryParams(queryParams.promQl, startMs / 1000, queryParams.stepSecs, endMs / 1000, queryParams.spread, @@ -57,9 +60,9 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider * @param queryParams PromQlQueryParams having query details * @param logicalPlan Logical plan */ - private def partitionUtil(routingKeys: Seq[(String, Option[scala.Seq[String]])], queryParams: PromQlQueryParams, + private def partitionUtil(routingKeys: Seq[(String, Set[String])], queryParams: PromQlQueryParams, logicalPlan: LogicalPlan) = { - val routingKeyMap = routingKeys.map(x => (x._1, x._2.get.head)).toMap + val routingKeyMap = routingKeys.map(x => (x._1, x._2.head)).toMap val offsetMs = LogicalPlanUtils.getOffsetMillis(logicalPlan) val periodicSeriesTimeWithOffset = TimeRange((queryParams.startSecs * 1000) - offsetMs, (queryParams.endSecs * 1000) - offsetMs) diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala index 7d83002b1d..652ec6f429 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala @@ -73,7 +73,7 @@ class SingleClusterPlanner(dsRef: DatasetRef, def materialize(logicalPlan: LogicalPlan, qContext: QueryContext): ExecPlan = { - if (shardMapperFunc.numShards == 0) throw new IllegalStateException("No shards available") + if (shardMapperFunc.numShards <= 0) throw new IllegalStateException("No shards available") val materialized = walkLogicalPlanTree(logicalPlan, qContext) match { case PlanResult(Seq(justOne), stitch) => diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SinglePartitionPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SinglePartitionPlanner.scala index b00f7904f2..3c39af6988 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SinglePartitionPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SinglePartitionPlanner.scala @@ -32,8 +32,9 @@ class SinglePartitionPlanner(planners: Map[String, QueryPlanner], plannerSelecto * If logical plan does not have metric, first planner present in planners is returned */ private def getPlanner(logicalPlan: LogicalPlan): QueryPlanner = { - LogicalPlanUtils.getMetricName(logicalPlan, datasetMetricColumn). - map(x => planners.get(plannerSelector(x.head)).get).getOrElse(planners.values.head) + val planner = LogicalPlanUtils.getMetricName(logicalPlan, datasetMetricColumn) + .map(x => planners.get(plannerSelector(x)).get) + if(planner.isEmpty) planners.values.head else planner.head } private def materializeSimpleQuery(logicalPlan: LogicalPlan, qContext: QueryContext): ExecPlan = { diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala index f3c19c8924..ca71bfd6a2 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala @@ -11,6 +11,7 @@ import filodb.core.query.{PromQlQueryParams, QueryConfig, QueryContext} import filodb.core.store.TimeRangeChunkScan import filodb.prometheus.ast.TimeStepParams import filodb.prometheus.parse.Parser +import filodb.query.LogicalPlan import filodb.query.exec._ class MultiPartitionPlannerSpec extends FunSpec with Matchers { @@ -165,6 +166,26 @@ class MultiPartitionPlannerSpec extends FunSpec with Matchers { execPlan.isInstanceOf[BinaryJoinExec] shouldEqual (true) } + it ("should have equal hashcode for identical getColumnFilterGroup") { + + val lp1 = Parser.queryRangeToLogicalPlan("test1{inst = \"inst-001\", job = \"app\", host = \"localhost\"}", + TimeStepParams(1000, 100, 2000)) + + val lp2 = Parser.queryRangeToLogicalPlan("test1{job = \"app\", host = \"localhost\", inst = \"inst-001\"}", + TimeStepParams(3000, 100, 5000)) + + val res1 = LogicalPlan.getColumnFilterGroup(lp1) + val res2 = LogicalPlan.getColumnFilterGroup(lp2) + + res1.size.shouldEqual(1) + res1(0).size.shouldEqual(4) + res2.size.shouldEqual(1) + res2(0).size.shouldEqual(4) + + res1(0).hashCode() shouldEqual res2(0).hashCode() + + } + it ("should generate PromQlRemoteExec plan for BinaryJoin when lhs and rhs are in same remote partition") { def partitions(timeRange: TimeRange): List[PartitionAssignment] = List(PartitionAssignment("remote", "remote-url", TimeRange(timeRange.startMs, timeRange.endMs))) diff --git a/query/src/main/scala/filodb/query/LogicalPlan.scala b/query/src/main/scala/filodb/query/LogicalPlan.scala index c12409fcbb..6f29de9536 100644 --- a/query/src/main/scala/filodb/query/LogicalPlan.scala +++ b/query/src/main/scala/filodb/query/LogicalPlan.scala @@ -1,6 +1,6 @@ package filodb.query -import filodb.core.query.{ColumnFilter, RangeParams} +import filodb.core.query.{ColumnFilter, Filter, RangeParams} //scalastyle:off number.of.types sealed trait LogicalPlan { @@ -337,10 +337,6 @@ case class ApplyAbsentFunction(vectors: PeriodicSeriesPlan, override def endMs: Long = vectors.endMs } -case class LabelValueOperator(columnName: String, value: Seq[String], operator: String) - -case class LabelValueOperatorGroup(labelValueOperators: Seq[LabelValueOperator]) - object LogicalPlan { /** * Get leaf Logical Plans @@ -353,53 +349,46 @@ object LogicalPlan { } } - def getLabelValueFromLogicalPlan(logicalPlan: LogicalPlan, labelName: String): Option[Seq[String]] = { - getAllLabelValueFromGroups(getLabelValueOperatorsFromLogicalPlan(logicalPlan), labelName) + def getColumnValues(logicalPlan: LogicalPlan, labelName: String): Set[String] = { + getColumnValues(getColumnFilterGroup(logicalPlan), labelName) } - def getAllLabelValueFromGroups(labelValues: Option[Seq[LabelValueOperatorGroup]], - labelName: String): Option[Seq[String]] = { - labelValues match { - case None => None - case _ => labelValues.get.flatMap(group => getLabelValueFromGroup(labelName, group)).distinct match { - case Nil => None - case lVFilters: Seq[String] => Some(lVFilters) - } + def getColumnValues(columnFilterGroup: Seq[Set[ColumnFilter]], + labelName: String): Set[String] = { + columnFilterGroup.flatMap (columnFilters => getColumnValues(columnFilters, labelName)) match { + case columnValues: Iterable[String] => if (columnValues.isEmpty) Set.empty else columnValues.toSet + case _ => Set.empty } } - def getLabelValueFromGroup(labelName: String, group: LabelValueOperatorGroup): Seq[String] = { - group.labelValueOperators.flatMap(lops => { - lops.columnName.equals(labelName) match { - case true => lops.value - case false => Seq() + def getColumnValues(columnFilters: Set[ColumnFilter], labelName: String): Set[String] = { + columnFilters.flatMap(cFilter => { + cFilter.column == labelName match { + case true => cFilter.filter.valuesStrings.map(_.toString) + case false => Seq.empty } }) } - private def getLabelValueOpsFromFilters(filters: Seq[ColumnFilter]): Option[LabelValueOperatorGroup] = { - Some(LabelValueOperatorGroup(filters.map(cf => LabelValueOperator(cf.column, - cf.filter.valuesStrings.map(_.toString).toSeq.sorted, cf.filter.operatorString)))) - } - - def getLabelValueOperatorsFromLogicalPlan(logicalPlan: LogicalPlan): Option[Seq[LabelValueOperatorGroup]] = { - LogicalPlan.findLeafLogicalPlans(logicalPlan).flatMap { lp => + def getColumnFilterGroup(logicalPlan: LogicalPlan): Seq[Set[ColumnFilter]] = { + LogicalPlan.findLeafLogicalPlans(logicalPlan) map { lp => lp match { - case lp: LabelValues => Some( - LabelValueOperatorGroup( - lp.labelConstraints.map(lbc => LabelValueOperator(lbc._1, Seq(lbc._2), "=")).toSeq)) - case lp: RawSeries => getLabelValueOpsFromFilters(lp.filters) - case lp: RawChunkMeta => getLabelValueOpsFromFilters(lp.filters) - case lp: SeriesKeysByFilters => getLabelValueOpsFromFilters(lp.filters) - case _: ScalarTimeBasedPlan => Nil // Plan does not have labels - case _: ScalarFixedDoublePlan => Nil - case _: ScalarBinaryOperation => Nil + case lp: LabelValues => + lp.labelConstraints.map(lbc => ColumnFilter(lbc._1, Filter.Equals(lbc._2))) toSet + case lp: RawSeries => lp.filters toSet + case lp: RawChunkMeta => lp.filters toSet + case lp: SeriesKeysByFilters => lp.filters toSet + case _: ScalarTimeBasedPlan => Set.empty[ColumnFilter] // Plan does not have labels + case _: ScalarFixedDoublePlan => Set.empty[ColumnFilter] + case _: ScalarBinaryOperation => Set.empty[ColumnFilter] case _ => throw new BadQueryException(s"Invalid logical plan $logicalPlan") } } match { - case Nil => None - case groupSeq: Seq[LabelValueOperatorGroup] => Some(groupSeq) + case groupSeq: Seq[Set[ColumnFilter]] => + if (groupSeq.isEmpty || groupSeq.forall(_.isEmpty)) Seq.empty else groupSeq + case _ => Seq.empty } } + } //scalastyle:on number.of.types \ No newline at end of file diff --git a/query/src/test/scala/filodb/query/LogicalPlanSpec.scala b/query/src/test/scala/filodb/query/LogicalPlanSpec.scala index 90333341d3..60eebfa6b0 100644 --- a/query/src/test/scala/filodb/query/LogicalPlanSpec.scala +++ b/query/src/test/scala/filodb/query/LogicalPlanSpec.scala @@ -9,44 +9,57 @@ import org.scalatest.{FunSpec, Matchers} class LogicalPlanSpec extends FunSpec with Matchers { - it("should get labelValueOps from logicalPlan") { + it("should get columnFilterGroup from logicalPlan") { val rawSeries = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("_name_", Equals("MetricName")), - ColumnFilter("instance", NotEquals("Inst-0"))), Seq("job", "instance"), Some(300000), None) + ColumnFilter("instance", NotEquals("Inst-0"))), Seq("_name_", "instance"), Some(300000), None) val periodicSeriesWithWindowing = PeriodicSeriesWithWindowing(rawSeries, 1000, 500, 5000, 100, SumOverTime) - val res = LogicalPlan.getLabelValueOperatorsFromLogicalPlan(periodicSeriesWithWindowing) - res.get.size.shouldEqual(1) - res.get(0).labelValueOperators.size.shouldEqual(2) - res.get(0).labelValueOperators(0).columnName.shouldEqual("_name_") - res.get(0).labelValueOperators(0).value.shouldEqual(Seq("MetricName")) - res.get(0).labelValueOperators(0).operator.shouldEqual("=") - res.get(0).labelValueOperators(1).columnName.shouldEqual("instance") - res.get(0).labelValueOperators(1).value.shouldEqual(Seq("Inst-0")) - res.get(0).labelValueOperators(1).operator.shouldEqual("!=") + val res = LogicalPlan.getColumnFilterGroup(periodicSeriesWithWindowing) + res.size.shouldEqual(1) + res(0).size.shouldEqual(2) + for (cfSet <- res(0)) { + if (cfSet.column == "_name_") { + cfSet.column.shouldEqual("_name_") + cfSet.filter.operatorString.shouldEqual("=") + cfSet.filter.valuesStrings shouldEqual(Set("MetricName")) + } else if (cfSet.column == "instance") { + cfSet.column.shouldEqual("instance") + cfSet.filter.operatorString.shouldEqual("!=") + cfSet.filter.valuesStrings shouldEqual(Set("Inst-0")) + } else { + fail("invalid entry in column filter sequence " + cfSet) + } + } } - it("should get labelValueOps from logicalPlan with filter In") { + it("should get columnFilterGroup from logicalPlan with filter In") { val rawSeries = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("_name_", Equals("MetricName")), - ColumnFilter("instance", In(Set("Inst-1", "Inst-0")))), Seq("job", "instance"), Some(300000), None) + ColumnFilter("instance", In(Set("Inst-1", "Inst-0")))), Seq("_name_", "instance"), Some(300000), None) val periodicSeriesWithWindowing = PeriodicSeriesWithWindowing(rawSeries, 1000, 500, 5000, 100, SumOverTime) - val res = LogicalPlan.getLabelValueOperatorsFromLogicalPlan(periodicSeriesWithWindowing) - res.get.size.shouldEqual(1) - res.get(0).labelValueOperators.size.shouldEqual(2) - res.get(0).labelValueOperators(0).columnName.shouldEqual("_name_") - res.get(0).labelValueOperators(0).value.shouldEqual(Seq("MetricName")) - res.get(0).labelValueOperators(0).operator.shouldEqual("=") - res.get(0).labelValueOperators(1).columnName.shouldEqual("instance") - res.get(0).labelValueOperators(1).value.shouldEqual(Seq("Inst-0", "Inst-1")) - res.get(0).labelValueOperators(1).operator.shouldEqual("in") + val res = LogicalPlan.getColumnFilterGroup(periodicSeriesWithWindowing) + res.size.shouldEqual(1) + for (cfSet <- res(0)) { + if (cfSet.column.equals("_name_")) { + cfSet.column.shouldEqual("_name_") + cfSet.filter.operatorString.shouldEqual("=") + cfSet.filter.valuesStrings shouldEqual(Set("MetricName")) + } else if (cfSet.column.equals("instance")) { + cfSet.column.shouldEqual("instance") + cfSet.filter.operatorString.shouldEqual("in") + cfSet.filter.valuesStrings shouldEqual(Set("Inst-0", "Inst-1")) + } else { + fail("invalid entry in column filter sequence " + cfSet) + } + } } - it("should get labelValueOps from BinaryJoin LogicalPlan") { + it("should get columnFilterGroup from BinaryJoin LogicalPlan") { val rawSeriesLhs = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("_name_", Equals("MetricName1")), - ColumnFilter("instance", EqualsRegex("Inst-0"))), Seq("job", "instance"), Some(300000), None) + ColumnFilter("instance", EqualsRegex("Inst-0"))), Seq("_name_", "instance"), Some(300000), None) val lhs = PeriodicSeries(rawSeriesLhs, 1000, 500, 50000) val rawSeriesRhs = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("job", Equals("MetricName2")), @@ -55,56 +68,69 @@ class LogicalPlanSpec extends FunSpec with Matchers { val binaryJoin = BinaryJoin(lhs, DIV, OneToOne, rhs) - val res = LogicalPlan.getLabelValueOperatorsFromLogicalPlan(binaryJoin) - res.get.size.shouldEqual(2) - res.get(0).labelValueOperators.size.shouldEqual(2) - res.get(0).labelValueOperators(0).columnName.shouldEqual("_name_") - res.get(0).labelValueOperators(0).value.shouldEqual(Seq("MetricName1")) - res.get(0).labelValueOperators(0).operator.shouldEqual("=") - res.get(0).labelValueOperators(1).columnName.shouldEqual("instance") - res.get(0).labelValueOperators(1).value.shouldEqual(Seq("Inst-0")) - res.get(0).labelValueOperators(1).operator.shouldEqual("=~") - res.get(1).labelValueOperators.size.shouldEqual(2) - res.get(1).labelValueOperators(0).columnName.shouldEqual("job") - res.get(1).labelValueOperators(0).value.shouldEqual(Seq("MetricName2")) - res.get(1).labelValueOperators(0).operator.shouldEqual("=") - res.get(1).labelValueOperators(1).columnName.shouldEqual("instance") - res.get(1).labelValueOperators(1).value.shouldEqual(Seq("Inst-1")) - res.get(1).labelValueOperators(1).operator.shouldEqual("!~") + val res = LogicalPlan.getColumnFilterGroup(binaryJoin) + + res.size.shouldEqual(2) + res(0).size.shouldEqual(2) + for (cfSet <- res(0)) { + if (cfSet.column == "_name_") { + cfSet.column.shouldEqual("_name_") + cfSet.filter.operatorString.shouldEqual("=") + cfSet.filter.valuesStrings shouldEqual(Set("MetricName1")) + } else if (cfSet.column == "instance") { + cfSet.column.shouldEqual("instance") + cfSet.filter.operatorString.shouldEqual("=~") + cfSet.filter.valuesStrings shouldEqual(Set("Inst-0")) + } else { + fail("invalid entry in column filter sequence " + cfSet) + } + } + res(1).size.shouldEqual(2) + for (cfSet <- res(1)) { + if (cfSet.column == "job") { + cfSet.column.shouldEqual("job") + cfSet.filter.operatorString.shouldEqual("=") + cfSet.filter.valuesStrings shouldEqual(Set("MetricName2")) + } else if (cfSet.column == "instance") { + cfSet.column.shouldEqual("instance") + cfSet.filter.operatorString.shouldEqual("!~") + cfSet.filter.valuesStrings shouldEqual(Set("Inst-1")) + } else { + fail("invalid entry in column filter sequence " + cfSet) + } + } } - it("should get labelValueOps fail for scalar logicalPlan") { + it("should get columnFilterGroup fail for scalar logicalPlan") { val periodicSeriesWithWindowing = ScalarTimeBasedPlan(ScalarFunctionId.Year, RangeParams(1000, 500, 5000)) - val res = LogicalPlan.getLabelValueOperatorsFromLogicalPlan(periodicSeriesWithWindowing) + val res = LogicalPlan.getColumnFilterGroup(periodicSeriesWithWindowing) res.isEmpty should be (true) - intercept[NoSuchElementException] { res.get } } it("should get MetricName fail for scalar logicalPlan") { val periodicSeriesWithWindowing = ScalarTimeBasedPlan(ScalarFunctionId.Year, RangeParams(1000, 500, 5000)) - val res = LogicalPlan.getLabelValueFromLogicalPlan(periodicSeriesWithWindowing, "_name_") + val res = LogicalPlan.getColumnValues(periodicSeriesWithWindowing, "_name_") res.isEmpty should be (true) - intercept[NoSuchElementException] { res.get } } it("should get MetricName from logicalPlan") { val rawSeries = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("_name_", Equals("MetricName")), - ColumnFilter("instance", Equals("Inst-0"))), Seq("job", "instance"), Some(300000), None) + ColumnFilter("instance", Equals("Inst-0"))), Seq("_name_", "instance"), Some(300000), None) val periodicSeriesWithWindowing = PeriodicSeriesWithWindowing(rawSeries, 1000, 500, 5000, 100, SumOverTime) - val res = LogicalPlan.getLabelValueFromLogicalPlan(periodicSeriesWithWindowing, "_name_") - res.get.shouldEqual(Seq("MetricName")) + val res = LogicalPlan.getColumnValues(periodicSeriesWithWindowing, "_name_") + res.shouldEqual(Set("MetricName")) } it("should get LabelName from logicalPlan with filter In") { val rawSeries = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("_name_", Equals("MetricName")), - ColumnFilter("instance", In(Set("Inst-0", "Inst-1")))), Seq("job", "instance"), Some(300000), None) + ColumnFilter("instance", In(Set("Inst-0", "Inst-1")))), Seq("_name_", "instance"), Some(300000), None) val periodicSeriesWithWindowing = PeriodicSeriesWithWindowing(rawSeries, 1000, 500, 5000, 100, SumOverTime) - val res = LogicalPlan.getLabelValueFromLogicalPlan(periodicSeriesWithWindowing, "instance") - res.get.shouldEqual(Seq("Inst-0", "Inst-1")) + val res = LogicalPlan.getColumnValues(periodicSeriesWithWindowing, "instance") + res.shouldEqual(Set("Inst-0", "Inst-1")) } it("should get MetricName from BinaryJoin LogicalPlan") { @@ -119,24 +145,24 @@ class LogicalPlanSpec extends FunSpec with Matchers { val binaryJoin = BinaryJoin(lhs, DIV, OneToOne, rhs) - val res = LogicalPlan.getLabelValueFromLogicalPlan(binaryJoin, "_name_") - res.get.shouldEqual(Seq("MetricName1")) + val res = LogicalPlan.getColumnValues(binaryJoin, "_name_") + res.shouldEqual(Set("MetricName1")) } it("should return None if label value is not present in logicalPlan") { val rawSeries = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("_name_", Equals("MetricName")), - ColumnFilter("instance", Equals("Inst-0"))), Seq("job", "instance"), Some(300000), None) + ColumnFilter("instance", Equals("Inst-0"))), Seq("_name_", "instance"), Some(300000), None) val periodicSeriesWithWindowing = PeriodicSeriesWithWindowing(rawSeries, 1000, 500, 5000, 100, SumOverTime) - val res = LogicalPlan.getLabelValueFromLogicalPlan(periodicSeriesWithWindowing, "_name") + val res = LogicalPlan.getColumnValues(periodicSeriesWithWindowing, "_name") res.isEmpty shouldEqual(true) } it("should concatenate results from lhs and rhs for BinaryJoin LogicalPlan") { val rawSeriesLhs = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("_name_", Equals("MetricName1")), - ColumnFilter("instance", Equals("Inst-0"))), Seq("job", "instance"), Some(300000), None) + ColumnFilter("instance", Equals("Inst-0"))), Seq("_name_", "instance"), Some(300000), None) val lhs = PeriodicSeries(rawSeriesLhs, 1000, 500, 50000) val rawSeriesRhs = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("job", Equals("MetricName2")), @@ -145,8 +171,112 @@ class LogicalPlanSpec extends FunSpec with Matchers { val binaryJoin = BinaryJoin(lhs, DIV, OneToOne, rhs) - val res = LogicalPlan.getLabelValueFromLogicalPlan(binaryJoin, "instance") - res.get.shouldEqual(Seq("Inst-0", "Inst-1")) + val res = LogicalPlan.getColumnValues(binaryJoin, "instance") + res.shouldEqual(Set("Inst-0", "Inst-1")) + } + + it("should sort ColumnFilters when only one group is present") { + + val rawSeries = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("name", Equals("MetricName")), + ColumnFilter("instance", NotEquals("Inst-0"))), Seq("name", "instance"), Some(300000), None) + val periodicSeriesWithWindowing = PeriodicSeriesWithWindowing(rawSeries, 1000, 500, 5000, 100, SumOverTime) + + val res = LogicalPlan.getColumnFilterGroup(periodicSeriesWithWindowing) + res.size.shouldEqual(1) + res(0).size.shouldEqual(2) + for (cfSet <- res(0)) { + if (cfSet.column.equals("name")) { + cfSet.column.shouldEqual("name") + cfSet.filter.operatorString.shouldEqual("=") + cfSet.filter.valuesStrings shouldEqual(Set("MetricName")) + } else if (cfSet.column.equals("instance")) { + cfSet.column.shouldEqual("instance") + cfSet.filter.operatorString.shouldEqual("!=") + cfSet.filter.valuesStrings shouldEqual(Set("Inst-0")) + } else { + fail("invalid entry in column filter sequence " + cfSet) + } + } + } + + it("should get label values from nested binary join and sort") { + val rawSeriesLhs1 = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("app", Equals("Mosaic")), + ColumnFilter("instance", EqualsRegex("Inst-1"))), Seq("name", "instance"), Some(300000), None) + val lhs1 = PeriodicSeries(rawSeriesLhs1, 1000, 500, 50000) + + val rawSeriesLhs2 = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("app", Equals("Cassandra")), + ColumnFilter("instance", EqualsRegex("Inst-0"))), Seq("name", "instance"), Some(300000), None) + val lhs2 = PeriodicSeries(rawSeriesLhs2, 1000, 500, 50000) + + val binaryJoin1 = BinaryJoin(lhs1, DIV, OneToOne, lhs2) + + val rawSeriesRhs = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("app", Equals("Test")), + ColumnFilter("instance", NotEqualsRegex("Inst-1"))), Seq("job", "instance"), Some(300000), None) + val rhs = PeriodicSeries(rawSeriesRhs, 1000, 500, 50000) + + + val binaryJoin2 = BinaryJoin(binaryJoin1, DIV, OneToOne, rhs) + + val res = LogicalPlan.getColumnFilterGroup(binaryJoin2) + res.size shouldEqual(3) + + res(0).size.shouldEqual(2) + for (cfSet <- res(0)) { + if (cfSet.column == "app") { + cfSet.column.shouldEqual("app") + cfSet.filter.operatorString.shouldEqual("=") + cfSet.filter.valuesStrings shouldEqual(Set("Mosaic")) + } else if (cfSet.column == "instance") { + cfSet.column.shouldEqual("instance") + cfSet.filter.operatorString.shouldEqual("=~") + cfSet.filter.valuesStrings shouldEqual(Set("Inst-1")) + } else { + fail("invalid entry in column filter sequence " + cfSet) + } + } + res(1).size.shouldEqual(2) + for (cfSet <- res(1)) { + if (cfSet.column == "app") { + cfSet.column.shouldEqual("app") + cfSet.filter.operatorString.shouldEqual("=") + cfSet.filter.valuesStrings shouldEqual(Set("Cassandra")) + } else if (cfSet.column == "instance") { + cfSet.column.shouldEqual("instance") + cfSet.filter.operatorString.shouldEqual("=~") + cfSet.filter.valuesStrings shouldEqual(Set("Inst-0")) + } else { + fail("invalid entry in column filter sequence " + cfSet) + } + } + res(2).size.shouldEqual(2) + for (cfSet <- res(2)) { + if (cfSet.column == "app") { + cfSet.column.shouldEqual("app") + cfSet.filter.operatorString.shouldEqual("=") + cfSet.filter.valuesStrings shouldEqual(Set("Test")) + } else if (cfSet.column == "instance") { + cfSet.column.shouldEqual("instance") + cfSet.filter.operatorString.shouldEqual("!~") + cfSet.filter.valuesStrings shouldEqual(Set("Inst-1")) + } else { + fail("invalid entry in column filter sequence " + cfSet) + } + } } + it("should have equal hashcode for identical ColumnFilterGroup") { + val rawSeries1 = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("name", Equals("MetricName")), + ColumnFilter("instance", NotEquals("Inst-0"))), Seq("name", "instance"), Some(300000), None) + val periodicSeriesWithWindowing1 = PeriodicSeriesWithWindowing(rawSeries1, 1000, 500, 5000, 100, SumOverTime) + val res1 = LogicalPlan.getColumnFilterGroup(periodicSeriesWithWindowing1) + val rawSeries2 = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("instance", NotEquals("Inst-0")), + ColumnFilter("name", Equals("MetricName"))), Seq("name", "instance"), Some(300000), None) + val periodicSeriesWithWindowing2 = PeriodicSeriesWithWindowing(rawSeries2, 1000, 500, 5000, 100, SumOverTime) + val res2 = LogicalPlan.getColumnFilterGroup(periodicSeriesWithWindowing2) + res1.size.shouldEqual(1) + res1(0).size.shouldEqual(2) + res2.size.shouldEqual(1) + res2(0).size.shouldEqual(2) + res1.hashCode() shouldEqual res2.hashCode() + } } From e108bf37d9bdd24f8625d9d275438d9db2e2912c Mon Sep 17 00:00:00 2001 From: "Brian S. O'Neill" Date: Mon, 22 Jun 2020 07:09:40 -0700 Subject: [PATCH 04/22] feat(memory): Make the headroom latch timeout proportional to free memory (#793) --- .../scala/filodb.memory/BlockManager.scala | 31 ++++++++++++++----- 1 file changed, 23 insertions(+), 8 deletions(-) diff --git a/memory/src/main/scala/filodb.memory/BlockManager.scala b/memory/src/main/scala/filodb.memory/BlockManager.scala index dc1e57e503..2f8edbb5aa 100644 --- a/memory/src/main/scala/filodb.memory/BlockManager.scala +++ b/memory/src/main/scala/filodb.memory/BlockManager.scala @@ -79,6 +79,10 @@ trait BlockManager { ensureFreeBytes((totalMemorySizeInBytes * pct * 0.01).toLong) } + def currentFreePercent: Double = { + (((numFreeBlocks * blockSizeInBytes).toDouble) / totalMemorySizeInBytes) * 100.0 + } + def totalMemorySizeInBytes: Long /** @@ -160,8 +164,6 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, override def blockSizeInBytes: Long = PageManager.getInstance().pageSize() * numPagesPerBlock - def availablePreAllocated: Long = numFreeBlocks * blockSizeInBytes - def usedMemory: Long = usedBlocks.size * blockSizeInBytes override def numFreeBlocks: Int = freeBlocks.size @@ -298,11 +300,11 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, if (acquired) { return true } - timeout <<= 1 - if (timeout > finalTimeoutMillis) { + if (timeout >= finalTimeoutMillis) { return false } Thread.`yield`() + timeout = Math.min(finalTimeoutMillis, timeout << 1) } false // never reached, but scala compiler complains otherwise } @@ -315,12 +317,25 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, * @param pct percentage: 0.0 to 100.0 */ def ensureHeadroom(pct: Double): Int = { + // Ramp up the timeout as the current headroom shrinks. Max timeout per attempt is a little + // over 2 seconds, and the total timeout can be double that, for a total of 4 seconds. + val maxTimeoutMillis = 2048 + val timeoutMillis = ((1.0 - (currentFreePercent / pct)) * maxTimeoutMillis).toInt + + if (timeoutMillis <= 0) { + // Headroom target is already met. + return numFreeBlocks + } + var numFree: Int = 0 val start = System.nanoTime() - // Give up after waiting (in total) a little over 2 seconds. - val acquired = tryExclusiveReclaimLock(1024) + val acquired = tryExclusiveReclaimLock(timeoutMillis) if (!acquired) { - logger.warn(s"Lock for BlockManager.ensureFreePercent timed out: ${reclaimLock}") + if (timeoutMillis >= maxTimeoutMillis / 2) { + // Start warning when the current headroom has dipped below the halfway point. + // The lock state is logged in case it's stuck due to a runaway query somewhere. + logger.warn(s"Lock for BlockManager.ensureHeadroom timed out: ${reclaimLock}") + } numFree = numFreeBlocks } else { try { @@ -329,7 +344,7 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, reclaimLock.releaseExclusive() } val numBytes = numFree * blockSizeInBytes - logger.debug(s"BlockManager.ensureFreePercent numFree: $numFree ($numBytes bytes)") + logger.debug(s"BlockManager.ensureHeadroom numFree: $numFree ($numBytes bytes)") } val stall = System.nanoTime() - start stats.blockHeadroomStall.increment(stall) From 4c7347b32acaf80c078223719f591ab6e39ea97c Mon Sep 17 00:00:00 2001 From: "Brian S. O'Neill" Date: Mon, 22 Jun 2020 17:18:32 -0700 Subject: [PATCH 05/22] fix(memory): Halt the process when locks aren't released properly. (#796) --- memory/src/main/scala/filodb.memory/data/ChunkMap.scala | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/memory/src/main/scala/filodb.memory/data/ChunkMap.scala b/memory/src/main/scala/filodb.memory/data/ChunkMap.scala index 39bb541d9b..b20bac4df3 100644 --- a/memory/src/main/scala/filodb.memory/data/ChunkMap.scala +++ b/memory/src/main/scala/filodb.memory/data/ChunkMap.scala @@ -130,9 +130,15 @@ object ChunkMap extends StrictLogging { if (numLocksReleased > 0) { logger.error(s"Number of locks was non-zero: $numLocksReleased. " + s"This is indicative of a possible lock acquisition/release bug.") + haltAndCatchFire() } execPlanTracker.put(t, execPlan) } + + def haltAndCatchFire(): Unit = { + logger.error(s"Shutting down process since it may be in an unstable/corrupt state.") + Runtime.getRuntime.halt(1) + } } /** @@ -285,8 +291,7 @@ class ChunkMap(val memFactory: MemFactory, var capacity: Int) { val lockState = UnsafeUtils.getIntVolatile(this, lockStateOffset) _logger.error(s"Following execPlan locks have not been released for a while: " + s"$locks2 $locks1 $execPlanTracker $lockState") - _logger.error(s"Shutting down process since it may be in an unstable/corrupt state.") - Runtime.getRuntime.halt(1) + haltAndCatchFire() } } } From 799c8bf0cb92a2763a51cb9aa2c498fcd547a21c Mon Sep 17 00:00:00 2001 From: "Brian S. O'Neill" Date: Wed, 24 Jun 2020 06:59:25 -0700 Subject: [PATCH 06/22] fix(memory): Comment out halt for now. (#797) --- memory/src/main/scala/filodb.memory/data/ChunkMap.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/memory/src/main/scala/filodb.memory/data/ChunkMap.scala b/memory/src/main/scala/filodb.memory/data/ChunkMap.scala index b20bac4df3..dc086f3a27 100644 --- a/memory/src/main/scala/filodb.memory/data/ChunkMap.scala +++ b/memory/src/main/scala/filodb.memory/data/ChunkMap.scala @@ -130,7 +130,8 @@ object ChunkMap extends StrictLogging { if (numLocksReleased > 0) { logger.error(s"Number of locks was non-zero: $numLocksReleased. " + s"This is indicative of a possible lock acquisition/release bug.") - haltAndCatchFire() + // FIXME: Causes failures when running the unit tests for some unknown reason. + //haltAndCatchFire() } execPlanTracker.put(t, execPlan) } From b9be6db634de920af55e3152c89358de21c7cf24 Mon Sep 17 00:00:00 2001 From: TanviBhavsar Date: Wed, 24 Jun 2020 11:11:26 -0700 Subject: [PATCH 07/22] feat(query, coordinator): support regex in shard keys (#795) --- cli/src/main/scala/filodb.cli/CliMain.scala | 6 +- .../queryplanner/LogicalPlanUtils.scala | 1 + .../queryplanner/MultiPartitionPlanner.scala | 4 +- .../queryplanner/ShardKeyRegexPlanner.scala | 66 ++++++++++ .../queryplanner/SingleClusterPlanner.scala | 15 +-- .../ShardKeyRegexPlannerSpec.scala | 120 ++++++++++++++++++ .../SinglePartitionPlannerSpec.scala | 2 +- .../src/test/scala/filodb.core/TestData.scala | 7 + .../main/scala/filodb/query/LogicalPlan.scala | 88 +++++++++++-- .../scala/filodb/query/LogicalPlanSpec.scala | 14 ++ 10 files changed, 301 insertions(+), 22 deletions(-) create mode 100644 coordinator/src/main/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlanner.scala create mode 100644 coordinator/src/test/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlannerSpec.scala diff --git a/cli/src/main/scala/filodb.cli/CliMain.scala b/cli/src/main/scala/filodb.cli/CliMain.scala index f979073735..2a8fa9745b 100644 --- a/cli/src/main/scala/filodb.cli/CliMain.scala +++ b/cli/src/main/scala/filodb.cli/CliMain.scala @@ -18,7 +18,7 @@ import filodb.coordinator.client.QueryCommands.StaticSpreadProvider import filodb.core._ import filodb.core.binaryrecord2.RecordBuilder import filodb.core.metadata.{Column, Schemas} -import filodb.core.query.{PromQlQueryParams, QueryContext, TsdbQueryParams, UnavailablePromQlQueryParams} +import filodb.core.query._ import filodb.core.store.ChunkSetInfoOnHeap import filodb.memory.MemFactory import filodb.memory.format.{BinaryVector, Classes, MemoryReader, RowReader} @@ -265,7 +265,9 @@ object CliMain extends ArgMain[Arguments] with FilodbClusterNode { def parseLabelValuesQuery(client: LocalClient, labelNames: Seq[String], constraints: Map[String, String], dataset: String, timeParams: TimeRangeParams, options: QOptions): Unit = { - val logicalPlan = LabelValues(labelNames, constraints, timeParams.start * 1000, timeParams.end * 1000) + // TODO support all filters + val filters = constraints.map { case (k, v) => ColumnFilter(k, Filter.Equals(v)) }.toSeq + val logicalPlan = LabelValues(labelNames, filters, timeParams.start * 1000, timeParams.end * 1000) executeQuery2(client, dataset, logicalPlan, options, UnavailablePromQlQueryParams) } diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala index 7be1d4a0f2..4e9319b207 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala @@ -157,4 +157,5 @@ object LogicalPlanUtils { } else { labels } + } diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala index b25ae286a8..5fd8974a29 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala @@ -169,8 +169,8 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider localPartitionPlanner.materialize(lp.copy(startMs = p.timeRange.startMs, endMs = p.timeRange.endMs), qContext) else createMetadataRemoteExec(qContext, queryParams, p, - Map("filter" -> lp.labelConstraints.map{case (k, v) => k + "=" + v}.mkString(","), - "labels" -> lp.labelNames.mkString(","))) + Map("filter" -> lp.filters.map{f => f.column + f.filter.operatorString + f.filter.valuesStrings.head}. + mkString(","), "labels" -> lp.labelNames.mkString(","))) } if (execPlans.size == 1) execPlans.head else LabelValuesDistConcatExec(qContext, InProcessPlanDispatcher, diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlanner.scala new file mode 100644 index 0000000000..790bfaba24 --- /dev/null +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlanner.scala @@ -0,0 +1,66 @@ +package filodb.coordinator.queryplanner + +import filodb.core.metadata.Dataset +import filodb.core.query.{ColumnFilter, QueryContext} +import filodb.core.query.Filter.{EqualsRegex, NotEqualsRegex} +import filodb.query.{Aggregate, BinaryJoin, LogicalPlan} +import filodb.query.exec.{DistConcatExec, ExecPlan, InProcessPlanDispatcher, ReduceAggregateExec} + +/** + * Responsible for query planning for queries having regex in shard column + * + * @param dataset dataset + * @param queryPlanner multiPartition query planner + * @param shardKeyMatcher used to get values for regex shard keys. Each inner sequence corresponds to matching regex + * value. For example: Seq(ColumnFilter(ws, Equals(demo)), ColumnFilter(ns, EqualsRegex(App*)) returns + * Seq(Seq(ColumnFilter(ws, Equals(demo)), ColumnFilter(ns, Equals(App1))), Seq(ColumnFilter(ws, Equals(demo)), + * ColumnFilter(ns, Equals(App2)) + */ + +class ShardKeyRegexPlanner(dataset: Dataset, + queryPlanner: QueryPlanner, + shardKeyMatcher: Seq[ColumnFilter] => Seq[Seq[ColumnFilter]]) extends QueryPlanner { + /** + * Converts a logical plan to execution plan. + * + * @param logicalPlan Logical plan after converting PromQL -> AST -> LogicalPlan + * @param qContext holder for additional query parameters + * @return materialized Execution Plan which can be dispatched + */ + override def materialize(logicalPlan: LogicalPlan, qContext: QueryContext): ExecPlan = { + logicalPlan match { + case a: Aggregate => materializeAggregate(a, qContext) + case b: BinaryJoin => materializeBinaryJoin(b, qContext) + case _ => materializeOthers(logicalPlan, qContext) + } + } + + private def getNonMetricShardKeyFilters(logicalPlan: LogicalPlan) = LogicalPlan. + getRawSeriesFilters(logicalPlan).map { s => s.filter(f => dataset.options.nonMetricShardColumns.contains(f.column))} + + private def generateExec(logicalPlan: LogicalPlan, nonMetricShardKeyFilters: Seq[ColumnFilter], + qContext: QueryContext) = shardKeyMatcher(nonMetricShardKeyFilters).map(f => + queryPlanner.materialize(logicalPlan.replaceFilters(f), qContext)) + + private def materializeBinaryJoin(binaryJoin: BinaryJoin, qContext: QueryContext): ExecPlan = { + if (getNonMetricShardKeyFilters(binaryJoin).forall(_.forall(f => !f.filter.isInstanceOf[EqualsRegex] && + !f.filter.isInstanceOf[NotEqualsRegex]))) queryPlanner.materialize(binaryJoin, qContext) + else throw new UnsupportedOperationException("Regex not supported for Binary Join") + } + + private def materializeAggregate(aggregate: Aggregate, queryContext: QueryContext): ExecPlan = { + val execPlans = generateExec(aggregate, getNonMetricShardKeyFilters(aggregate).head, queryContext) + if (execPlans.size == 1) execPlans.head + else ReduceAggregateExec(queryContext, InProcessPlanDispatcher, execPlans, aggregate.operator, aggregate.params) + } + + private def materializeOthers(logicalPlan: LogicalPlan, queryContext: QueryContext): ExecPlan = { + val nonMetricShardKeyFilters = getNonMetricShardKeyFilters(logicalPlan) + // For queries which don't have RawSeries filters like metadata and fixed scalar queries + if (nonMetricShardKeyFilters.head.isEmpty) queryPlanner.materialize(logicalPlan, queryContext) + else { + val execPlans = generateExec(logicalPlan, nonMetricShardKeyFilters.head, queryContext) + if (execPlans.size == 1) execPlans.head else DistConcatExec(queryContext, InProcessPlanDispatcher, execPlans) + } + } +} diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala index 652ec6f429..977903b20c 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala @@ -316,7 +316,7 @@ class SingleClusterPlanner(dsRef: DatasetRef, * period, returns it as is. */ private def boundToStartTimeToEarliestRetained(startMs: Long, stepMs: Long, - windowMs: Long, offsetMs: Long): Long = { + windowMs: Long, offsetMs: Long): Long = { // In case query is earlier than earliestRetainedTimestamp then we need to drop the first few instants // to prevent inaccurate results being served. Inaccuracy creeps in because data can be in memory for which // equivalent data may not be in cassandra. Aggregations cannot be guaranteed to be complete. @@ -378,22 +378,21 @@ class SingleClusterPlanner(dsRef: DatasetRef, private def materializeLabelValues(qContext: QueryContext, lp: LabelValues): PlanResult = { - val filters = lp.labelConstraints.map { case (k, v) => ColumnFilter(k, Filter.Equals(v)) }.toSeq // If the label is PromMetricLabel and is different than dataset's metric name, // replace it with dataset's metric name. (needed for prometheus plugins) val metricLabelIndex = lp.labelNames.indexOf(PromMetricLabel) val labelNames = if (metricLabelIndex > -1 && dsOptions.metricColumn != PromMetricLabel) lp.labelNames.updated(metricLabelIndex, dsOptions.metricColumn) else lp.labelNames - val shardsToHit = if (shardColumns.toSet.subsetOf(lp.labelConstraints.keySet)) { - shardsFromFilters(filters, qContext) + val shardsToHit = if (shardColumns.toSet.subsetOf(lp.filters.map(_.column).toSet)) { + shardsFromFilters(lp.filters, qContext) } else { mdNoShardKeyFilterRequests.increment() shardMapperFunc.assignedShards } val metaExec = shardsToHit.map { shard => val dispatcher = dispatcherForShard(shard) - exec.LabelValuesExec(qContext, dispatcher, dsRef, shard, filters, labelNames, lp.startMs, lp.endMs) + exec.LabelValuesExec(qContext, dispatcher, dsRef, shard, lp.filters, labelNames, lp.startMs, lp.endMs) } PlanResult(metaExec, false) } @@ -442,7 +441,7 @@ class SingleClusterPlanner(dsRef: DatasetRef, private def materializeFunctionArgs(functionParams: Seq[FunctionArgsPlan], qContext: QueryContext): Seq[FuncArgs] = { - if (functionParams.isEmpty){ + if (functionParams.isEmpty) { Nil } else { functionParams.map { param => @@ -467,7 +466,7 @@ class SingleClusterPlanner(dsRef: DatasetRef, PlanResult(Seq(topPlan), vectors.needsStitch) } else { vectors.plans.foreach(_.addRangeVectorTransformer(ScalarFunctionMapper(lp.function, - RangeParams(lp.startMs, lp.stepMs, lp.endMs)))) + RangeParams(lp.startMs, lp.stepMs, lp.endMs)))) vectors } } @@ -475,7 +474,7 @@ class SingleClusterPlanner(dsRef: DatasetRef, private def materializeApplySortFunction(qContext: QueryContext, lp: ApplySortFunction): PlanResult = { val vectors = walkLogicalPlanTree(lp.vectors, qContext) - if(vectors.plans.length > 1) { + if (vectors.plans.length > 1) { val targetActor = pickDispatcher(vectors.plans) val topPlan = DistConcatExec(qContext, targetActor, vectors.plans) topPlan.addRangeVectorTransformer(SortFunctionMapper(lp.function)) diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlannerSpec.scala new file mode 100644 index 0000000000..725e53900c --- /dev/null +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlannerSpec.scala @@ -0,0 +1,120 @@ +package filodb.coordinator.queryplanner + +import akka.actor.ActorSystem +import akka.testkit.TestProbe +import com.typesafe.config.ConfigFactory +import org.scalatest.{FunSpec, Matchers} +import org.scalatest.concurrent.ScalaFutures + +import filodb.coordinator.ShardMapper +import filodb.core.MetricsTestData +import filodb.core.metadata.Schemas +import filodb.core.query.{ColumnFilter, PromQlQueryParams, QueryConfig, QueryContext} +import filodb.core.query.Filter.Equals +import filodb.prometheus.ast.TimeStepParams +import filodb.prometheus.parse.Parser +import filodb.query.exec._ + +class ShardKeyRegexPlannerSpec extends FunSpec with Matchers with ScalaFutures { + + private val dataset = MetricsTestData.timeseriesDatasetMultipleShardKeys + private val dsRef = dataset.ref + private val schemas = Schemas(dataset.schema) + private implicit val system = ActorSystem() + private val node = TestProbe().ref + + private val routingConfigString = "routing {\n buddy {\n http {\n timeout = 10.seconds\n }\n }\n}" + private val routingConfig = ConfigFactory.parseString(routingConfigString) + private val config = ConfigFactory.load("application_test.conf").getConfig("filodb.query"). + withFallback(routingConfig) + private val queryConfig = new QueryConfig(config) + + private val promQlQueryParams = PromQlQueryParams("sum(heap_usage)", 100, 1, 1000, None) + + private val localMapper = new ShardMapper(32) + for {i <- 0 until 32} localMapper.registerNode(Seq(i), node) + + + val localPlanner = new SingleClusterPlanner(dsRef, schemas, localMapper, earliestRetainedTimestampFn = 0, queryConfig) + + it("should generate Exec plan for simple query") { + val lp = Parser.queryToLogicalPlan("test{_ws_ = \"demo\", _ns_ =~ \"app.*\", instance = \"Inst-1\" }", 1000) + val shardKeyMatcher = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), + ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), + ColumnFilter("_ns_", Equals("App-2"))))} + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcher) + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + execPlan.isInstanceOf[DistConcatExec] shouldEqual(true) + execPlan.children(0).children.head.isInstanceOf[MultiSchemaPartitionsExec] + execPlan.children(0).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", Equals("App-1"))) shouldEqual(true) + execPlan.children(1).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", Equals("App-2"))) shouldEqual(true) + } + + it("should generate Exec plan for Aggregate query") { + val lp = Parser.queryToLogicalPlan("sum(test{_ws_ = \"demo\", _ns_ =~ \"app.*\", instance = \"Inst-1\" })", 1000) + val shardKeyMatcher = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), + ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), + ColumnFilter("_ns_", Equals("App-2"))))} + val engine = new ShardKeyRegexPlanner( dataset, localPlanner, shardKeyMatcher) + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + execPlan.isInstanceOf[ReduceAggregateExec] shouldEqual(true) + execPlan.children(0).children.head.isInstanceOf[MultiSchemaPartitionsExec] + execPlan.children(0).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", Equals("App-1"))) shouldEqual(true) + execPlan.children(1).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", Equals("App-2"))) shouldEqual(true) + } + + it("should generate Exec plan for time()") { + val lp = Parser.queryToLogicalPlan("time()", 1000) + val shardKeyMatcher = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), + ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), + ColumnFilter("_ns_", Equals("App-2"))))} + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcher) + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + execPlan.isInstanceOf[TimeScalarGeneratorExec] shouldEqual(true) + } + + it("should generate Exec plan for Scalar Binary Operation") { + val lp = Parser.queryToLogicalPlan("1 + test{_ws_ = \"demo\", _ns_ =~ \"app.*\", instance = \"Inst-1\" }", 1000) + val shardKeyMatcher = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), + ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), + ColumnFilter("_ns_", Equals("App-2"))))} + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcher) + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + execPlan.isInstanceOf[DistConcatExec] shouldEqual(true) + execPlan.children(0).children.head.isInstanceOf[MultiSchemaPartitionsExec] + execPlan.children(0).children.head.rangeVectorTransformers(0).isInstanceOf[PeriodicSamplesMapper] shouldEqual true + execPlan.children(0).children.head.rangeVectorTransformers(1).isInstanceOf[ScalarOperationMapper] shouldEqual true + execPlan.children(0).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", Equals("App-1"))) shouldEqual(true) + execPlan.children(1).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", Equals("App-2"))) shouldEqual(true) + } + + it("should generate Exec plan for Binary join without regex") { + val lp = Parser.queryToLogicalPlan("test1{_ws_ = \"demo\", _ns_ = \"app\"} + " + + "test2{_ws_ = \"demo\", _ns_ = \"app\"}", 1000) + val shardKeyMatcher = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), + ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), + ColumnFilter("_ns_", Equals("App-2"))))} + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcher) + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + execPlan.isInstanceOf[BinaryJoinExec] shouldEqual(true) + } + + it ("should generate Exec plan for Metadata query") { + val lp = Parser.metadataQueryToLogicalPlan("http_requests_total{job=\"prometheus\", method=\"GET\"}", + TimeStepParams(1000, 1000, 3000)) + + val shardKeyMatcher = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), + ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), + ColumnFilter("_ns_", Equals("App-2"))))} + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcher) + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + execPlan.isInstanceOf[PartKeysDistConcatExec] shouldEqual (true) + } +} + diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/SinglePartitionPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/SinglePartitionPlannerSpec.scala index f643ce41b6..6ae22ba901 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/SinglePartitionPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/SinglePartitionPlannerSpec.scala @@ -16,7 +16,7 @@ import filodb.prometheus.parse.Parser import filodb.query._ import filodb.query.exec._ -class SinglePartitionPlannerSpec extends FunSpec with Matchers{ +class SinglePartitionPlannerSpec extends FunSpec with Matchers { private implicit val system = ActorSystem() private val node = TestProbe().ref diff --git a/core/src/test/scala/filodb.core/TestData.scala b/core/src/test/scala/filodb.core/TestData.scala index 88e5e373c6..1cd3e0d5a1 100644 --- a/core/src/test/scala/filodb.core/TestData.scala +++ b/core/src/test/scala/filodb.core/TestData.scala @@ -479,6 +479,13 @@ object MetricsTestData { None, DatasetOptions(Seq("_metric_", "_ns_"), "_metric_")).get + val timeseriesDatasetMultipleShardKeys = Dataset.make("timeseries", + Seq("tags:map"), + Seq("timestamp:ts", "value:double:detectDrops=true"), + Seq.empty, + None, + DatasetOptions(Seq("_metric_", "_ws_", "_ns_"), "_metric_")).get + val downsampleDataset = Dataset.make("tsdbdata", Seq("tags:map"), Seq("timestamp:ts", "min:double", "max:double", "sum:double", "count:double", "avg:double"), diff --git a/query/src/main/scala/filodb/query/LogicalPlan.scala b/query/src/main/scala/filodb/query/LogicalPlan.scala index 6f29de9536..75223c7452 100644 --- a/query/src/main/scala/filodb/query/LogicalPlan.scala +++ b/query/src/main/scala/filodb/query/LogicalPlan.scala @@ -1,6 +1,6 @@ package filodb.query -import filodb.core.query.{ColumnFilter, Filter, RangeParams} +import filodb.core.query.{ColumnFilter, RangeParams} //scalastyle:off number.of.types sealed trait LogicalPlan { @@ -10,6 +10,18 @@ sealed trait LogicalPlan { * It is false for RawSeriesLikePlan, MetadataQueryPlan, RawChunkMeta, ScalarTimeBasedPlan and ScalarFixedDoublePlan */ def isRoutable: Boolean = true + + /** + * Replace filters present in logical plan + */ + def replaceFilters(filters: Seq[ColumnFilter]): LogicalPlan = { + this match { + case p: PeriodicSeriesPlan => p.replacePeriodicSeriesFilters(filters) + case r: RawSeriesLikePlan => r.replaceRawSeriesFilters(filters) + case l: LabelValues => l.copy(filters = filters) + case s: SeriesKeysByFilters => s.copy(filters = filters) + } + } } /** @@ -20,6 +32,7 @@ sealed trait LogicalPlan { sealed trait RawSeriesLikePlan extends LogicalPlan { override def isRoutable: Boolean = false def isRaw: Boolean = false + def replaceRawSeriesFilters(newFilters: Seq[ColumnFilter]): RawSeriesLikePlan } sealed trait NonLeafLogicalPlan extends LogicalPlan { @@ -44,6 +57,8 @@ sealed trait PeriodicSeriesPlan extends LogicalPlan { * Periodic Query step time in millis */ def endMs: Long + + def replacePeriodicSeriesFilters(filters: Seq[ColumnFilter]): PeriodicSeriesPlan } sealed trait MetadataQueryPlan extends LogicalPlan { @@ -73,10 +88,16 @@ case class RawSeries(rangeSelector: RangeSelector, lookbackMs: Option[Long] = None, offsetMs: Option[Long] = None) extends RawSeriesLikePlan { override def isRaw: Boolean = true + + override def replaceRawSeriesFilters(newFilters: Seq[ColumnFilter]): RawSeriesLikePlan = { + val filterColumns = newFilters.map(_.column) + val updatedFilters = this.filters.filterNot(f => filterColumns.contains(f.column)) ++ newFilters + this.copy(filters = updatedFilters) + } } case class LabelValues(labelNames: Seq[String], - labelConstraints: Map[String, String], + filters: Seq[ColumnFilter], startMs: Long, endMs: Long) extends MetadataQueryPlan @@ -98,6 +119,12 @@ case class RawChunkMeta(rangeSelector: RangeSelector, override def startMs: Long = ??? override def stepMs: Long = ??? override def endMs: Long = ??? + + override def replacePeriodicSeriesFilters(newFilters: Seq[ColumnFilter]): PeriodicSeriesPlan = { + val filterColumns = newFilters.map(_.column) + val updatedFilters = this.filters.filterNot(f => filterColumns.contains(f.column)) ++ newFilters + this.copy(filters = updatedFilters) + } } /** @@ -117,6 +144,9 @@ case class PeriodicSeries(rawSeries: RawSeriesLikePlan, endMs: Long, offsetMs: Option[Long] = None) extends PeriodicSeriesPlan with NonLeafLogicalPlan { override def children: Seq[LogicalPlan] = Seq(rawSeries) + + override def replacePeriodicSeriesFilters(filters: Seq[ColumnFilter]): PeriodicSeriesPlan = this.copy(rawSeries = + rawSeries.replaceRawSeriesFilters(filters)) } /** @@ -136,6 +166,9 @@ case class PeriodicSeriesWithWindowing(series: RawSeriesLikePlan, offsetMs: Option[Long] = None) extends PeriodicSeriesPlan with NonLeafLogicalPlan { override def children: Seq[LogicalPlan] = Seq(series) + + override def replacePeriodicSeriesFilters(filters: Seq[ColumnFilter]): PeriodicSeriesPlan = this.copy(series = + series.replaceRawSeriesFilters(filters)) } /** @@ -154,6 +187,8 @@ case class Aggregate(operator: AggregationOperator, override def startMs: Long = vectors.startMs override def stepMs: Long = vectors.stepMs override def endMs: Long = vectors.endMs + override def replacePeriodicSeriesFilters(filters: Seq[ColumnFilter]): PeriodicSeriesPlan = this.copy(vectors = + vectors.replacePeriodicSeriesFilters(filters)) } /** @@ -181,6 +216,8 @@ case class BinaryJoin(lhs: PeriodicSeriesPlan, override def stepMs: Long = lhs.stepMs override def endMs: Long = lhs.endMs override def isRoutable: Boolean = lhs.isRoutable && rhs.isRoutable + override def replacePeriodicSeriesFilters(filters: Seq[ColumnFilter]): PeriodicSeriesPlan = this.copy(lhs = + lhs.replacePeriodicSeriesFilters(filters), rhs = rhs.replacePeriodicSeriesFilters(filters)) } /** @@ -195,6 +232,8 @@ case class ScalarVectorBinaryOperation(operator: BinaryOperator, override def stepMs: Long = vector.stepMs override def endMs: Long = vector.endMs override def isRoutable: Boolean = vector.isRoutable + override def replacePeriodicSeriesFilters(filters: Seq[ColumnFilter]): PeriodicSeriesPlan = this.copy(vector = + vector.replacePeriodicSeriesFilters(filters)) } /** @@ -210,6 +249,8 @@ case class ApplyInstantFunction(vectors: PeriodicSeriesPlan, override def stepMs: Long = vectors.stepMs override def endMs: Long = vectors.endMs override def isRoutable: Boolean = vectors.isRoutable + override def replacePeriodicSeriesFilters(filters: Seq[ColumnFilter]): PeriodicSeriesPlan = this.copy(vectors = + vectors.replacePeriodicSeriesFilters(filters)) } /** @@ -221,6 +262,8 @@ case class ApplyInstantFunctionRaw(vectors: RawSeries, functionArgs: Seq[FunctionArgsPlan] = Nil) extends RawSeriesLikePlan with NonLeafLogicalPlan { override def children: Seq[LogicalPlan] = Seq(vectors) + override def replaceRawSeriesFilters(newFilters: Seq[ColumnFilter]): RawSeriesLikePlan = this.copy(vectors = + vectors.replaceRawSeriesFilters(newFilters).asInstanceOf[RawSeries]) } /** @@ -234,6 +277,8 @@ case class ApplyMiscellaneousFunction(vectors: PeriodicSeriesPlan, override def startMs: Long = vectors.startMs override def stepMs: Long = vectors.stepMs override def endMs: Long = vectors.endMs + override def replacePeriodicSeriesFilters(filters: Seq[ColumnFilter]): PeriodicSeriesPlan = this.copy(vectors = + vectors.replacePeriodicSeriesFilters(filters)) } /** @@ -245,19 +290,21 @@ case class ApplySortFunction(vectors: PeriodicSeriesPlan, override def startMs: Long = vectors.startMs override def stepMs: Long = vectors.stepMs override def endMs: Long = vectors.endMs + override def replacePeriodicSeriesFilters(filters: Seq[ColumnFilter]): PeriodicSeriesPlan = this.copy(vectors = + vectors.replacePeriodicSeriesFilters(filters)) } /** * Nested logical plan for argument of function * Example: clamp_max(node_info{job = "app"},scalar(http_requests_total{job = "app"})) */ -trait FunctionArgsPlan extends LogicalPlan +trait FunctionArgsPlan extends LogicalPlan with PeriodicSeriesPlan /** * Generate scalar * Example: scalar(http_requests_total), time(), hour() */ -trait ScalarPlan extends LogicalPlan with PeriodicSeriesPlan with FunctionArgsPlan +trait ScalarPlan extends FunctionArgsPlan /** * Generate scalar from vector @@ -272,6 +319,8 @@ final case class ScalarVaryingDoublePlan(vectors: PeriodicSeriesPlan, override def stepMs: Long = vectors.stepMs override def endMs: Long = vectors.endMs override def isRoutable: Boolean = vectors.isRoutable + override def replacePeriodicSeriesFilters(filters: Seq[ColumnFilter]): PeriodicSeriesPlan = this.copy(vectors = + vectors.replacePeriodicSeriesFilters(filters)) } /** @@ -283,6 +332,7 @@ final case class ScalarTimeBasedPlan(function: ScalarFunctionId, rangeParams: Ra override def startMs: Long = rangeParams.startSecs * 1000 override def stepMs: Long = rangeParams.stepSecs * 1000 override def endMs: Long = rangeParams.endSecs * 1000 + override def replacePeriodicSeriesFilters(filters: Seq[ColumnFilter]): PeriodicSeriesPlan = this // No Filter } /** @@ -296,6 +346,7 @@ final case class ScalarFixedDoublePlan(scalar: Double, override def startMs: Long = timeStepParams.startSecs * 1000 override def stepMs: Long = timeStepParams.stepSecs * 1000 override def endMs: Long = timeStepParams.endSecs * 1000 + override def replacePeriodicSeriesFilters(filters: Seq[ColumnFilter]): PeriodicSeriesPlan = this } //scalastyle:off number.of.types @@ -309,6 +360,8 @@ final case class VectorPlan(scalars: ScalarPlan) extends PeriodicSeriesPlan with override def stepMs: Long = scalars.stepMs override def endMs: Long = scalars.endMs override def isRoutable: Boolean = scalars.isRoutable + override def replacePeriodicSeriesFilters(filters: Seq[ColumnFilter]): PeriodicSeriesPlan = this.copy(scalars = + scalars.replacePeriodicSeriesFilters(filters).asInstanceOf[ScalarPlan]) } /** @@ -322,6 +375,13 @@ case class ScalarBinaryOperation(operator: BinaryOperator, override def stepMs: Long = rangeParams.stepSecs * 1000 override def endMs: Long = rangeParams.endSecs * 1000 override def isRoutable: Boolean = false + override def replacePeriodicSeriesFilters(filters: Seq[ColumnFilter]): PeriodicSeriesPlan = { + val updatedLhs = if (lhs.isRight) Right(lhs.right.get.replacePeriodicSeriesFilters(filters). + asInstanceOf[ScalarBinaryOperation]) else Left(lhs.left.get) + val updatedRhs = if (lhs.isRight) Right(rhs.right.get.replacePeriodicSeriesFilters(filters). + asInstanceOf[ScalarBinaryOperation]) else Left(rhs.left.get) + this.copy(lhs = updatedLhs, rhs = updatedRhs) + } } /** @@ -335,6 +395,8 @@ case class ApplyAbsentFunction(vectors: PeriodicSeriesPlan, override def startMs: Long = vectors.startMs override def stepMs: Long = vectors.stepMs override def endMs: Long = vectors.endMs + override def replacePeriodicSeriesFilters(filters: Seq[ColumnFilter]): PeriodicSeriesPlan = this.copy(vectors = + vectors.replacePeriodicSeriesFilters(filters)) } object LogicalPlan { @@ -356,15 +418,15 @@ object LogicalPlan { def getColumnValues(columnFilterGroup: Seq[Set[ColumnFilter]], labelName: String): Set[String] = { columnFilterGroup.flatMap (columnFilters => getColumnValues(columnFilters, labelName)) match { - case columnValues: Iterable[String] => if (columnValues.isEmpty) Set.empty else columnValues.toSet - case _ => Set.empty + case columnValues: Iterable[String] => if (columnValues.isEmpty) Set.empty else columnValues.toSet + case _ => Set.empty } } def getColumnValues(columnFilters: Set[ColumnFilter], labelName: String): Set[String] = { columnFilters.flatMap(cFilter => { cFilter.column == labelName match { - case true => cFilter.filter.valuesStrings.map(_.toString) + case true => cFilter.filter.valuesStrings.map(_.toString) case false => Seq.empty } }) @@ -373,8 +435,7 @@ object LogicalPlan { def getColumnFilterGroup(logicalPlan: LogicalPlan): Seq[Set[ColumnFilter]] = { LogicalPlan.findLeafLogicalPlans(logicalPlan) map { lp => lp match { - case lp: LabelValues => - lp.labelConstraints.map(lbc => ColumnFilter(lbc._1, Filter.Equals(lbc._2))) toSet + case lp: LabelValues => lp.filters toSet case lp: RawSeries => lp.filters toSet case lp: RawChunkMeta => lp.filters toSet case lp: SeriesKeysByFilters => lp.filters toSet @@ -390,5 +451,14 @@ object LogicalPlan { } } + def getRawSeriesFilters(logicalPlan: LogicalPlan): Seq[Seq[ColumnFilter]] = { + LogicalPlan.findLeafLogicalPlans(logicalPlan).map { l => + l match + { + case lp: RawSeries => lp.filters + case _ => Seq.empty + } + } + } } //scalastyle:on number.of.types \ No newline at end of file diff --git a/query/src/test/scala/filodb/query/LogicalPlanSpec.scala b/query/src/test/scala/filodb/query/LogicalPlanSpec.scala index 60eebfa6b0..e978852631 100644 --- a/query/src/test/scala/filodb/query/LogicalPlanSpec.scala +++ b/query/src/test/scala/filodb/query/LogicalPlanSpec.scala @@ -5,6 +5,7 @@ import filodb.core.query.Filter.{Equals, EqualsRegex, In, NotEquals, NotEqualsRe import filodb.query.BinaryOperator.DIV import filodb.query.Cardinality.OneToOne import filodb.query.RangeFunctionId.SumOverTime + import org.scalatest.{FunSpec, Matchers} class LogicalPlanSpec extends FunSpec with Matchers { @@ -264,6 +265,19 @@ class LogicalPlanSpec extends FunSpec with Matchers { } } + it("should update logicalPlan filter") { + val currFilter = ColumnFilter("instance", EqualsRegex("Inst*")) + val rawSeries = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("_name_", Equals("MetricName")), + currFilter, ColumnFilter("job", Equals("job1"))), Seq("_name_", "instance"), Some(300000), None) + val periodicSeriesWithWindowing = PeriodicSeriesWithWindowing(rawSeries, 1000, 500, 5000, 100, SumOverTime) + val updatedFilter = ColumnFilter("instance", Equals("Inst1")) + val res = periodicSeriesWithWindowing.replaceFilters(Seq(updatedFilter)) + res.asInstanceOf[PeriodicSeriesWithWindowing].series.asInstanceOf[RawSeries].filters. + contains(updatedFilter) shouldEqual(true) + res.asInstanceOf[PeriodicSeriesWithWindowing].series.asInstanceOf[RawSeries].filters. + contains(currFilter) shouldEqual(false) + } + it("should have equal hashcode for identical ColumnFilterGroup") { val rawSeries1 = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("name", Equals("MetricName")), ColumnFilter("instance", NotEquals("Inst-0"))), Seq("name", "instance"), Some(300000), None) From 68492ce523af4c3defe2a29c0b48f2408f134d8d Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Wed, 24 Jun 2020 13:12:40 -0700 Subject: [PATCH 08/22] bug(query): Aggregation was short-circuiting iteration (#798) --- .../scala/filodb/query/exec/AggrOverRangeVectors.scala | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala b/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala index 1ec4cd38fe..2339ab6148 100644 --- a/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala +++ b/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala @@ -158,7 +158,15 @@ object RangeVectorAggregator extends StrictLogging { rvs.groupBy(grouping).mapValues { rvs => new Iterator[rowAgg.AggHolderType] { val itsAndKeys = rvs.map { rv => (rv.rows, rv.key) } - def hasNext: Boolean = itsAndKeys.forall(_._1.hasNext) + def hasNext: Boolean = { + // Dont use forAll since it short-circuits hasNext invocation + // It is important to invoke hasNext on all iterators to release shared locks + var hnRet = false + itsAndKeys.foreach { itKey => + if (itKey._1.hasNext) hnRet = true + } + hnRet + } def next(): rowAgg.AggHolderType = { acc.resetToZero() itsAndKeys.foreach { case (rowIter, rvk) => From e70e2c715d1dfc6fe2e00438a2f1cdd4692287bf Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Thu, 25 Jun 2020 10:12:12 -0700 Subject: [PATCH 09/22] fix(jmh): A few more fixes to QueryInMemoryBenchmark (#792) --- .../filodb.jmh/QueryInMemoryBenchmark.scala | 34 ++++++++++++------- .../filodb/standalone/SimpleProfiler.java | 5 +++ 2 files changed, 26 insertions(+), 13 deletions(-) diff --git a/jmh/src/main/scala/filodb.jmh/QueryInMemoryBenchmark.scala b/jmh/src/main/scala/filodb.jmh/QueryInMemoryBenchmark.scala index 2018037351..e11a4fbbd5 100644 --- a/jmh/src/main/scala/filodb.jmh/QueryInMemoryBenchmark.scala +++ b/jmh/src/main/scala/filodb.jmh/QueryInMemoryBenchmark.scala @@ -9,6 +9,7 @@ import akka.actor.ActorSystem import ch.qos.logback.classic.{Level, Logger} import com.typesafe.config.ConfigFactory import com.typesafe.scalalogging.StrictLogging +import kamon.Kamon import monix.eval.Task import monix.execution.Scheduler import monix.reactive.Observable @@ -26,6 +27,13 @@ import filodb.prometheus.parse.Parser import filodb.query.{QueryError => QError, QueryResult => QueryResult2} import filodb.timeseries.TestTimeseriesProducer +object Params { + final val numShards = 32 + final val numSamples = 720 // 2 hours * 3600 / 10 sec interval + final val numSeries = 100 + final val numQueries = 100 +} + //scalastyle:off regex /** * A macrobenchmark (IT-test level) for QueryEngine2 aggregations, in-memory only (no on-demand paging) @@ -35,21 +43,19 @@ import filodb.timeseries.TestTimeseriesProducer */ @State(Scope.Thread) class QueryInMemoryBenchmark extends StrictLogging { - org.slf4j.LoggerFactory.getLogger("filodb").asInstanceOf[Logger].setLevel(Level.WARN) + Kamon.init() // Needed for metrics logging + org.slf4j.LoggerFactory.getLogger("filodb").asInstanceOf[Logger].setLevel(Level.INFO) import filodb.coordinator._ import client.Client.{actorAsk, asyncAsk} import client.QueryCommands._ import NodeClusterActor._ + import Params._ import filodb.standalone.SimpleProfiler - val prof = new SimpleProfiler(5, 60, 50) + val prof = new SimpleProfiler(10, 120, 50) - val numShards = 32 - val numSamples = 720 // 2 hours * 3600 / 10 sec interval - val numSeries = 100 val startTime = System.currentTimeMillis - (3600*1000) - val numQueries = 500 // Please make sure this number matches the OperationsPerInvocation below val queryIntervalMin = 55 // # minutes between start and stop val queryStep = 150 // # of seconds between each query sample "step" val spread = 5 @@ -145,7 +151,7 @@ class QueryInMemoryBenchmark extends StrictLogging { @Benchmark @BenchmarkMode(Array(Mode.Throughput)) @OutputTimeUnit(TimeUnit.SECONDS) - @OperationsPerInvocation(500) + @OperationsPerInvocation(numQueries) def someOverlapQueries(): Unit = { val futures = (0 until numQueries).map { n => val qCmd = queryCommands(n % queryCommands.length) @@ -164,16 +170,18 @@ class QueryInMemoryBenchmark extends StrictLogging { val qParams2 = TimeStepParams(queryTime/1000, noOverlapStep, (queryTime/1000) + queryIntervalMin*60) val logicalPlans2 = queries.map { q => Parser.queryRangeToLogicalPlan(q, qParams2) } val queryCommands2 = logicalPlans2.map { plan => - LogicalPlan2Query(dataset.ref, plan, QueryContext(Some(new StaticSpreadProvider(SpreadChange(0, 1))), 10000)) + LogicalPlan2Query(dataset.ref, plan, QueryContext(Some(new StaticSpreadProvider(SpreadChange(0, spread))), 10000)) } @Benchmark @BenchmarkMode(Array(Mode.Throughput)) @OutputTimeUnit(TimeUnit.SECONDS) - @OperationsPerInvocation(500) + @OperationsPerInvocation(numQueries) def noOverlapQueries(): Unit = { val futures = (0 until numQueries).map { n => - val f = asyncAsk(coordinator, queryCommands2(n % queryCommands2.length)) + val qCmd = queryCommands2(n % queryCommands2.length) + val time = System.currentTimeMillis + val f = asyncAsk(coordinator, qCmd.copy(qContext = qCmd.qContext.copy(queryId = n.toString, submitTime = time))) f.onSuccess { case q: QueryResult2 => queriesSucceeded += 1 case e: QError => queriesFailed += 1 @@ -195,7 +203,7 @@ class QueryInMemoryBenchmark extends StrictLogging { @Benchmark @BenchmarkMode(Array(Mode.Throughput)) @OutputTimeUnit(TimeUnit.SECONDS) - @OperationsPerInvocation(500) + @OperationsPerInvocation(numQueries) def singleThreadedRawQuery(): Long = { val querySession = QuerySession(QueryContext(), queryConfig) @@ -213,7 +221,7 @@ class QueryInMemoryBenchmark extends StrictLogging { @Benchmark @BenchmarkMode(Array(Mode.Throughput)) @OutputTimeUnit(TimeUnit.SECONDS) - @OperationsPerInvocation(500) + @OperationsPerInvocation(numQueries) def singleThreadedMinOverTimeQuery(): Long = { val f = Observable.fromIterable(0 until numQueries).mapAsync(1) { n => val querySession = QuerySession(QueryContext(), queryConfig) @@ -230,7 +238,7 @@ class QueryInMemoryBenchmark extends StrictLogging { @Benchmark @BenchmarkMode(Array(Mode.Throughput)) @OutputTimeUnit(TimeUnit.SECONDS) - @OperationsPerInvocation(500) + @OperationsPerInvocation(numQueries) def singleThreadedSumRateCCQuery(): Long = { val f = Observable.fromIterable(0 until numQueries).mapAsync(1) { n => val querySession = QuerySession(QueryContext(), queryConfig) diff --git a/standalone/src/main/java/filodb/standalone/SimpleProfiler.java b/standalone/src/main/java/filodb/standalone/SimpleProfiler.java index a2fb25dfcf..5446eefea2 100644 --- a/standalone/src/main/java/filodb/standalone/SimpleProfiler.java +++ b/standalone/src/main/java/filodb/standalone/SimpleProfiler.java @@ -321,6 +321,11 @@ private StackTraceElement[] examine(ThreadInfo info) { return null; } + // XXX: only profile threads which are query scheduler threads + // if (!info.getThreadName().startsWith("query-sched")) { + // return null; + // } + StackTraceElement[] trace = info.getStackTrace(); // Reject internal threads which have no trace at all. From 2b999cb7e7d34416ce3be62db6050bae815c9ee1 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Thu, 25 Jun 2020 10:22:24 -0700 Subject: [PATCH 10/22] bug(core): Populate ODPed chunks in reverse order (#799) OPDed chunks are populated oldest first. This can potentially cause another query to assume that chunks are already there in memory because it checks the oldest chunk time in the chunk-map. This PR makes population of chunks in newest first order. This way concurrent queries will take the ODP path. Not efficient since ODP will be done twice but, acceptable for now. At least it will yield right query results. --- .../columnstore/CassandraColumnStoreSpec.scala | 4 ++-- .../filodb.core/memstore/DemandPagedChunkStore.scala | 5 +++-- .../filodb.core/memstore/PagedReadablePartition.scala | 10 +++++----- .../src/main/scala/filodb.core/store/ChunkSource.scala | 2 +- 4 files changed, 11 insertions(+), 10 deletions(-) diff --git a/cassandra/src/test/scala/filodb.cassandra/columnstore/CassandraColumnStoreSpec.scala b/cassandra/src/test/scala/filodb.cassandra/columnstore/CassandraColumnStoreSpec.scala index 966aa2fab8..f1466f3031 100644 --- a/cassandra/src/test/scala/filodb.cassandra/columnstore/CassandraColumnStoreSpec.scala +++ b/cassandra/src/test/scala/filodb.cassandra/columnstore/CassandraColumnStoreSpec.scala @@ -221,7 +221,7 @@ class CassandraColumnStoreSpec extends ColumnStoreSpec { val parts = lz4ColStore.readRawPartitions(dataset.ref, 0.millis.toMillis, partScan).toListL.runAsync.futureValue parts should have length (1) - parts(0).chunkSets should have length (1) - parts(0).chunkSets(0).vectors.toSeq shouldEqual sourceChunks.head.chunks + parts(0).chunkSetsTimeOrdered should have length (1) + parts(0).chunkSetsTimeOrdered(0).vectors.toSeq shouldEqual sourceChunks.head.chunks } } diff --git a/core/src/main/scala/filodb.core/memstore/DemandPagedChunkStore.scala b/core/src/main/scala/filodb.core/memstore/DemandPagedChunkStore.scala index 9c268c77af..e6945bb55f 100644 --- a/core/src/main/scala/filodb.core/memstore/DemandPagedChunkStore.scala +++ b/core/src/main/scala/filodb.core/memstore/DemandPagedChunkStore.scala @@ -71,9 +71,10 @@ extends RawToPartitionMaker with StrictLogging { tsShard.getPartition(rawPartition.partitionKey).map { tsPart => logger.debug(s"Populating paged chunks for shard=${tsShard.shardNum} partId=${tsPart.partID}") tsShard.shardStats.partitionsPagedFromColStore.increment() - tsShard.shardStats.numChunksPagedIn.increment(rawPartition.chunkSets.size) + tsShard.shardStats.numChunksPagedIn.increment(rawPartition.chunkSetsTimeOrdered.size) // One chunkset at a time, load them into offheap and populate the partition - rawPartition.chunkSets.foreach { case RawChunkSet(infoBytes, rawVectors) => + // Populate newest chunk first so concurrent queries dont assume all data is populated in to chunk-map already + rawPartition.chunkSetsTimeOrdered.reverseIterator.foreach { case RawChunkSet(infoBytes, rawVectors) => // If the chunk is empty, skip it. If no call to allocateOffheap is made, then no check // is made to ensure that the block has room even for metadata. The call to endMetaSpan // might end up returning 0, because the last block doesn't have any room. It's diff --git a/core/src/main/scala/filodb.core/memstore/PagedReadablePartition.scala b/core/src/main/scala/filodb.core/memstore/PagedReadablePartition.scala index 82f7852e71..63440c4b2d 100644 --- a/core/src/main/scala/filodb.core/memstore/PagedReadablePartition.scala +++ b/core/src/main/scala/filodb.core/memstore/PagedReadablePartition.scala @@ -36,14 +36,14 @@ class PagedReadablePartition(override val schema: Schema, import PagedReadablePartition._ val notNeededColIds = if (colIds.nonEmpty) schema.dataInfos.indices.toSet -- colIds.toSet else Set.empty - partData.chunkSets.foreach { vectors => + partData.chunkSetsTimeOrdered.foreach { vectors => // release vectors that are not needed so they can be GCed quickly before scans // finish. This is a temporary workaround since we dont have ability to fetch // specific columns from Cassandra notNeededColIds.foreach(i => vectors.vectors(i) = emptyByteBuffer) } - override def numChunks: Int = partData.chunkSets.length + override def numChunks: Int = partData.chunkSetsTimeOrdered.length override def appendingChunkLen: Int = 0 @@ -51,10 +51,10 @@ class PagedReadablePartition(override val schema: Schema, override def infos(startTime: Long, endTime: Long): ChunkInfoIterator = chunkInfoIteratorImpl - override def hasChunks(method: ChunkScanMethod): Boolean = partData.chunkSets.nonEmpty + override def hasChunks(method: ChunkScanMethod): Boolean = partData.chunkSetsTimeOrdered.nonEmpty override def hasChunksAt(id: ChunkID): Boolean = - partData.chunkSets.iterator + partData.chunkSetsTimeOrdered.iterator .map(c => ChunkSetInfoOnHeap(c.infoBytes, c.vectors)) .exists(_.id == id) @@ -68,7 +68,7 @@ class PagedReadablePartition(override val schema: Schema, private def chunkInfoIteratorImpl = { new ChunkInfoIterator { - private val iter = partData.chunkSets.iterator + private val iter = partData.chunkSetsTimeOrdered.iterator override def close(): Unit = {} override def hasNext: Boolean = iter.hasNext override def nextInfo = ??? // intentionally not implemented since users dont bother with off-heap diff --git a/core/src/main/scala/filodb.core/store/ChunkSource.scala b/core/src/main/scala/filodb.core/store/ChunkSource.scala index 35efdcc104..d1f9461cd6 100644 --- a/core/src/main/scala/filodb.core/store/ChunkSource.scala +++ b/core/src/main/scala/filodb.core/store/ChunkSource.scala @@ -60,7 +60,7 @@ final case class RawChunkSet(infoBytes: Array[Byte], vectors: Array[ByteBuffer]) /** * Raw data for a partition, with one RawChunkSet per ID read */ -final case class RawPartData(partitionKey: Array[Byte], chunkSets: Seq[RawChunkSet]) +final case class RawPartData(partitionKey: Array[Byte], chunkSetsTimeOrdered: Seq[RawChunkSet]) trait ChunkSource extends RawChunkSource with StrictLogging { /** From 427c989a98c4cde8c93f715a35b684dc21722c59 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Thu, 25 Jun 2020 16:53:55 -0700 Subject: [PATCH 11/22] bug(core): ODP Bug fix for Concurrent queries + ODP Unit Test (#801) * A callback in ODP flow was not being invoked if partition creation was needed for concurrent queries. * Also added much needed unit test for ODP. Tests full and partial ODP --- .../columnstore/OdpSpec.scala | 195 ++++++++++++++++++ .../downsample}/OffHeapMemory.scala | 4 +- .../memstore/OnDemandPagingShard.scala | 5 +- .../downsampler/DownsamplerMainSpec.scala | 4 +- 4 files changed, 203 insertions(+), 5 deletions(-) create mode 100644 cassandra/src/test/scala/filodb.cassandra/columnstore/OdpSpec.scala rename {spark-jobs/src/main/scala/filodb/downsampler/chunk => core/src/main/scala/filodb.core/downsample}/OffHeapMemory.scala (98%) diff --git a/cassandra/src/test/scala/filodb.cassandra/columnstore/OdpSpec.scala b/cassandra/src/test/scala/filodb.cassandra/columnstore/OdpSpec.scala new file mode 100644 index 0000000000..f74041b729 --- /dev/null +++ b/cassandra/src/test/scala/filodb.cassandra/columnstore/OdpSpec.scala @@ -0,0 +1,195 @@ +package filodb.cassandra.columnstore + +import scala.concurrent.Future + +import com.typesafe.config.ConfigFactory +import monix.execution.Scheduler +import monix.reactive.Observable +import org.scalatest.{BeforeAndAfterAll, FunSpec, Matchers} +import org.scalatest.concurrent.ScalaFutures +import org.scalatest.time.{Millis, Seconds, Span} + +import filodb.cassandra.DefaultFiloSessionProvider +import filodb.core.{MachineMetricsData, TestData} +import filodb.core.binaryrecord2.{BinaryRecordRowReader, RecordBuilder} +import filodb.core.downsample.OffHeapMemory +import filodb.core.memstore._ +import filodb.core.memstore.FiloSchedulers.QuerySchedName +import filodb.core.metadata.{Dataset, Schemas} +import filodb.core.query.{ColumnFilter, QueryConfig, QueryContext, QuerySession} +import filodb.core.query.Filter.Equals +import filodb.core.store.{InMemoryMetaStore, PartKeyRecord, StoreConfig, TimeRangeChunkScan} +import filodb.memory.format.ZeroCopyUTF8String._ +import filodb.query.{QueryResponse, QueryResult} +import filodb.query.exec.{InProcessPlanDispatcher, MultiSchemaPartitionsExec} + +class OdpSpec extends FunSpec with Matchers with BeforeAndAfterAll with ScalaFutures { + + implicit val defaultPatience = PatienceConfig(timeout = Span(30, Seconds), interval = Span(250, Millis)) + + val config = ConfigFactory.load("application_test.conf").getConfig("filodb") + + implicit val s = monix.execution.Scheduler.Implicits.global + lazy val session = new DefaultFiloSessionProvider(config.getConfig("cassandra")).session + lazy val colStore = new CassandraColumnStore(config, s, session) + + val rawDataStoreConfig = StoreConfig(ConfigFactory.parseString( """ + |flush-interval = 1h + |shard-mem-size = 1MB + |ingestion-buffer-mem-size = 30MB + """.stripMargin)) + + val offheapMem = new OffHeapMemory(Seq(Schemas.gauge), + Map.empty, 100, rawDataStoreConfig) + val schemas = Schemas(Schemas.gauge) + + val dataset = Dataset("prometheus", Schemas.gauge) + val gaugeName = "my_gauge" + var gaugePartKeyBytes: Array[Byte] = _ + val seriesTags = Map("_ws_".utf8 -> "my_ws".utf8, "_ns_".utf8 -> "my_ns".utf8) + val shardStats = new TimeSeriesShardStats(dataset.ref, -1) + + val firstSampleTime = 74373042000L + val numSamples = 100 + val queryScheduler = Scheduler.fixedPool(s"$QuerySchedName", 3) + + // First create the tables in C* + override def beforeAll(): Unit = { + super.beforeAll() + colStore.initialize(dataset.ref, 1).futureValue + colStore.truncate(dataset.ref, 1).futureValue + } + + override def afterAll(): Unit = { + super.afterAll() + queryScheduler.shutdown() + offheapMem.free() + } + + it ("should write gauge data to cassandra") { + val partBuilder = new RecordBuilder(offheapMem.nativeMemoryManager) + val partKey = partBuilder.partKeyFromObjects(Schemas.gauge, gaugeName, seriesTags) + + val part = new TimeSeriesPartition(0, Schemas.gauge, partKey, + 0, offheapMem.bufferPools(Schemas.gauge.schemaHash), shardStats, + offheapMem.nativeMemoryManager, 1) + + gaugePartKeyBytes = part.partKeyBytes + + val rawSamples = Stream.from(0).map { i => + Seq(firstSampleTime + i, i.toDouble, gaugeName, seriesTags) + }.take(numSamples) + + MachineMetricsData.records(dataset, rawSamples).records.foreach { case (base, offset) => + val rr = new BinaryRecordRowReader(Schemas.gauge.ingestionSchema, base, offset) + part.ingest( System.currentTimeMillis(), rr, offheapMem.blockMemFactory) + part.switchBuffers(offheapMem.blockMemFactory, true) + } + val chunks = part.makeFlushChunks(offheapMem.blockMemFactory) + + colStore.write(dataset.ref, Observable.fromIterator(chunks)).futureValue + val pk = PartKeyRecord(gaugePartKeyBytes, firstSampleTime, firstSampleTime + numSamples, Some(150)) + colStore.writePartKeys(dataset.ref, 0, Observable.now(pk), 259200, 34).futureValue + } + + it ("should be able to do full ODP for non concurrent queries") { + val policy = new FixedMaxPartitionsEvictionPolicy(20) + val memStore = new TimeSeriesMemStore(config, colStore, new InMemoryMetaStore(), Some(policy)) + try { + memStore.setup(dataset.ref, schemas, 0, TestData.storeConf) + memStore.recoverIndex(dataset.ref, 0).futureValue + memStore.refreshIndexForTesting(dataset.ref) + + val rvs = query(memStore).futureValue.asInstanceOf[QueryResult] + rvs.result.size shouldEqual 1 + rvs.result.head.rows.toList.size shouldEqual numSamples + } finally { + memStore.shutdown() + } + } + + it ("should be able to do full ODP for concurrent queries") { + val policy = new FixedMaxPartitionsEvictionPolicy(20) + val memStore = new TimeSeriesMemStore(config, colStore, new InMemoryMetaStore(), Some(policy)) + try { + memStore.setup(dataset.ref, schemas, 0, TestData.storeConf) + memStore.recoverIndex(dataset.ref, 0).futureValue + memStore.refreshIndexForTesting(dataset.ref) + + // issue 2 concurrent queries + val res = (0 to 1).map(_ => query(memStore)) + + // all queries should result in all chunks + res.foreach { r => + val rvs = r.futureValue.asInstanceOf[QueryResult] + rvs.result.size shouldEqual 1 + rvs.result.head.rows.toList.size shouldEqual numSamples + } + } finally { + memStore.shutdown() + } + } + + it ("should be able to do partial ODP for non concurrent queries") { + val policy = new FixedMaxPartitionsEvictionPolicy(20) + val memStore = new TimeSeriesMemStore(config, colStore, new InMemoryMetaStore(), Some(policy)) + try { + memStore.setup(dataset.ref, schemas, 0, TestData.storeConf) + memStore.recoverIndex(dataset.ref, 0).futureValue + memStore.refreshIndexForTesting(dataset.ref) + + // ingrest some more samples to trigger partial odp + val rawSamples = Stream.from(0).map { i => + Seq(firstSampleTime + numSamples + i, i.toDouble, gaugeName, seriesTags) + }.take(numSamples) + + memStore.ingest(dataset.ref, 0, SomeData(MachineMetricsData.records(dataset, rawSamples).records, 300)) + + val rvs = query(memStore).futureValue.asInstanceOf[QueryResult] + rvs.result.size shouldEqual 1 + rvs.result.head.rows.toList.size shouldEqual numSamples * 2 + } finally { + memStore.shutdown() + } + } + + it ("should be able to do partial ODP for concurrent queries") { + val policy = new FixedMaxPartitionsEvictionPolicy(20) + val memStore = new TimeSeriesMemStore(config, colStore, new InMemoryMetaStore(), Some(policy)) + try { + memStore.setup(dataset.ref, schemas, 0, TestData.storeConf) + memStore.recoverIndex(dataset.ref, 0).futureValue + memStore.refreshIndexForTesting(dataset.ref) + + // ingrest some more samples to trigger partial odp + val rawSamples = Stream.from(0).map { i => + Seq(firstSampleTime + numSamples + i, i.toDouble, gaugeName, seriesTags) + }.take(numSamples) + + memStore.ingest(dataset.ref, 0, SomeData(MachineMetricsData.records(dataset, rawSamples).records, 300)) + + // issue 2 concurrent queries + val res = (0 to 1).map(_ => query(memStore)) + + // all queries should result in all chunks + res.foreach { r => + val rvs = r.futureValue.asInstanceOf[QueryResult] + rvs.result.size shouldEqual 1 + rvs.result.head.rows.toList.size shouldEqual numSamples * 2 + } + } finally { + memStore.shutdown() + } + } + + def query(memStore: TimeSeriesMemStore): Future[QueryResponse] = { + val colFilters = seriesTags.map { case (t, v) => ColumnFilter(t.toString, Equals(v.toString)) }.toSeq + val queryFilters = colFilters :+ ColumnFilter("_metric_", Equals(gaugeName)) + val exec = MultiSchemaPartitionsExec(QueryContext(sampleLimit = numSamples * 2), InProcessPlanDispatcher, + dataset.ref, 0, queryFilters, TimeRangeChunkScan(firstSampleTime, firstSampleTime + 2 * numSamples)) + val queryConfig = new QueryConfig(config.getConfig("query")) + val querySession = QuerySession(QueryContext(), queryConfig) + exec.execute(memStore, querySession)(queryScheduler).runAsync(queryScheduler) + } +} + diff --git a/spark-jobs/src/main/scala/filodb/downsampler/chunk/OffHeapMemory.scala b/core/src/main/scala/filodb.core/downsample/OffHeapMemory.scala similarity index 98% rename from spark-jobs/src/main/scala/filodb/downsampler/chunk/OffHeapMemory.scala rename to core/src/main/scala/filodb.core/downsample/OffHeapMemory.scala index 1b72bb3d11..2d6992b915 100644 --- a/spark-jobs/src/main/scala/filodb/downsampler/chunk/OffHeapMemory.scala +++ b/core/src/main/scala/filodb.core/downsample/OffHeapMemory.scala @@ -1,4 +1,4 @@ -package filodb.downsampler.chunk +package filodb.core.downsample import com.typesafe.scalalogging.StrictLogging @@ -50,4 +50,4 @@ class OffHeapMemory(schemas: Seq[Schema], nativeMemoryManager.shutdown() } -} \ No newline at end of file +} diff --git a/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala b/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala index 28d32780bf..6a6311dcc9 100644 --- a/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala +++ b/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala @@ -217,7 +217,10 @@ TimeSeriesShard(ref, schemas, storeConfig, shardNum, bufferMemoryManager, rawSto } // create the partition and update data structures (but no need to add to Lucene!) // NOTE: if no memory, then no partition! - case p: TimeSeriesPartition => Some(p) + case p: TimeSeriesPartition => + // invoke callback even if we didn't create the partition + callback(p.partID, p.partKeyBytes) + Some(p) } }.toVector.flatten } diff --git a/spark-jobs/src/test/scala/filodb/downsampler/DownsamplerMainSpec.scala b/spark-jobs/src/test/scala/filodb/downsampler/DownsamplerMainSpec.scala index 4d768ebae9..275da49012 100644 --- a/spark-jobs/src/test/scala/filodb/downsampler/DownsamplerMainSpec.scala +++ b/spark-jobs/src/test/scala/filodb/downsampler/DownsamplerMainSpec.scala @@ -18,14 +18,14 @@ import filodb.cardbuster.CardinalityBuster import filodb.core.GlobalScheduler._ import filodb.core.MachineMetricsData import filodb.core.binaryrecord2.{BinaryRecordRowReader, RecordBuilder, RecordSchema} -import filodb.core.downsample.DownsampledTimeSeriesStore +import filodb.core.downsample.{DownsampledTimeSeriesStore, OffHeapMemory} import filodb.core.memstore.{PagedReadablePartition, TimeSeriesPartition} import filodb.core.memstore.FiloSchedulers.QuerySchedName import filodb.core.metadata.{Dataset, Schemas} import filodb.core.query._ import filodb.core.query.Filter.Equals import filodb.core.store.{AllChunkScan, PartKeyRecord, SinglePartitionScan, StoreConfig} -import filodb.downsampler.chunk.{BatchDownsampler, Downsampler, DownsamplerSettings, OffHeapMemory} +import filodb.downsampler.chunk.{BatchDownsampler, Downsampler, DownsamplerSettings} import filodb.downsampler.index.{DSIndexJobSettings, IndexJobDriver} import filodb.memory.format.{PrimitiveVectorReader, UnsafeUtils} import filodb.memory.format.ZeroCopyUTF8String._ From 14cde44e35339f3d0945ef87550ce736d892cb11 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Fri, 26 Jun 2020 11:48:01 -0700 Subject: [PATCH 12/22] feat(core): More accurate data sizing for queries on downsample shard (#802) If there is a lot of churn in time series, raw and downsample shards limit queries too aggressively due to assumption that every matching time series is ingesting for entire time range. Downsample shard can do more accurate sizing estimate by paying cost of index lookup. This is what this PR does. It looks up start/end times of all matching partitions and then decides whether to continue the query. --- .../DownsampledTimeSeriesShard.scala | 49 +++++++------------ .../memstore/OnDemandPagingShard.scala | 2 +- .../memstore/TimeSeriesShard.scala | 3 +- .../scala/filodb.core/metadata/Schemas.scala | 43 +++++++++++++--- 4 files changed, 58 insertions(+), 39 deletions(-) diff --git a/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesShard.scala b/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesShard.scala index 19aae40be2..f83ba9bc17 100644 --- a/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesShard.scala @@ -216,17 +216,19 @@ class DownsampledTimeSeriesShard(rawDatasetRef: DatasetRef, case FilteredPartitionScan(split, filters) => if (filters.nonEmpty) { - val res = partKeyIndex.partIdsFromFilters(filters, - chunkMethod.startTime, - chunkMethod.endTime) - val firstPartId = if (res.isEmpty) None else Some(res(0)) - - val _schema = firstPartId.map(schemaIDFromPartID) + // This API loads all part keys into heap and can potentially be large size for + // high cardinality queries, but it is needed to do multiple + // iterations over the part keys. First iteration is for data size estimation. + // Second iteration is for query result evaluation. Loading everything to heap + // is expensive, but we do it to handle data sizing for metrics that have + // continuous churn. See capDataScannedPerShardCheck method. + val recs = partKeyIndex.partKeyRecordsFromFilters(filters, chunkMethod.startTime, chunkMethod.endTime) + val _schema = recs.headOption.map { pkRec => + RecordSchema.schemaID(pkRec.partKey, UnsafeUtils.arayOffset) + } stats.queryTimeRangeMins.record((chunkMethod.endTime - chunkMethod.startTime) / 60000 ) - - // send index result in the partsInMemory field of lookup - PartLookupResult(shardNum, chunkMethod, res, - _schema, debox.Map.empty[Int, Long], debox.Buffer.empty) + PartLookupResult(shardNum, chunkMethod, debox.Buffer.empty, + _schema, debox.Map.empty, debox.Buffer.empty, recs) } else { throw new UnsupportedOperationException("Cannot have empty filters") } @@ -247,10 +249,8 @@ class DownsampledTimeSeriesShard(rawDatasetRef: DatasetRef, // Create a ReadablePartition objects that contain the time series data. This can be either a // PagedReadablePartitionOnHeap or PagedReadablePartitionOffHeap. This will be garbage collected/freed // when query is complete. - val partKeys = lookup.partsInMemory.iterator().map(partKeyFromPartId) - Observable.fromIterator(partKeys) - // 3 times value configured for raw dataset since expected throughput for downsampled cluster is much lower - .mapAsync(downsampleStoreConfig.demandPagingParallelism) { partBytes => + Observable.fromIterable(lookup.pkRecords) + .mapAsync(downsampleStoreConfig.demandPagingParallelism) { partRec => val partLoadSpan = Kamon.spanBuilder(s"single-partition-cassandra-latency") .asChildOf(Kamon.currentSpan()) .tag("dataset", rawDatasetRef.toString) @@ -259,7 +259,7 @@ class DownsampledTimeSeriesShard(rawDatasetRef: DatasetRef, // TODO test multi-partition scan if latencies are high store.readRawPartitions(downsampledDataset, downsampleStoreConfig.maxChunkTime.toMillis, - SinglePartitionScan(partBytes, shardNum), + SinglePartitionScan(partRec.partKey, shardNum), lookup.chunkMethod) .map { pd => val part = makePagedPartition(pd, lookup.firstSchemaId.get, colIds) @@ -268,30 +268,19 @@ class DownsampledTimeSeriesShard(rawDatasetRef: DatasetRef, partLoadSpan.finish() part } - .defaultIfEmpty(makePagedPartition(RawPartData(partBytes, Seq.empty), lookup.firstSchemaId.get, colIds)) + .defaultIfEmpty(makePagedPartition(RawPartData(partRec.partKey, Seq.empty), lookup.firstSchemaId.get, colIds)) .headL } } private def capDataScannedPerShardCheck(lookup: PartLookupResult, resolution: Long) = { lookup.firstSchemaId.foreach { schId => - lookup.chunkMethod match { - case TimeRangeChunkScan(st, end) => - schemas.ensureQueriedDataSizeWithinLimit(schId, lookup.partsInMemory.length, - downsampleStoreConfig.flushInterval.toMillis, - resolution, end - st, downsampleStoreConfig.maxDataPerShardQuery) - case _ => - } + schemas.ensureQueriedDataSizeWithinLimit(schId, lookup.pkRecords, + downsampleStoreConfig.flushInterval.toMillis, + resolution, lookup.chunkMethod, downsampleStoreConfig.maxDataPerShardQuery) } } - protected def schemaIDFromPartID(partID: Int): Int = { - partKeyIndex.partKeyFromPartId(partID).map { pkBytesRef => - val unsafeKeyOffset = PartKeyLuceneIndex.bytesRefToUnsafeOffset(pkBytesRef.offset) - RecordSchema.schemaID(pkBytesRef.bytes, unsafeKeyOffset) - }.getOrElse(throw new IllegalStateException(s"PartId $partID returned by lucene, but partKey not found")) - } - private def chooseDownsampleResolution(chunkScanMethod: ChunkScanMethod): (Long, DatasetRef) = { chunkScanMethod match { case AllChunkScan => diff --git a/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala b/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala index 6a6311dcc9..5c6d63f702 100644 --- a/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala +++ b/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala @@ -58,7 +58,7 @@ TimeSeriesShard(ref, schemas, storeConfig, shardNum, bufferMemoryManager, rawSto lookup.chunkMethod match { case TimeRangeChunkScan(st, end) => val numMatches = lookup.partsInMemory.length + lookup.partIdsNotInMemory.length - schemas.ensureQueriedDataSizeWithinLimit(schId, numMatches, + schemas.ensureQueriedDataSizeWithinLimitApprox(schId, numMatches, storeConfig.flushInterval.toMillis, assumedResolution, end - st, storeConfig.maxDataPerShardQuery) case _ => diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala index d9fc677d14..f2ca3fa6f6 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala @@ -189,7 +189,8 @@ case class PartLookupResult(shard: Int, partsInMemory: debox.Buffer[Int], firstSchemaId: Option[Int] = None, partIdsMemTimeGap: debox.Map[Int, Long] = debox.Map.empty, - partIdsNotInMemory: debox.Buffer[Int] = debox.Buffer.empty) + partIdsNotInMemory: debox.Buffer[Int] = debox.Buffer.empty, + pkRecords: Seq[PartKeyLuceneIndexRecord] = Seq.empty) final case class SchemaMismatch(expected: String, found: String) extends Exception(s"Multiple schemas found, please filter. Expected schema $expected, found schema $found") diff --git a/core/src/main/scala/filodb.core/metadata/Schemas.scala b/core/src/main/scala/filodb.core/metadata/Schemas.scala index cfb245bb1a..635617c458 100644 --- a/core/src/main/scala/filodb.core/metadata/Schemas.scala +++ b/core/src/main/scala/filodb.core/metadata/Schemas.scala @@ -9,9 +9,10 @@ import filodb.core.GlobalConfig import filodb.core.Types._ import filodb.core.binaryrecord2._ import filodb.core.downsample.{ChunkDownsampler, DownsamplePeriodMarker} +import filodb.core.memstore.PartKeyLuceneIndexRecord import filodb.core.metadata.Column.ColumnType import filodb.core.query.ColumnInfo -import filodb.core.store.ChunkSetInfo +import filodb.core.store.{ChunkScanMethod, ChunkSetInfo} import filodb.memory.BinaryRegion import filodb.memory.format._ import filodb.memory.format.BinaryVector.BinaryVectorPtr @@ -295,12 +296,12 @@ final case class Schemas(part: PartitionSchema, * (b) it also assigns bytes per sample based on schema which is much of a swag. In reality, it would depend on * number of histogram buckets, samples per chunk etc. */ - def ensureQueriedDataSizeWithinLimit(schemaId: Int, - numTsPartitions: Int, - chunkDurationMillis: Long, - resolutionMs: Long, - queryDurationMs: Long, - dataSizeLimit: Long): Unit = { + def ensureQueriedDataSizeWithinLimitApprox(schemaId: Int, + numTsPartitions: Int, + chunkDurationMillis: Long, + resolutionMs: Long, + queryDurationMs: Long, + dataSizeLimit: Long): Unit = { val numSamplesPerChunk = chunkDurationMillis / resolutionMs // find number of chunks to be scanned. Ceil division needed here val numChunksPerTs = (queryDurationMs + chunkDurationMillis - 1) / chunkDurationMillis @@ -313,6 +314,34 @@ final case class Schemas(part: PartitionSchema, s"(b) reduce query time range, currently at ${queryDurationMs / 1000 / 60 } minutes") } + /** + * This method estimates data size with much better accuracy than ensureQueriedDataSizeWithinLimitApprox + * since it accepts the start/end times of each matching part key. It is able to handle estimation with + * time series churn much better + */ + def ensureQueriedDataSizeWithinLimit(schemaId: Int, + pkRecs: Seq[PartKeyLuceneIndexRecord], + chunkDurationMillis: Long, + resolutionMs: Long, + chunkMethod: ChunkScanMethod, + dataSizeLimit: Long): Unit = { + val numSamplesPerChunk = chunkDurationMillis / resolutionMs + val bytesPerSample = bytesPerSampleSwag(schemaId) + var estDataSize = 0d + val quRange = chunkMethod.startTime to chunkMethod.endTime + pkRecs.foreach { pkRec => + val intersection = Math.min(chunkMethod.endTime, pkRec.endTime) - Math.max(chunkMethod.startTime, pkRec.startTime) + // find number of chunks to be scanned. Ceil division needed here + val numChunks = (intersection + chunkDurationMillis - 1) / chunkDurationMillis + estDataSize += bytesPerSample * numSamplesPerChunk * numChunks + } + require(estDataSize < dataSizeLimit, + s"With match of ${pkRecs.length} time series, estimate of $estDataSize bytes exceeds limit of " + + s"$dataSizeLimit bytes queried per shard for ${_schemas(schemaId).name} schema. Try one or more of these: " + + s"(a) narrow your query filters to reduce to fewer than the current ${pkRecs.length} matches " + + s"(b) reduce query time range, currently at ${quRange.length / 1000 / 60 } minutes") + } + /** * Returns the Schema for a given schemaID, or UnknownSchema if not found */ From a12bb27d90e30871377975fd08d31182fdb65ef1 Mon Sep 17 00:00:00 2001 From: "Brian S. O'Neill" Date: Mon, 29 Jun 2020 14:58:36 -0700 Subject: [PATCH 13/22] fix(query): Add test validation to be enabled later. (#806) --- .../src/main/scala/filodb.memory/data/ChunkMap.scala | 6 +++++- .../query/exec/InProcessPlanDispatcherSpec.scala | 12 +++++++++--- .../query/exec/rangefn/AbsentFunctionSpec.scala | 9 +++++++-- .../exec/rangefn/AggrOverTimeFunctionsSpec.scala | 9 +++++++-- 4 files changed, 28 insertions(+), 8 deletions(-) diff --git a/memory/src/main/scala/filodb.memory/data/ChunkMap.scala b/memory/src/main/scala/filodb.memory/data/ChunkMap.scala index dc086f3a27..a75ce687ab 100644 --- a/memory/src/main/scala/filodb.memory/data/ChunkMap.scala +++ b/memory/src/main/scala/filodb.memory/data/ChunkMap.scala @@ -120,7 +120,7 @@ object ChunkMap extends StrictLogging { * consumption from a query iterator. If there are lingering locks, * it is quite possible a lock acquire or release bug exists */ - def validateNoSharedLocks(execPlan: String): Unit = { + def validateNoSharedLocks(execPlan: String, unitTest: Boolean = false): Unit = { val t = Thread.currentThread() if (execPlanTracker.containsKey(t)) { logger.error(s"Current thread ${t.getName} did not release lock for execPlan: ${execPlanTracker.get(t)}") @@ -128,6 +128,10 @@ object ChunkMap extends StrictLogging { val numLocksReleased = ChunkMap.releaseAllSharedLocks() if (numLocksReleased > 0) { + if (unitTest) { + // FIXME: Uncomment this when tests have been fixed. + //throw new Error(s"Number of locks was non-zero: $numLocksReleased") + } logger.error(s"Number of locks was non-zero: $numLocksReleased. " + s"This is indicative of a possible lock acquisition/release bug.") // FIXME: Causes failures when running the unit tests for some unknown reason. diff --git a/query/src/test/scala/filodb/query/exec/InProcessPlanDispatcherSpec.scala b/query/src/test/scala/filodb/query/exec/InProcessPlanDispatcherSpec.scala index c82695a2f4..96c228f961 100644 --- a/query/src/test/scala/filodb/query/exec/InProcessPlanDispatcherSpec.scala +++ b/query/src/test/scala/filodb/query/exec/InProcessPlanDispatcherSpec.scala @@ -9,7 +9,7 @@ import scala.concurrent.duration.FiniteDuration import com.typesafe.config.{Config, ConfigFactory} import monix.eval.Task import monix.execution.Scheduler -import org.scalatest.{BeforeAndAfterAll, FunSpec, Matchers} +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSpec, Matchers} import org.scalatest.concurrent.ScalaFutures import org.scalatest.time.{Millis, Seconds, Span} @@ -21,11 +21,13 @@ import filodb.core.metadata.{Column, Dataset, Schemas} import filodb.core.query.{ColumnFilter, Filter, QueryConfig, QueryContext, QuerySession} import filodb.core.store.{AllChunkScan, InMemoryMetaStore, NullColumnStore} import filodb.memory.MemFactory +import filodb.memory.data.ChunkMap import filodb.memory.format.{SeqRowReader, ZeroCopyUTF8String} import filodb.query._ // So, this is effectively a test for NonLeafExecPlan -class InProcessPlanDispatcherSpec extends FunSpec with Matchers with ScalaFutures with BeforeAndAfterAll { +class InProcessPlanDispatcherSpec extends FunSpec + with Matchers with ScalaFutures with BeforeAndAfter with BeforeAndAfterAll { import ZeroCopyUTF8String._ import filodb.core.{MachineMetricsData => MMD} @@ -45,6 +47,10 @@ class InProcessPlanDispatcherSpec extends FunSpec with Matchers with ScalaFuture memStore.refreshIndexForTesting(MMD.histMaxDS.ref) } + after { + ChunkMap.validateNoSharedLocks("InProcessPlanDispatcherSpec", true) + } + override def afterAll(): Unit = { memStore.shutdown() } @@ -171,4 +177,4 @@ case class DummyDispatcher(memStore: TimeSeriesMemStore, querySession: QuerySess (implicit sched: Scheduler): Task[QueryResponse] = { plan.execute(memStore, querySession) } -} \ No newline at end of file +} diff --git a/query/src/test/scala/filodb/query/exec/rangefn/AbsentFunctionSpec.scala b/query/src/test/scala/filodb/query/exec/rangefn/AbsentFunctionSpec.scala index 7a43ae816a..61c2381564 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/AbsentFunctionSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/AbsentFunctionSpec.scala @@ -4,16 +4,21 @@ package filodb.query.exec.rangefn import com.typesafe.config.{Config, ConfigFactory} import monix.execution.Scheduler.Implicits.global import monix.reactive.Observable -import org.scalatest.{FunSpec, Matchers} +import org.scalatest.{BeforeAndAfter, FunSpec, Matchers} import org.scalatest.concurrent.ScalaFutures import filodb.core.MetricsTestData import filodb.core.query._ import filodb.core.query.Filter.{Equals, NotEqualsRegex} +import filodb.memory.data.ChunkMap import filodb.memory.format.{RowReader, ZeroCopyUTF8String} import filodb.query.exec -class AbsentFunctionSpec extends FunSpec with Matchers with ScalaFutures { +class AbsentFunctionSpec extends FunSpec with Matchers with ScalaFutures with BeforeAndAfter { + after { + ChunkMap.validateNoSharedLocks("AbsentFunctionSpec", true) + } + val config: Config = ConfigFactory.load("application_test.conf").getConfig("filodb") val resultSchema = ResultSchema(MetricsTestData.timeseriesSchema.infosFromIDs(0 to 1), 1) val queryConfig = new QueryConfig(config.getConfig("query")) diff --git a/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala b/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala index 84407eeaed..8ebce29b2f 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala @@ -4,20 +4,21 @@ import scala.collection.mutable.ArrayBuffer import scala.util.Random import com.typesafe.config.ConfigFactory -import org.scalatest.{BeforeAndAfterAll, FunSpec, Matchers} +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSpec, Matchers} import filodb.core.{MetricsTestData, QueryTimeoutException, TestData, MachineMetricsData => MMD} import filodb.core.memstore.{TimeSeriesPartition, TimeSeriesPartitionSpec, WriteBufferPool} import filodb.core.query._ import filodb.core.store.AllChunkScan import filodb.memory._ +import filodb.memory.data.ChunkMap import filodb.memory.format.{TupleRowReader, vectors => bv} import filodb.query.exec._ /** * A common trait for windowing query tests which uses real chunks and real RawDataRangeVectors */ -trait RawDataWindowingSpec extends FunSpec with Matchers with BeforeAndAfterAll { +trait RawDataWindowingSpec extends FunSpec with Matchers with BeforeAndAfter with BeforeAndAfterAll { import MetricsTestData._ private val blockStore = new PageAlignedBlockManager(100 * 1024 * 1024, @@ -31,6 +32,10 @@ trait RawDataWindowingSpec extends FunSpec with Matchers with BeforeAndAfterAll MMD.dummyContext, true) protected val tsBufferPool2 = new WriteBufferPool(TestData.nativeMem, downsampleSchema.data, storeConf) + after { + ChunkMap.validateNoSharedLocks(getClass().toString(), true) + } + override def afterAll(): Unit = { blockStore.releaseBlocks() } From c22c0e73cfaa855dd005f4cfdc12fd653be97b64 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Mon, 29 Jun 2020 15:28:48 -0700 Subject: [PATCH 14/22] debug(core): Log details if onReclaim blockMetaSize assertion fails, and halt process proactively (#805) --- .../scala/filodb.core/memstore/TimeSeriesShard.scala | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala index f2ca3fa6f6..5e9c194bb4 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala @@ -281,8 +281,18 @@ class TimeSeriesShard(val ref: DatasetRef, val partID = UnsafeUtils.getInt(metaAddr) val partition = partitions.get(partID) if (partition != UnsafeUtils.ZeroPointer) { - assert(numBytes == partition.schema.data.blockMetaSize) + // The number of bytes passed in is the metadata size which depends on schema. It should match the + // TSPartition's blockMetaSize; if it doesn't that is a flag for possible corruption, and we should halt + // the process to be safe and log details for further debugging. val chunkID = UnsafeUtils.getLong(metaAddr + 4) + if (numBytes != partition.schema.data.blockMetaSize) { + logger.error(f"POSSIBLE CORRUPTION DURING onReclaim(metaAddr=0x$metaAddr%08x, numBytes=$numBytes)" + + s"Expected meta size: ${partition.schema.data.blockMetaSize} for schema=${partition.schema}" + + s" Reclaiming chunk chunkID=$chunkID from shard=$shardNum " + + s"partID=$partID ${partition.stringPartition}") + logger.warn("Halting FiloDB...") + sys.exit(33) // Special onReclaim corruption exit code + } partition.removeChunksAt(chunkID) logger.debug(s"Reclaiming chunk chunkID=$chunkID from shard=$shardNum " + s"partID=$partID ${partition.stringPartition}") From d3983200d6806746f97e0b038c740625ae2526ca Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Tue, 30 Jun 2020 08:28:56 -0700 Subject: [PATCH 15/22] feat(query): Introducing RangeVectorCursor (#804) Today RangeVectors expose sample rows as Iterator[RowReader] which cannot be closed. Because of this, there isn't a cascading way to close the ChunkInfoIterator, thereby causing room for developer error where locks are not released. This PR changes RangeVector to use RangeVectorCursor which is designed to cascade the close call to all wrapped iterators, finally culminating with close of the ChunkInfoIterator(s), thereby releasing the locks reliably. This PR will force developers during compile time to think about closing their underlying cursors. --- .../client/SerializationSpec.scala | 9 +++- .../query/PartitionTimeRangeReader.scala | 6 ++- .../scala/filodb.core/query/RangeVector.scala | 27 +++++++---- .../filodb.core/query/RangeVectorCursor.scala | 42 +++++++++++++++++ .../filodb.core/store/ReadablePartition.scala | 8 ++-- .../filodb.core/query/RangeVectorSpec.scala | 5 +- .../query/exec/AggrOverRangeVectors.scala | 23 +++++++-- .../filodb/query/exec/BinaryJoinExec.scala | 10 +++- .../scala/filodb/query/exec/ExecPlan.scala | 1 - .../query/exec/HistogramQuantileMapper.scala | 3 +- .../filodb/query/exec/MetadataExecPlan.scala | 3 ++ .../query/exec/MetadataRemoteExec.scala | 1 + .../query/exec/PeriodicSamplesMapper.scala | 16 +++++-- .../filodb/query/exec/PromQlRemoteExec.scala | 11 +++-- .../query/exec/RangeVectorTransformer.scala | 47 +++++++++++-------- .../filodb/query/exec/StitchRvsExec.scala | 6 ++- .../aggregator/CountValuesRowAggregator.scala | 1 + .../aggregator/QuantileRowAggregator.scala | 2 +- .../aggregator/TopBottomKRowAggregator.scala | 1 + .../scala/filodb/query/ResultTypesSpec.scala | 14 ++++-- .../query/exec/AggrOverRangeVectorsSpec.scala | 9 ++-- .../query/exec/BinaryJoinExecSpec.scala | 31 +++++++----- .../query/exec/BinaryJoinGroupingSpec.scala | 29 ++++++++---- .../exec/BinaryJoinSetOperatorSpec.scala | 37 ++++++++++----- .../exec/HistToPromSeriesMapperSpec.scala | 2 + .../exec/HistogramQuantileMapperSpec.scala | 4 ++ .../query/exec/LastSampleFunctionSpec.scala | 16 +++---- .../filodb/query/exec/StitchRvsExecSpec.scala | 10 ++-- .../query/exec/WindowIteratorSpec.scala | 11 +++-- .../exec/rangefn/AbsentFunctionSpec.scala | 14 ++++-- .../exec/rangefn/BinaryOperatorSpec.scala | 37 ++++++++++----- .../exec/rangefn/InstantFunctionSpec.scala | 33 ++++++++----- .../query/exec/rangefn/LabelReplaceSpec.scala | 28 +++++++---- .../query/exec/rangefn/LableJoinSpec.scala | 14 ++++-- .../exec/rangefn/ScalarFunctionSpec.scala | 41 +++++++++------- .../query/exec/rangefn/SortFunctionSpec.scala | 30 ++++++++---- 36 files changed, 396 insertions(+), 186 deletions(-) create mode 100644 core/src/main/scala/filodb.core/query/RangeVectorCursor.scala diff --git a/coordinator/src/test/scala/filodb.coordinator/client/SerializationSpec.scala b/coordinator/src/test/scala/filodb.coordinator/client/SerializationSpec.scala index 05df4f7e2a..876183ce46 100644 --- a/coordinator/src/test/scala/filodb.coordinator/client/SerializationSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/client/SerializationSpec.scala @@ -14,7 +14,7 @@ import filodb.core.metadata.{Dataset, Schemas} import filodb.core.metadata.Column.ColumnType import filodb.core.query.QueryConfig import filodb.core.store.IngestionConfig -import filodb.memory.format.{RowReader, SeqRowReader, UTF8MapIteratorRowReader, ZeroCopyUTF8String => UTF8Str} +import filodb.memory.format.{SeqRowReader, UTF8MapIteratorRowReader, ZeroCopyUTF8String => UTF8Str} import filodb.prometheus.ast.TimeStepParams import filodb.prometheus.parse.Parser import filodb.query.{QueryResult => QueryResult2, _} @@ -139,7 +139,10 @@ class SerializationSpec extends ActorTest(SerializationSpecConfig.getNewSystem) new ColumnInfo("value", ColumnType.DoubleColumn)) val srvs = for { i <- 0 to 9 } yield { val rv = new RangeVector { - override val rows: Iterator[RowReader] = rowbuf.iterator + override val rows: RangeVectorCursor = { + import NoCloseCursor._ + rowbuf.iterator + } override val key: RangeVectorKey = rvKey } val srv = SerializedRangeVector(rv, cols) @@ -284,6 +287,7 @@ class SerializationSpec extends ActorTest(SerializationSpecConfig.getNewSystem) UTF8Str("key2") -> UTF8Str("val2")) val key = CustomRangeVectorKey(keysMap) val cols = Seq(ColumnInfo("value", ColumnType.DoubleColumn)) + import filodb.core.query.NoCloseCursor._ val ser = SerializedRangeVector(IteratorBackedRangeVector(key, Iterator.empty), cols) val schema = ResultSchema(MachineMetricsData.dataset1.schema.infosFromIDs(0 to 0), 1) @@ -300,6 +304,7 @@ class SerializationSpec extends ActorTest(SerializationSpecConfig.getNewSystem) val expected = Seq(Map("App-0" -> "App-1")) val schema = new ResultSchema(Seq(new ColumnInfo("_ns_", ColumnType.MapColumn)), 1) val cols = Seq(ColumnInfo("value", ColumnType.MapColumn)) + import filodb.core.query.NoCloseCursor._ val ser = Seq(SerializedRangeVector(IteratorBackedRangeVector(new CustomRangeVectorKey(Map.empty), new UTF8MapIteratorRowReader(input.toIterator)), cols)) diff --git a/core/src/main/scala/filodb.core/query/PartitionTimeRangeReader.scala b/core/src/main/scala/filodb.core/query/PartitionTimeRangeReader.scala index d66578c915..a115b88aa4 100644 --- a/core/src/main/scala/filodb.core/query/PartitionTimeRangeReader.scala +++ b/core/src/main/scala/filodb.core/query/PartitionTimeRangeReader.scala @@ -16,7 +16,7 @@ final class PartitionTimeRangeReader(part: ReadablePartition, startTime: Long, endTime: Long, infos: ChunkInfoIterator, - columnIDs: Array[Int]) extends Iterator[RowReader] { + columnIDs: Array[Int]) extends RangeVectorCursor { // MinValue = no current chunk private var curChunkID = Long.MinValue private final val vectorIts = new Array[TypedIterator](columnIDs.size) @@ -97,4 +97,8 @@ final class PartitionTimeRangeReader(part: ReadablePartition, rowNo += 1 rowReader } + + final def close(): Unit = { + infos.close() + } } \ No newline at end of file diff --git a/core/src/main/scala/filodb.core/query/RangeVector.scala b/core/src/main/scala/filodb.core/query/RangeVector.scala index 9a7c976fc7..389616984b 100644 --- a/core/src/main/scala/filodb.core/query/RangeVector.scala +++ b/core/src/main/scala/filodb.core/query/RangeVector.scala @@ -2,6 +2,8 @@ package filodb.core.query import java.time.{LocalDateTime, YearMonth, ZoneOffset} +import scala.collection.Iterator + import com.typesafe.scalalogging.StrictLogging import debox.Buffer import kamon.Kamon @@ -97,7 +99,7 @@ object CustomRangeVectorKey { */ trait RangeVector { def key: RangeVectorKey - def rows: Iterator[RowReader] + def rows(): RangeVectorCursor def numRows: Option[Int] = None def prettyPrint(formatTime: Boolean = true): String = "RV String Not supported" } @@ -122,7 +124,8 @@ trait ScalarRangeVector extends SerializableRangeVector { * ScalarRangeVector which has time specific value */ final case class ScalarVaryingDouble(private val timeValueMap: Map[Long, Double]) extends ScalarRangeVector { - override def rows: Iterator[RowReader] = timeValueMap.toList.sortWith(_._1 < _._1). + import NoCloseCursor._ + override def rows: RangeVectorCursor = timeValueMap.toList.sortWith(_._1 < _._1). map { x => new TransientRow(x._1, x._2) }.iterator def getValue(time: Long): Double = timeValueMap(time) @@ -135,7 +138,8 @@ trait ScalarSingleValue extends ScalarRangeVector { def rangeParams: RangeParams var numRowsInt : Int = 0 - override def rows: Iterator[RowReader] = { + override def rows(): RangeVectorCursor = { + import NoCloseCursor._ Iterator.from(0, rangeParams.stepSecs.toInt).takeWhile(_ <= rangeParams.endSecs - rangeParams.startSecs).map { i => numRowsInt += 1 val t = i + rangeParams.startSecs @@ -220,11 +224,11 @@ final case class DaysInMonthScalar(rangeParams: RangeParams) extends ScalarSingl // First column of columnIDs should be the timestamp column final case class RawDataRangeVector(key: RangeVectorKey, - val partition: ReadablePartition, + partition: ReadablePartition, chunkMethod: ChunkScanMethod, columnIDs: Array[Int]) extends RangeVector { // Iterators are stateful, for correct reuse make this a def - def rows: Iterator[RowReader] = partition.timeRangeRows(chunkMethod, columnIDs) + def rows(): RangeVectorCursor = partition.timeRangeRows(chunkMethod, columnIDs) // Obtain ChunkSetInfos from specific window of time from partition def chunkInfos(windowStart: Long, windowEnd: Long): ChunkInfoIterator = partition.infos(windowStart, windowEnd) @@ -243,8 +247,9 @@ final case class ChunkInfoRangeVector(key: RangeVectorKey, chunkMethod: ChunkScanMethod, column: Column) extends RangeVector { val reader = new ChunkInfoRowReader(column) + import NoCloseCursor._ // Iterators are stateful, for correct reuse make this a def - def rows: Iterator[RowReader] = partition.infos(chunkMethod).map { info => + def rows(): RangeVectorCursor = partition.infos(chunkMethod).map { info => reader.setInfo(info) reader } @@ -265,9 +270,9 @@ final class SerializedRangeVector(val key: RangeVectorKey, java.io.Serializable { override val numRows = Some(numRowsInt) - + import NoCloseCursor._ // Possible for records to spill across containers, so we read from all containers - override def rows: Iterator[RowReader] = + override def rows: RangeVectorCursor = containers.toIterator.flatMap(_.iterate(schema)).slice(startRecordNo, startRecordNo + numRowsInt) /** @@ -319,6 +324,7 @@ object SerializedRangeVector extends StrictLogging { builder.addFromReader(rows.next, 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() @@ -356,14 +362,14 @@ object SerializedRangeVector extends StrictLogging { } final case class IteratorBackedRangeVector(key: RangeVectorKey, - rows: Iterator[RowReader]) extends RangeVector + rows: RangeVectorCursor) extends RangeVector final case class BufferRangeVector(key: RangeVectorKey, timestamps: Buffer[Long], values: Buffer[Double]) extends RangeVector { require(timestamps.length == values.length, s"${timestamps.length} ts != ${values.length} values") - def rows: Iterator[RowReader] = new Iterator[RowReader] { + def rows(): RangeVectorCursor = new RangeVectorCursor { val row = new TransientRow() var n = 0 def hasNext: Boolean = n < timestamps.length @@ -372,5 +378,6 @@ final case class BufferRangeVector(key: RangeVectorKey, n += 1 row } + def close(): Unit = {} } } diff --git a/core/src/main/scala/filodb.core/query/RangeVectorCursor.scala b/core/src/main/scala/filodb.core/query/RangeVectorCursor.scala new file mode 100644 index 0000000000..dd552e3d4d --- /dev/null +++ b/core/src/main/scala/filodb.core/query/RangeVectorCursor.scala @@ -0,0 +1,42 @@ +package filodb.core.query + +import scala.collection.Iterator + +import filodb.memory.format.RowReader + +/** + * Please note this is not the ideal contract of cursor. + * Instead, it is a stop-gap implementation that gets us ability to + * release resources from a query. Earlier implementation purely on + * Iterators didnt help us nicely with that. The expectation is that + * moving to this trait will help us get compile time checks + * that force developer to care for "closing" the cursor before + * completing the query + */ +trait RangeVectorCursor extends Iterator[RowReader] with java.io.Closeable { self => + /** + * This method mut release all resources (example locks) acquired + * for the purpose of executing this query + */ + def close(): Unit + def mapRow(f: RowReader => RowReader): RangeVectorCursor = new RangeVectorCursor { + def hasNext = self.hasNext + def next() = f(self.next()) + def close(): Unit = self.close() + } +} + +class CustomCloseCursor(iter: Iterator[RowReader])(cl: => Unit) extends RangeVectorCursor { + override def close(): Unit = cl // invoke function + override def hasNext: Boolean = iter.hasNext + override def next(): RowReader = iter.next() +} + +object NoCloseCursor { + implicit class NoCloseCursor(iter: Iterator[RowReader]) extends RangeVectorCursor { + override def close(): Unit = {} + override def hasNext: Boolean = iter.hasNext + override def next(): RowReader = iter.next() + } +} + diff --git a/core/src/main/scala/filodb.core/store/ReadablePartition.scala b/core/src/main/scala/filodb.core/store/ReadablePartition.scala index ca72d582bb..593094ff1e 100644 --- a/core/src/main/scala/filodb.core/store/ReadablePartition.scala +++ b/core/src/main/scala/filodb.core/store/ReadablePartition.scala @@ -2,8 +2,8 @@ package filodb.core.store import filodb.core.Types.{ChunkID, ColumnId} import filodb.core.metadata.Schema -import filodb.core.query.PartitionTimeRangeReader -import filodb.memory.format.{BinaryVector, MemoryReader, RowReader, UnsafeUtils, VectorDataReader} +import filodb.core.query.{PartitionTimeRangeReader, RangeVectorCursor} +import filodb.memory.format.{BinaryVector, MemoryReader, UnsafeUtils, VectorDataReader} trait FiloPartition { @@ -116,10 +116,10 @@ trait ReadablePartition extends FiloPartition { * @param endTime ending timestamp, in milliseconds since Epoch * @param columnIDs the column IDs to query */ - final def timeRangeRows(startTime: Long, endTime: Long, columnIDs: Array[ColumnId]): Iterator[RowReader] = + final def timeRangeRows(startTime: Long, endTime: Long, columnIDs: Array[ColumnId]): RangeVectorCursor = new PartitionTimeRangeReader(this, startTime, endTime, infos(startTime, endTime), columnIDs) - final def timeRangeRows(method: ChunkScanMethod, columnIDs: Array[ColumnId]): Iterator[RowReader] = + final def timeRangeRows(method: ChunkScanMethod, columnIDs: Array[ColumnId]): RangeVectorCursor = new PartitionTimeRangeReader(this, method.startTime, method.endTime, infos(method), columnIDs) } diff --git a/core/src/test/scala/filodb.core/query/RangeVectorSpec.scala b/core/src/test/scala/filodb.core/query/RangeVectorSpec.scala index 94ba0e51f1..ae4bfa6531 100644 --- a/core/src/test/scala/filodb.core/query/RangeVectorSpec.scala +++ b/core/src/test/scala/filodb.core/query/RangeVectorSpec.scala @@ -3,7 +3,7 @@ package filodb.core.query import org.scalatest.{FunSpec, Matchers} import filodb.core.metadata.Column.ColumnType -import filodb.memory.format.{RowReader, SeqRowReader, ZeroCopyUTF8String} +import filodb.memory.format.{SeqRowReader, ZeroCopyUTF8String} class RangeVectorSpec extends FunSpec with Matchers { @@ -13,7 +13,8 @@ class RangeVectorSpec extends FunSpec with Matchers { val tuples = (numRawSamples until 0).by(-1).map(n => (now - n * reportingInterval, n.toDouble)) class TuplesRangeVector(inputTuples: Seq[(Long, Double)]) extends RangeVector { - override def rows: Iterator[RowReader] = inputTuples.map { t => + import NoCloseCursor._ + override def rows(): RangeVectorCursor = inputTuples.map { t => new SeqRowReader(Seq[Any](t._1, t._2)) }.iterator override def key: RangeVectorKey = new RangeVectorKey { diff --git a/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala b/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala index 2339ab6148..6638c7ce22 100644 --- a/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala +++ b/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala @@ -1,5 +1,7 @@ package filodb.query.exec +import scala.collection.mutable.ArrayBuffer + import com.typesafe.scalalogging.StrictLogging import monix.eval.Task import monix.reactive.Observable @@ -7,7 +9,7 @@ import scalaxy.loops._ import filodb.core.query._ import filodb.memory.format.ZeroCopyUTF8String -import filodb.query. _ +import filodb.query._ import filodb.query.exec.aggregator.RowAggregator /** @@ -119,6 +121,10 @@ final case class AggregatePresenter(aggrOp: AggregationOperator, */ object RangeVectorAggregator extends StrictLogging { + trait CloseableIterator[R] extends Iterator[R] { + def close(): Unit + } + /** * This method is the facade for map and reduce steps of the aggregation. * In the reduction-only (non-leaf) phases, skipMapPhase should be true. @@ -132,7 +138,7 @@ object RangeVectorAggregator extends StrictLogging { // now reduce each group and create one result range vector per group val groupedResult = mapReduceInternal(rvs, rowAgg, skipMapPhase, grouping) groupedResult.map { case (rvk, aggHolder) => - val rowIterator = aggHolder.map(_.toRowReader) + val rowIterator = new CustomCloseCursor(aggHolder.map(_.toRowReader))(aggHolder.close()) IteratorBackedRangeVector(rvk, rowIterator) } } @@ -151,12 +157,13 @@ object RangeVectorAggregator extends StrictLogging { private def mapReduceInternal(rvs: List[RangeVector], rowAgg: RowAggregator, skipMapPhase: Boolean, - grouping: RangeVector => RangeVectorKey): Map[RangeVectorKey, Iterator[rowAgg.AggHolderType]] = { + grouping: RangeVector => RangeVectorKey): + Map[RangeVectorKey, CloseableIterator[rowAgg.AggHolderType]] = { logger.trace(s"mapReduceInternal on ${rvs.size} RangeVectors...") var acc = rowAgg.zero val mapInto = rowAgg.newRowToMapInto rvs.groupBy(grouping).mapValues { rvs => - new Iterator[rowAgg.AggHolderType] { + new CloseableIterator[rowAgg.AggHolderType] { val itsAndKeys = rvs.map { rv => (rv.rows, rv.key) } def hasNext: Boolean = { // Dont use forAll since it short-circuits hasNext invocation @@ -175,6 +182,7 @@ object RangeVectorAggregator extends StrictLogging { } acc } + def close() = rvs.foreach(_.rows().close()) } } } @@ -193,6 +201,8 @@ object RangeVectorAggregator extends StrictLogging { // Can't use an Array here because rowAgg.AggHolderType does not have a ClassTag val accs = collection.mutable.ArrayBuffer.fill(outputLen)(rowAgg.zero) var count = 0 + // keeps track of all iters to close + val toClose = ArrayBuffer.empty[RangeVectorCursor] // FoldLeft means we create the source PeriodicMapper etc and process immediately. We can release locks right away // NOTE: ChunkedWindowIterator automatically releases locks after last window. So it should all just work. :) @@ -200,6 +210,7 @@ object RangeVectorAggregator extends StrictLogging { source.foldLeftF(accs) { case (_, rv) => count += 1 val rowIter = rv.rows + toClose += rowIter for { i <- 0 until outputLen optimized } { accs(i) = rowAgg.reduceAggregate(accs(i), rowIter.next) } @@ -210,6 +221,7 @@ object RangeVectorAggregator extends StrictLogging { source.foldLeftF(accs) { case (_, rv) => count += 1 val rowIter = rv.rows + toClose += rowIter for { i <- 0 until outputLen optimized } { val mapped = rowAgg.map(rv.key, rowIter.next, mapIntos(i)) accs(i) = rowAgg.reduceMappedRow(accs(i), mapped) @@ -220,7 +232,8 @@ object RangeVectorAggregator extends StrictLogging { aggObs.flatMap { _ => if (count > 0) { - Observable.now(IteratorBackedRangeVector(CustomRangeVectorKey.empty, accs.toIterator.map(_.toRowReader))) + val iter = new CustomCloseCursor(accs.toIterator.map(_.toRowReader))(toClose.foreach(_.close())) + Observable.now(IteratorBackedRangeVector(CustomRangeVectorKey.empty, iter)) } else { Observable.empty } diff --git a/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala b/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala index 59277a6676..11efb47024 100644 --- a/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala +++ b/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala @@ -142,8 +142,9 @@ final case class BinaryJoinExec(queryContext: QueryContext, CustomRangeVectorKey(result) } - private def binOp(lhsRows: Iterator[RowReader], rhsRows: Iterator[RowReader]): Iterator[RowReader] = { - new Iterator[RowReader] { + private def binOp(lhsRows: RangeVectorCursor, + rhsRows: RangeVectorCursor): RangeVectorCursor = { + new RangeVectorCursor { val cur = new TransientRow() val binFunc = BinaryOperatorFunction.factoryMethod(binaryOp) override def hasNext: Boolean = lhsRows.hasNext && rhsRows.hasNext @@ -153,6 +154,11 @@ final case class BinaryJoinExec(queryContext: QueryContext, cur.setValues(lhsRow.getLong(0), binFunc.calculate(lhsRow.getDouble(1), rhsRow.getDouble(1))) cur } + + override def close(): Unit = { + lhsRows.close() + rhsRows.close() + } } } diff --git a/query/src/main/scala/filodb/query/exec/ExecPlan.scala b/query/src/main/scala/filodb/query/exec/ExecPlan.scala index 9ed7f309d8..0c9230f339 100644 --- a/query/src/main/scala/filodb/query/exec/ExecPlan.scala +++ b/query/src/main/scala/filodb/query/exec/ExecPlan.scala @@ -158,7 +158,6 @@ trait ExecPlan extends QueryCommand { s"Try applying more filters or reduce time range.") srv } - .take(queryContext.sampleLimit) .toListL .map { r => val numBytes = builder.allContainers.map(_.numBytes).sum diff --git a/query/src/main/scala/filodb/query/exec/HistogramQuantileMapper.scala b/query/src/main/scala/filodb/query/exec/HistogramQuantileMapper.scala index 3386f36098..4928e2dd3e 100644 --- a/query/src/main/scala/filodb/query/exec/HistogramQuantileMapper.scala +++ b/query/src/main/scala/filodb/query/exec/HistogramQuantileMapper.scala @@ -75,7 +75,7 @@ final case class HistogramQuantileMapper(funcParams: Seq[FuncArgs]) extends Rang val buckets = sortedBucketRvs.map { b => Bucket(b._1, 0d) } // create the result iterator that lazily produces quantile for each timestamp - val quantileResult = new Iterator[RowReader] { + val quantileResult = new RangeVectorCursor { val row = new TransientRow() override def hasNext: Boolean = samples.forall(_.hasNext) override def next(): RowReader = { @@ -87,6 +87,7 @@ final case class HistogramQuantileMapper(funcParams: Seq[FuncArgs]) extends Rang row.value = histogramQuantile(quantile, buckets) row } + override def close(): Unit = rvs.foreach(_.rows().close()) } IteratorBackedRangeVector(histBuckets._1, quantileResult) } diff --git a/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala b/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala index eed647cfe8..0e68125d2d 100644 --- a/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala +++ b/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala @@ -52,6 +52,7 @@ trait MetadataDistConcatExec extends NonLeafExecPlan { } } } + import NoCloseCursor._ IteratorBackedRangeVector(new CustomRangeVectorKey(Map.empty), new UTF8MapIteratorRowReader(metadataResult.toIterator)) } @@ -86,6 +87,7 @@ final case class PartKeysExec(queryContext: QueryContext, case memStore: MemStore => val response = memStore.partKeysWithFilters(dataset, shard, filters, fetchFirstLastSampleTimes, end, start, queryContext.sampleLimit) + import NoCloseCursor._ Observable.now(IteratorBackedRangeVector( new CustomRangeVectorKey(Map.empty), UTF8MapIteratorRowReader(response))) case other => @@ -126,6 +128,7 @@ final case class LabelValuesExec(queryContext: QueryContext, case false => memStore.labelValuesWithFilters(dataset, shard, filters, columns, endMs, startMs, queryContext.sampleLimit) } + import NoCloseCursor._ Observable.now(IteratorBackedRangeVector(new CustomRangeVectorKey(Map.empty), new UTF8MapIteratorRowReader(response))) } else { diff --git a/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala b/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala index de3adb06c0..06b9043899 100644 --- a/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala +++ b/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala @@ -44,6 +44,7 @@ case class MetadataRemoteExec(queryEndpoint: String, val iteratorMap = data.map { r => r.map { v => (v._1.utf8, v._2.utf8) }} + import NoCloseCursor._ val rangeVector = IteratorBackedRangeVector(new CustomRangeVectorKey(Map.empty), new UTF8MapIteratorRowReader(iteratorMap.toIterator)) diff --git a/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala b/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala index cc0a86cc59..39132d4ec2 100644 --- a/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala +++ b/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala @@ -108,7 +108,7 @@ final case class PeriodicSamplesMapper(start: Long, override def key: RangeVectorKey = rv.key - override def rows: Iterator[RowReader] = rv.rows.map { r => + override def rows(): RangeVectorCursor = rv.rows.mapRow { r => row.setLong(0, r.getLong(0) + o) row.setDouble(1, r.getDouble(1)) row @@ -158,7 +158,7 @@ abstract class ChunkedWindowIterator[R <: MutableRowReader]( window: Long, rangeFunction: ChunkedRangeFunction[R], querySession: QuerySession) -extends Iterator[R] with StrictLogging { +extends RangeVectorCursor with StrictLogging { // Lazily open the iterator and obtain the lock. This allows one thread to create the // iterator, but the lock is owned by the thread actually performing the iteration. private lazy val windowIt = { @@ -170,6 +170,8 @@ extends Iterator[R] with StrictLogging { it } + def close: Unit = rv.rows().close() + def sampleToEmit: R override def hasNext: Boolean = windowIt.hasMoreWindows @@ -240,13 +242,13 @@ class QueueBasedWindow(q: IndexedArrayQueue[TransientRow]) extends Window { * Decorates a raw series iterator to apply a range vector function * on periodic time windows */ -class SlidingWindowIterator(raw: Iterator[RowReader], +class SlidingWindowIterator(raw: RangeVectorCursor, start: Long, step: Long, end: Long, window: Long, rangeFunction: RangeFunction, - queryConfig: QueryConfig) extends Iterator[TransientRow] { + queryConfig: QueryConfig) extends RangeVectorCursor { private val sampleToEmit = new TransientRow() private var curWindowEnd = start @@ -272,6 +274,8 @@ class SlidingWindowIterator(raw: Iterator[RowReader], // to avoid creation of object per sample, we use a pool val windowSamplesPool = new TransientRowPool() + override def close(): Unit = raw.close() + override def hasNext: Boolean = curWindowEnd <= end override def next(): TransientRow = { val curWindowStart = curWindowEnd - window @@ -342,7 +346,7 @@ class SlidingWindowIterator(raw: Iterator[RowReader], /**   * Converts the long value column to double.   */ -class LongToDoubleIterator(iter: Iterator[RowReader]) extends Iterator[TransientRow] { +class LongToDoubleIterator(iter: RangeVectorCursor) extends RangeVectorCursor { val sampleToEmit = new TransientRow() override final def hasNext: Boolean = iter.hasNext override final def next(): TransientRow = { @@ -351,6 +355,8 @@ class LongToDoubleIterator(iter: Iterator[RowReader]) extends Iterator[Transient sampleToEmit.setDouble(1, next.getLong(1).toDouble) sampleToEmit } + + override def close(): Unit = iter.close() } /** diff --git a/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala b/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala index 04ad3ae302..7aa50fcd9c 100644 --- a/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala +++ b/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala @@ -2,6 +2,10 @@ package filodb.query.exec import java.util.concurrent.TimeUnit +import scala.concurrent.Future +import scala.concurrent.duration._ +import scala.sys.ShutdownHookThread + import com.softwaremill.sttp.asynchttpclient.future.AsyncHttpClientFutureBackend import com.softwaremill.sttp.circe._ import com.typesafe.scalalogging.StrictLogging @@ -9,15 +13,11 @@ import kamon.Kamon import kamon.trace.Span import monix.eval.Task import monix.execution.Scheduler -import scala.concurrent.Future -import scala.concurrent.duration._ -import scala.sys.ShutdownHookThread import filodb.core.DatasetRef import filodb.core.metadata.Column.ColumnType import filodb.core.query._ import filodb.core.store.ChunkSource -import filodb.memory.format.RowReader import filodb.memory.format.ZeroCopyUTF8String._ import filodb.query._ @@ -124,7 +124,8 @@ case class PromQlRemoteExec(queryEndpoint: String, override def key: RangeVectorKey = CustomRangeVectorKey(r.metric.map (m => m._1.utf8 -> m._2.utf8)) - override def rows: Iterator[RowReader] = { + override def rows(): RangeVectorCursor = { + import NoCloseCursor._ samples.iterator.collect { case v: Sampl => row.setLong(0, v.timestamp * 1000) row.setDouble(1, v.value) diff --git a/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala b/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala index e68f1ae758..84cb709877 100644 --- a/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala +++ b/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala @@ -142,10 +142,11 @@ final case class InstantVectorFunctionMapper(function: InstantFunctionId, } } -private class DoubleInstantFuncIterator(rows: Iterator[RowReader], +private class DoubleInstantFuncIterator(rows: RangeVectorCursor, instantFunction: DoubleInstantFunction, scalar: Seq[ScalarRangeVector], - result: TransientRow = new TransientRow()) extends Iterator[RowReader] { + result: TransientRow = new TransientRow()) extends + RangeVectorCursor { final def hasNext: Boolean = rows.hasNext final def next(): RowReader = { val next = rows.next() @@ -155,12 +156,14 @@ private class DoubleInstantFuncIterator(rows: Iterator[RowReader], result.setValues(timestamp, newValue) result } + final def close(): Unit = rows.close() } -private class H2DoubleInstantFuncIterator(rows: Iterator[RowReader], +private class H2DoubleInstantFuncIterator(rows: RangeVectorCursor, instantFunction: HistToDoubleIFunction, scalar: Seq[ScalarRangeVector], - result: TransientRow = new TransientRow()) extends Iterator[RowReader] { + result: TransientRow = new TransientRow()) + extends RangeVectorCursor { final def hasNext: Boolean = rows.hasNext final def next(): RowReader = { val next = rows.next() @@ -169,12 +172,14 @@ private class H2DoubleInstantFuncIterator(rows: Iterator[RowReader], result.setValues(timestamp, newValue) result } + final def close(): Unit = rows.close() } -private class HD2DoubleInstantFuncIterator(rows: Iterator[RowReader], +private class HD2DoubleInstantFuncIterator(rows: RangeVectorCursor, instantFunction: HDToDoubleIFunction, scalar: Seq[ScalarRangeVector], - result: TransientRow = new TransientRow()) extends Iterator[RowReader] { + result: TransientRow = new TransientRow()) + extends RangeVectorCursor { final def hasNext: Boolean = rows.hasNext final def next(): RowReader = { val next = rows.next() @@ -184,6 +189,7 @@ private class HD2DoubleInstantFuncIterator(rows: Iterator[RowReader], result.setValues(timestamp, newValue) result } + final def close(): Unit = rows.close() } /** @@ -214,7 +220,7 @@ final case class ScalarOperationMapper(operator: BinaryOperator, private def evaluate(source: Observable[RangeVector], scalarRangeVector: ScalarRangeVector) = { source.map { rv => - val resultIterator: Iterator[RowReader] = new Iterator[RowReader]() { + val resultIterator: RangeVectorCursor = new RangeVectorCursor() { private val rows = rv.rows private val result = new TransientRow() @@ -230,6 +236,7 @@ final case class ScalarOperationMapper(operator: BinaryOperator, result.setValues(timestamp, newValue) result } + override def close(): Unit = rv.rows().close() } IteratorBackedRangeVector(rv.key, resultIterator) } @@ -334,7 +341,7 @@ final case class VectorFunctionMapper() extends RangeVectorTransformer { source.map { rv => new RangeVector { override def key: RangeVectorKey = rv.key - override def rows: Iterator[RowReader] = rv.rows + override def rows(): RangeVectorCursor = rv.rows } } } @@ -366,18 +373,20 @@ final case class AbsentFunctionMapper(columnFilter: Seq[ColumnFilter], rangePara } val nonNanTimestamps = source.foldLeftL(List[Long]())(addNonNanTimestamps) - val resultRv = nonNanTimestamps.map { - t => - val rowList = new ListBuffer[TransientRow]() - for (i <- rangeParams.startSecs to rangeParams.endSecs by rangeParams.stepSecs) { - if (!t.contains(i * 1000)) - rowList += new TransientRow(i * 1000, 1) - } - new RangeVector { - override def key: RangeVectorKey = if (rowList.isEmpty) CustomRangeVectorKey.empty else keysFromFilter - override def rows: Iterator[RowReader] = rowList.iterator - } + val resultRv = nonNanTimestamps.map { t => + val rowList = new ListBuffer[TransientRow]() + for (i <- rangeParams.startSecs to rangeParams.endSecs by rangeParams.stepSecs) { + if (!t.contains(i * 1000)) + rowList += new TransientRow(i * 1000, 1) + } + new RangeVector { + override def key: RangeVectorKey = if (rowList.isEmpty) CustomRangeVectorKey.empty else keysFromFilter + override def rows(): RangeVectorCursor = { + import NoCloseCursor._ + rowList.iterator } + } + } Observable.fromTask(resultRv) } diff --git a/query/src/main/scala/filodb/query/exec/StitchRvsExec.scala b/query/src/main/scala/filodb/query/exec/StitchRvsExec.scala index f68912e911..024ec93feb 100644 --- a/query/src/main/scala/filodb/query/exec/StitchRvsExec.scala +++ b/query/src/main/scala/filodb/query/exec/StitchRvsExec.scala @@ -13,10 +13,10 @@ import filodb.query.Query.qLogger object StitchRvsExec { - def merge(vectors: Seq[Iterator[RowReader]]): Iterator[RowReader] = { + def merge(vectors: Seq[RangeVectorCursor]): RangeVectorCursor = { // This is an n-way merge without using a heap. // Heap is not used since n is expected to be very small (almost always just 1 or 2) - new Iterator[RowReader] { + new RangeVectorCursor { val bVectors = vectors.map(_.buffered) val mins = new mutable.ArrayBuffer[BufferedIterator[RowReader]](2) val noResult = new TransientRow(0, 0) @@ -50,6 +50,8 @@ object StitchRvsExec { noResult } } + + override def close(): Unit = vectors.foreach(_.close()) } } } 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 a51799f6d1..355a4fd351 100644 --- a/query/src/main/scala/filodb/query/exec/aggregator/CountValuesRowAggregator.scala +++ b/query/src/main/scala/filodb/query/exec/aggregator/CountValuesRowAggregator.scala @@ -107,6 +107,7 @@ class CountValuesRowAggregator(label: String, limit: Int = 1000) extends RowAggr } } finally { + aggRangeVector.rows.close() ChunkMap.releaseAllSharedLocks() } resRvs.map { case (key, builder) => diff --git a/query/src/main/scala/filodb/query/exec/aggregator/QuantileRowAggregator.scala b/query/src/main/scala/filodb/query/exec/aggregator/QuantileRowAggregator.scala index dc4dc706c8..df7fb687ee 100644 --- a/query/src/main/scala/filodb/query/exec/aggregator/QuantileRowAggregator.scala +++ b/query/src/main/scala/filodb/query/exec/aggregator/QuantileRowAggregator.scala @@ -65,7 +65,7 @@ class QuantileRowAggregator(q: Double) extends RowAggregator { def present(aggRangeVector: RangeVector, limit: Int): Seq[RangeVector] = { val mutRow = new TransientRow() - val result = aggRangeVector.rows.map { r => + val result = aggRangeVector.rows.mapRow { r => val qVal = ArrayDigest.fromBytes(r.getBuffer(1)).quantile(q) mutRow.setValues(r.getLong(0), qVal) mutRow 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 2eaa39c3c5..0be6a0ff01 100644 --- a/query/src/main/scala/filodb/query/exec/aggregator/TopBottomKRowAggregator.scala +++ b/query/src/main/scala/filodb/query/exec/aggregator/TopBottomKRowAggregator.scala @@ -107,6 +107,7 @@ class TopBottomKRowAggregator(k: Int, bottomK: Boolean) extends RowAggregator { } } } finally { + aggRangeVector.rows().close() ChunkMap.releaseAllSharedLocks() } resRvs.map { case (key, builder) => diff --git a/query/src/test/scala/filodb/query/ResultTypesSpec.scala b/query/src/test/scala/filodb/query/ResultTypesSpec.scala index 7c4bd4edd1..c0a90adb8e 100644 --- a/query/src/test/scala/filodb/query/ResultTypesSpec.scala +++ b/query/src/test/scala/filodb/query/ResultTypesSpec.scala @@ -5,7 +5,7 @@ import org.scalatest.concurrent.ScalaFutures import filodb.core.metadata.Column.ColumnType import filodb.core.query._ -import filodb.memory.format.{RowReader, ZeroCopyUTF8String} +import filodb.memory.format.ZeroCopyUTF8String class ResultTypesSpec extends FunSpec with Matchers with ScalaFutures { @@ -21,7 +21,8 @@ class ResultTypesSpec extends FunSpec with Matchers with ScalaFutures { override def key: RangeVectorKey = ignoreKey - override def rows: Iterator[RowReader] = Seq( + import NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 3.3d), new TransientRow(2L, 5.1d)).iterator @@ -38,7 +39,8 @@ class ResultTypesSpec extends FunSpec with Matchers with ScalaFutures { override def key: RangeVectorKey = ignoreKey - override def rows: Iterator[RowReader] = Seq( + import NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 3.3d)).toIterator override def numRows: Option[Int] = Option(rows.size) @@ -49,7 +51,8 @@ class ResultTypesSpec extends FunSpec with Matchers with ScalaFutures { override def key: RangeVectorKey = ignoreKey - override def rows: Iterator[RowReader] = Seq( + import NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 9.4d)).toIterator override def numRows: Option[Int] = Option(rows.size) @@ -65,7 +68,8 @@ class ResultTypesSpec extends FunSpec with Matchers with ScalaFutures { override def key: RangeVectorKey = CustomRangeVectorKey(Map.empty) - override def rows: Iterator[RowReader] = Seq( + import NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 3.3d)).toIterator override def numRows: Option[Int] = Option(rows.size) diff --git a/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala b/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala index 54be905e64..80c1e2f921 100644 --- a/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala +++ b/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala @@ -2,14 +2,15 @@ package filodb.query.exec import scala.annotation.tailrec import scala.util.Random + import com.tdunning.math.stats.TDigest import monix.execution.Scheduler.Implicits.global import monix.reactive.Observable import org.scalatest.concurrent.ScalaFutures + import filodb.core.{MachineMetricsData => MMD} import filodb.core.metadata.Column.ColumnType import filodb.core.query._ -import filodb.memory.format.RowReader import filodb.memory.format.ZeroCopyUTF8String._ import filodb.query.AggregationOperator import filodb.query.exec.aggregator.RowAggregator @@ -32,11 +33,12 @@ class AggrOverRangeVectorsSpec extends RawDataWindowingSpec with ScalaFutures { def noGrouping(rv: RangeVector): RangeVectorKey = noKey val samples: Array[RangeVector] = Array.fill(100)(new RangeVector { + import NoCloseCursor._ val data = Stream.from(0).map { n=> new TransientRow(n.toLong, rand.nextDouble()) }.take(20) override def key: RangeVectorKey = ignoreKey - override def rows: Iterator[RowReader] = data.iterator + override def rows(): RangeVectorCursor = data.iterator }) // Sum @@ -295,8 +297,9 @@ class AggrOverRangeVectorsSpec extends RawDataWindowingSpec with ScalaFutures { private def toRv(samples: Seq[(Long, Double)]): RangeVector = { new RangeVector { + import NoCloseCursor._ override def key: RangeVectorKey = ignoreKey - override def rows: Iterator[RowReader] = samples.map(r => new TransientRow(r._1, r._2)).iterator + override def rows(): RangeVectorCursor = samples.map(r => new TransientRow(r._1, r._2)).iterator } } diff --git a/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala b/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala index 6c6dec9928..ce328bb810 100644 --- a/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala @@ -12,7 +12,6 @@ import org.scalatest.concurrent.ScalaFutures import filodb.core.metadata.Column.ColumnType import filodb.core.query._ -import filodb.memory.format.RowReader import filodb.memory.format.ZeroCopyUTF8String._ import filodb.query._ @@ -44,7 +43,8 @@ class BinaryJoinExecSpec extends FunSpec with Matchers with ScalaFutures { Map("__name__".utf8 -> s"someMetricLhs".utf8, "tag1".utf8 -> s"tag1-$i".utf8, "tag2".utf8 -> s"tag2-$i".utf8)) - val rows: Iterator[RowReader] = data(i).iterator + import NoCloseCursor._ + val rows: RangeVectorCursor = data(i).iterator } } @@ -54,7 +54,8 @@ class BinaryJoinExecSpec extends FunSpec with Matchers with ScalaFutures { Map("__name__".utf8 -> s"someMetricRhs".utf8, "tag1".utf8 -> samplesLhs(i).key.labelValues("tag1".utf8), "tag2".utf8 -> samplesLhs(i).key.labelValues("tag2".utf8))) - val rows: Iterator[RowReader] = data(i).iterator + import NoCloseCursor._ + val rows: RangeVectorCursor = data(i).iterator } } @@ -65,7 +66,8 @@ class BinaryJoinExecSpec extends FunSpec with Matchers with ScalaFutures { "tag1".utf8 -> s"tag1-$i".utf8, "tag2".utf8 -> s"tag2-1".utf8, "job".utf8 -> s"somejob".utf8)) - val rows: Iterator[RowReader] = data(i).iterator + import NoCloseCursor._ + val rows: RangeVectorCursor = data(i).iterator } } @@ -75,7 +77,8 @@ class BinaryJoinExecSpec extends FunSpec with Matchers with ScalaFutures { Map("__name__".utf8 -> s"someMetricRhs".utf8, "tag1".utf8 -> s"tag1-$i".utf8, "job".utf8 -> s"somejob".utf8)) - val rows: Iterator[RowReader] = data(i).iterator + import NoCloseCursor._ + val rows: RangeVectorCursor = data(i).iterator } } @@ -145,7 +148,8 @@ class BinaryJoinExecSpec extends FunSpec with Matchers with ScalaFutures { Map("__name__".utf8 -> s"someMetricRhs".utf8, "tag1".utf8 -> "tag1-uniqueValue".utf8, "tag2".utf8 -> samplesLhs(2).key.labelValues("tag2".utf8))) // duplicate value - val rows: Iterator[RowReader] = data(2).iterator + import NoCloseCursor._ + val rows: RangeVectorCursor = data(2).iterator } val samplesRhs2 = scala.util.Random.shuffle(duplicate +: samplesRhs.toList) // they may come out of order @@ -175,7 +179,8 @@ class BinaryJoinExecSpec extends FunSpec with Matchers with ScalaFutures { Map("__name__".utf8 -> s"someMetricRhs".utf8, "tag1".utf8 -> "tag1-uniqueValue".utf8, "tag2".utf8 -> samplesLhs(2).key.labelValues("tag2".utf8))) // duplicate value - val rows: Iterator[RowReader] = data(2).iterator + import NoCloseCursor._ + val rows: RangeVectorCursor = data(2).iterator } val samplesLhs2 = scala.util.Random.shuffle(duplicate +: samplesLhs.toList) // they may come out of order @@ -270,7 +275,8 @@ class BinaryJoinExecSpec extends FunSpec with Matchers with ScalaFutures { Map("metric".utf8 -> s"someMetricLhs".utf8, "tag1".utf8 -> s"tag1-$i".utf8, "tag2".utf8 -> s"tag2-$i".utf8)) - val rows: Iterator[RowReader] = data(i).iterator + import NoCloseCursor._ + val rows: RangeVectorCursor = data(i).iterator } } @@ -280,7 +286,8 @@ class BinaryJoinExecSpec extends FunSpec with Matchers with ScalaFutures { Map("metric".utf8 -> s"someMetricRhs".utf8, "tag1".utf8 -> samplesLhs(i).key.labelValues("tag1".utf8), "tag2".utf8 -> samplesLhs(i).key.labelValues("tag2".utf8))) - val rows: Iterator[RowReader] = data(i).iterator + import NoCloseCursor._ + val rows: RangeVectorCursor = data(i).iterator } } @@ -312,7 +319,8 @@ class BinaryJoinExecSpec extends FunSpec with Matchers with ScalaFutures { Map("metric".utf8 -> s"someMetricLhs".utf8, "tag1".utf8 -> s"tag1-$i".utf8, "tag2".utf8 -> s"tag2-$i".utf8)) - val rows: Iterator[RowReader] = data(i).iterator + import NoCloseCursor._ + val rows: RangeVectorCursor = data(i).iterator } } @@ -322,7 +330,8 @@ class BinaryJoinExecSpec extends FunSpec with Matchers with ScalaFutures { Map("metric".utf8 -> s"someMetricRhs".utf8, "tag1".utf8 -> samplesLhs(i).key.labelValues("tag1".utf8), "tag2".utf8 -> samplesLhs(i).key.labelValues("tag2".utf8))) - val rows: Iterator[RowReader] = data(i).iterator + import NoCloseCursor._ + val rows: RangeVectorCursor = data(i).iterator } } diff --git a/query/src/test/scala/filodb/query/exec/BinaryJoinGroupingSpec.scala b/query/src/test/scala/filodb/query/exec/BinaryJoinGroupingSpec.scala index e64dc08436..3573dc11ad 100644 --- a/query/src/test/scala/filodb/query/exec/BinaryJoinGroupingSpec.scala +++ b/query/src/test/scala/filodb/query/exec/BinaryJoinGroupingSpec.scala @@ -12,7 +12,7 @@ import org.scalatest.concurrent.ScalaFutures import filodb.core.metadata.Column.ColumnType import filodb.core.query._ -import filodb.memory.format.{RowReader, ZeroCopyUTF8String} +import filodb.memory.format.ZeroCopyUTF8String import filodb.memory.format.ZeroCopyUTF8String._ import filodb.query._ import filodb.query.exec.aggregator.RowAggregator @@ -47,7 +47,8 @@ class BinaryJoinGroupingSpec extends FunSpec with Matchers with ScalaFutures { "mode".utf8 -> s"idle".utf8) ) - override def rows: Iterator[RowReader] = Seq( + import NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 3)).iterator }, new RangeVector { @@ -58,7 +59,8 @@ class BinaryJoinGroupingSpec extends FunSpec with Matchers with ScalaFutures { "mode".utf8 -> s"user".utf8) ) - override def rows: Iterator[RowReader] = Seq( + import NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 1)).iterator }, new RangeVector { @@ -69,7 +71,8 @@ class BinaryJoinGroupingSpec extends FunSpec with Matchers with ScalaFutures { "mode".utf8 -> s"idle".utf8) ) - override def rows: Iterator[RowReader] = Seq( + import NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 8)).iterator }, new RangeVector { @@ -80,7 +83,8 @@ class BinaryJoinGroupingSpec extends FunSpec with Matchers with ScalaFutures { "mode".utf8 -> s"user".utf8) ) - override def rows: Iterator[RowReader] = Seq( + import NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 2)).iterator } ) @@ -93,7 +97,8 @@ class BinaryJoinGroupingSpec extends FunSpec with Matchers with ScalaFutures { "role".utf8 -> s"prometheus".utf8) ) - override def rows: Iterator[RowReader] = Seq( + import NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 1)).iterator } ) @@ -106,7 +111,8 @@ class BinaryJoinGroupingSpec extends FunSpec with Matchers with ScalaFutures { "job".utf8 -> "node".utf8 )) - override def rows: Iterator[RowReader] = Seq( + import NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 2)).iterator } ) @@ -320,7 +326,8 @@ class BinaryJoinGroupingSpec extends FunSpec with Matchers with ScalaFutures { "mode".utf8 -> s"idle".utf8) ) - override def rows: Iterator[RowReader] = Seq( + import NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 3)).iterator }, new RangeVector { @@ -331,7 +338,8 @@ class BinaryJoinGroupingSpec extends FunSpec with Matchers with ScalaFutures { "mode".utf8 -> s"user".utf8) ) - override def rows: Iterator[RowReader] = Seq( + import NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 1)).iterator }) @@ -344,7 +352,8 @@ class BinaryJoinGroupingSpec extends FunSpec with Matchers with ScalaFutures { "role".utf8 -> s"prometheus".utf8) ) - override def rows: Iterator[RowReader] = Seq( + import NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 1)).iterator } ) diff --git a/query/src/test/scala/filodb/query/exec/BinaryJoinSetOperatorSpec.scala b/query/src/test/scala/filodb/query/exec/BinaryJoinSetOperatorSpec.scala index 83019579a0..bf5710fe32 100644 --- a/query/src/test/scala/filodb/query/exec/BinaryJoinSetOperatorSpec.scala +++ b/query/src/test/scala/filodb/query/exec/BinaryJoinSetOperatorSpec.scala @@ -7,13 +7,13 @@ import monix.eval.Task import monix.execution.Scheduler import monix.execution.Scheduler.Implicits.global import monix.reactive.Observable -import org.scalatest.concurrent.ScalaFutures import org.scalatest.{FunSpec, Matchers} +import org.scalatest.concurrent.ScalaFutures import filodb.core.MetricsTestData import filodb.core.metadata.Column.ColumnType import filodb.core.query._ -import filodb.memory.format.{RowReader, ZeroCopyUTF8String} +import filodb.memory.format.ZeroCopyUTF8String import filodb.memory.format.ZeroCopyUTF8String._ import filodb.query._ @@ -51,7 +51,8 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures "group".utf8 -> s"production".utf8) ) - override def rows: Iterator[RowReader] = Seq( + import NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 100)).iterator }, new RangeVector { @@ -62,7 +63,8 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures "group".utf8 -> s"production".utf8) ) - override def rows: Iterator[RowReader] = Seq( + import NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 200)).iterator }, new RangeVector { @@ -73,7 +75,8 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures "group".utf8 -> s"canary".utf8) ) - override def rows: Iterator[RowReader] = Seq( + import NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 300)).iterator }, new RangeVector { @@ -84,7 +87,8 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures "group".utf8 -> s"canary".utf8) ) - override def rows: Iterator[RowReader] = Seq( + import NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 400)).iterator }, new RangeVector { @@ -95,7 +99,8 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures "group".utf8 -> s"production".utf8) ) - override def rows: Iterator[RowReader] = Seq( + import NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 500)).iterator }, new RangeVector { @@ -106,7 +111,8 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures "group".utf8 -> s"production".utf8) ) - override def rows: Iterator[RowReader] = Seq( + import NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 600)).iterator }, new RangeVector { @@ -117,7 +123,8 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures "group".utf8 -> s"canary".utf8) ) - override def rows: Iterator[RowReader] = Seq( + import NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 700)).iterator }, new RangeVector { @@ -128,7 +135,8 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures "group".utf8 -> s"canary".utf8) ) - override def rows: Iterator[RowReader] = Seq( + import NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 800)).iterator } ) @@ -136,7 +144,8 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures new RangeVector { override def key: RangeVectorKey = noKey - override def rows: Iterator[RowReader] = Seq( + import NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 1)).iterator } ) @@ -148,7 +157,8 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures "l".utf8 -> "x".utf8) ) - override def rows: Iterator[RowReader] = Seq( + import NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 100)).iterator }, new RangeVector { @@ -157,7 +167,8 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures "l".utf8 -> "y".utf8) ) - override def rows: Iterator[RowReader] = Seq( + import NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 200)).iterator } ) diff --git a/query/src/test/scala/filodb/query/exec/HistToPromSeriesMapperSpec.scala b/query/src/test/scala/filodb/query/exec/HistToPromSeriesMapperSpec.scala index 0b9fcaca99..a57379d24c 100644 --- a/query/src/test/scala/filodb/query/exec/HistToPromSeriesMapperSpec.scala +++ b/query/src/test/scala/filodb/query/exec/HistToPromSeriesMapperSpec.scala @@ -28,6 +28,7 @@ class HistToPromSeriesMapperSpec extends FunSpec with Matchers with ScalaFutures ColumnInfo("value", ColumnType.HistogramColumn)), 1) it("should convert single schema histogram to appropriate Prom bucket time series") { + import NoCloseCursor._ val rv = IteratorBackedRangeVector(rvKey, rows.toIterator) val mapper = HistToPromSeriesMapper(MMD.histDataset.schema.partition) @@ -63,6 +64,7 @@ class HistToPromSeriesMapperSpec extends FunSpec with Matchers with ScalaFutures val tenRows = tenBTimes.zip(tenBHists).map { case (t, h) => new TransientHistRow(t, h) } it("should convert multiple schema histograms to Prom bucket time series") { + import filodb.core.query.NoCloseCursor._ val rv = IteratorBackedRangeVector(rvKey, (rows ++ tenRows).toIterator) val mapper = HistToPromSeriesMapper(MMD.histDataset.schema.partition) diff --git a/query/src/test/scala/filodb/query/exec/HistogramQuantileMapperSpec.scala b/query/src/test/scala/filodb/query/exec/HistogramQuantileMapperSpec.scala index 52810a192a..289ba2c9a6 100644 --- a/query/src/test/scala/filodb/query/exec/HistogramQuantileMapperSpec.scala +++ b/query/src/test/scala/filodb/query/exec/HistogramQuantileMapperSpec.scala @@ -71,6 +71,7 @@ class HistogramQuantileMapperSpec extends FunSpec with Matchers with ScalaFuture it ("should calculate histogram_quantile correctly") { val histRvs = bucketValues.zipWithIndex.map { case (rv, i) => + import NoCloseCursor._ IteratorBackedRangeVector(histBuckets1(i), rv.map(s => new TransientRow(s._1, s._2.toDouble)).toIterator) } @@ -79,6 +80,7 @@ class HistogramQuantileMapperSpec extends FunSpec with Matchers with ScalaFuture } it ("should calculate histogram_quantile correctly for multiple histograms") { + import NoCloseCursor._ val histRvs = bucketValues.zipWithIndex.map { case (rv, i) => IteratorBackedRangeVector(histBuckets1(i), rv.map(s => new TransientRow(s._1, s._2.toDouble)).toIterator) } ++ bucketValues.zipWithIndex.map { case (rv, i) => @@ -90,6 +92,7 @@ class HistogramQuantileMapperSpec extends FunSpec with Matchers with ScalaFuture } it ("should sort the buckets to calculate histogram_quantile correctly ") { + import NoCloseCursor._ val histRvs = bucketValues.zipWithIndex.map { case (rv, i) => IteratorBackedRangeVector(histBuckets1(i), rv.map(s => new TransientRow(s._1, s._2.toDouble)).toIterator) } @@ -111,6 +114,7 @@ class HistogramQuantileMapperSpec extends FunSpec with Matchers with ScalaFuture Array[(Int, Double)]( (10, 34), (20, 42), (30, 46), (40, Double.NaN) ), Array[(Int, Double)]( (10, 35), (20, 45), (30, 46), (40, 89) ) ).zipWithIndex.map { case (rv, i) => + import NoCloseCursor._ IteratorBackedRangeVector(histBuckets1(i), rv.map(s => new TransientRow(s._1, s._2)).toIterator) } diff --git a/query/src/test/scala/filodb/query/exec/LastSampleFunctionSpec.scala b/query/src/test/scala/filodb/query/exec/LastSampleFunctionSpec.scala index 38f53f6ab0..9db822df28 100644 --- a/query/src/test/scala/filodb/query/exec/LastSampleFunctionSpec.scala +++ b/query/src/test/scala/filodb/query/exec/LastSampleFunctionSpec.scala @@ -2,10 +2,10 @@ package filodb.query.exec import java.util.concurrent.ThreadLocalRandom -import filodb.core.query.TransientRow - import scala.concurrent.duration._ import scala.util.Random + +import filodb.core.query.{RangeVectorCursor, TransientRow} import filodb.query.exec.rangefn.{LastSampleChunkedFunctionD, LastSampleFunction, RawDataWindowingSpec} class LastSampleFunctionSpec extends RawDataWindowingSpec { @@ -60,8 +60,8 @@ class LastSampleFunctionSpec extends RawDataWindowingSpec { it ("should return NaN when no reported samples for more than 5 minutes - static samples") { // note std dev for interval between reported samples is 5 mins - val samplesWithLongGap = Seq((59725569L,1.524759725569E12), (60038121L,1.524760038121E12), - (60370409L,1.524760370409E12), (60679268L,1.524760679268E12), (60988895L,1.524760988895E12)) + val samplesWithLongGap = Seq((59725569L, 1.524759725569E12), (60038121L, 1.524760038121E12), + (60370409L, 1.524760370409E12), (60679268L, 1.524760679268E12), (60988895L, 1.524760988895E12)) val rvWithLongGap = timeValueRV(samplesWithLongGap) val start = 60330762L val end = 63030762L @@ -132,7 +132,7 @@ class LastSampleFunctionSpec extends RawDataWindowingSpec { } def validateLastSamples(input: Seq[(Long, Double)], - output: Iterator[TransientRow], + output: RangeVectorCursor, start: Long, end: Long, step: Int): Unit = { @@ -141,12 +141,12 @@ class LastSampleFunctionSpec extends RawDataWindowingSpec { var cur = start while (cur <= end) { val observed = output.next() - observed.timestamp shouldEqual cur + observed.getLong(0) shouldEqual cur val expected = validationMap.floorEntry(cur) if (expected == null || cur - expected.getKey > 5.minutes.toMillis) { - observed.value.isNaN shouldEqual true + observed.getDouble(1).isNaN shouldEqual true } else { - observed.value shouldEqual expected.getValue + observed.getDouble(1) shouldEqual expected.getValue } cur = cur + step } diff --git a/query/src/test/scala/filodb/query/exec/StitchRvsExecSpec.scala b/query/src/test/scala/filodb/query/exec/StitchRvsExecSpec.scala index 9f4818bf35..21672764a1 100644 --- a/query/src/test/scala/filodb/query/exec/StitchRvsExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/StitchRvsExecSpec.scala @@ -1,10 +1,12 @@ package filodb.query.exec -import filodb.core.query.{ColumnInfo, QueryContext, ResultSchema, TransientRow} - import scala.annotation.tailrec + import org.scalatest.{FunSpec, Matchers} + import filodb.core.metadata.Column.ColumnType.{DoubleColumn, TimestampColumn} +import filodb.core.query.{ColumnInfo, QueryContext, ResultSchema, TransientRow} +import filodb.core.query.NoCloseCursor.NoCloseCursor import filodb.memory.format.UnsafeUtils import filodb.query.QueryResult @@ -167,7 +169,9 @@ class StitchRvsExecSpec extends FunSpec with Matchers { } def mergeAndValidate(rvs: Seq[Seq[(Long, Double)]], expected: Seq[(Long, Double)]): Unit = { - val inputSeq = rvs.map { rows => rows.iterator.map(r => new TransientRow(r._1, r._2)) } + val inputSeq = rvs.map { rows => + new NoCloseCursor(rows.iterator.map(r => new TransientRow(r._1, r._2))) + } val result = StitchRvsExec.merge(inputSeq).map(r => (r.getLong(0), r.getDouble(1))) compareIter(result, expected.toIterator) } diff --git a/query/src/test/scala/filodb/query/exec/WindowIteratorSpec.scala b/query/src/test/scala/filodb/query/exec/WindowIteratorSpec.scala index cd3b1f809a..92019388d9 100644 --- a/query/src/test/scala/filodb/query/exec/WindowIteratorSpec.scala +++ b/query/src/test/scala/filodb/query/exec/WindowIteratorSpec.scala @@ -137,13 +137,14 @@ class WindowIteratorSpec extends RawDataWindowingSpec { it ("should ignore out of order samples for RateFunction") { val rawRows = counterSamples.map(s => new TransientRow(s._1, s._2)) + import filodb.core.query.NoCloseCursor._ val slidingWinIterator = new SlidingWindowIterator(rawRows.iterator, 1538416154000L, 20000, 1538416649000L, 20000, RangeFunction(tsResSchema, Some(InternalRangeFunction.Rate), ColumnType.DoubleColumn, queryConfig, useChunked = false).asSliding, queryConfig) slidingWinIterator.foreach{ v => // if out of order samples are not removed, counter correction causes rate to spike up to very high value - v.value should be < 10000d + v.getDouble(1) should be < 10000d } } @@ -164,10 +165,11 @@ class WindowIteratorSpec extends RawDataWindowingSpec { val start = 50L val end = 1000L val step = 5 + import filodb.core.query.NoCloseCursor._ val slidingWinIterator = new SlidingWindowIterator(rawRows.iterator, start, step, end, 0, RangeFunction(tsResSchema, None, ColumnType.DoubleColumn, queryConfig, useChunked = false).asSliding, queryConfig) - val result = slidingWinIterator.map(v => (v.timestamp, v.value)).toSeq + val result = slidingWinIterator.map(v => (v.getLong(0), v.getDouble(1))).toSeq result.map(_._1) shouldEqual (start to end).by(step) result.foreach{ v => v._2 should not equal 698713d @@ -234,12 +236,13 @@ class WindowIteratorSpec extends RawDataWindowingSpec { 1548191556000L -> 1.0357142857142858 ) val rawRows = samples.map(s => new TransientRow(s._1, s._2)) + import filodb.core.query.NoCloseCursor._ val slidingWinIterator = new SlidingWindowIterator(rawRows.iterator, 1548191496000L, 15000, 1548191796000L, 300000, RangeFunction(tsResSchema, Some(InternalRangeFunction.Rate), ColumnType.DoubleColumn, queryConfig, useChunked = false).asSliding, queryConfig) slidingWinIterator.foreach { v => - windowResults.find(a => a._1 == v.timestamp).foreach(b => v.value shouldEqual b._2 +- 0.0000000001) + windowResults.find(a => a._1 == v.getLong(0)).foreach(b => v.getDouble(1) shouldEqual b._2 +- 0.0000000001) } val rv = timeValueRV(samples) @@ -247,7 +250,7 @@ class WindowIteratorSpec extends RawDataWindowingSpec { RangeFunction(tsResSchema, Some(Rate), ColumnType.DoubleColumn, queryConfig, useChunked = true).asChunkedD, querySession) chunkedIt.foreach { v => - windowResults.find(a => a._1 == v.timestamp).foreach(b => v.value shouldEqual b._2 +- 0.0000000001) + windowResults.find(a => a._1 == v.getLong(0)).foreach(b => v.getDouble(1) shouldEqual b._2 +- 0.0000000001) } } diff --git a/query/src/test/scala/filodb/query/exec/rangefn/AbsentFunctionSpec.scala b/query/src/test/scala/filodb/query/exec/rangefn/AbsentFunctionSpec.scala index 61c2381564..c90a837ab7 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/AbsentFunctionSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/AbsentFunctionSpec.scala @@ -11,7 +11,7 @@ import filodb.core.MetricsTestData import filodb.core.query._ import filodb.core.query.Filter.{Equals, NotEqualsRegex} import filodb.memory.data.ChunkMap -import filodb.memory.format.{RowReader, ZeroCopyUTF8String} +import filodb.memory.format.ZeroCopyUTF8String import filodb.query.exec class AbsentFunctionSpec extends FunSpec with Matchers with ScalaFutures with BeforeAndAfter { @@ -40,13 +40,15 @@ class AbsentFunctionSpec extends FunSpec with Matchers with ScalaFutures with Be new RangeVector { override def key: RangeVectorKey = testKey1 - override def rows: Iterator[RowReader] = Seq( + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1000L, 1d)).iterator }, new RangeVector { override def key: RangeVectorKey = testKey2 - override def rows: Iterator[RowReader] = Seq( + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1000L, 5d)).iterator }) @@ -54,7 +56,8 @@ class AbsentFunctionSpec extends FunSpec with Matchers with ScalaFutures with Be new RangeVector { override def key: RangeVectorKey = testKey1 - override def rows: Iterator[RowReader] = Seq( + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1000L, Double.NaN), new TransientRow(2000L, 1d), new TransientRow(3000L, Double.NaN)).iterator @@ -62,7 +65,8 @@ class AbsentFunctionSpec extends FunSpec with Matchers with ScalaFutures with Be new RangeVector { override def key: RangeVectorKey = testKey2 - override def rows: Iterator[RowReader] = Seq( + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1000L, 5d), new TransientRow(2000L, Double.NaN), new TransientRow(3000L, Double.NaN)).iterator diff --git a/query/src/test/scala/filodb/query/exec/rangefn/BinaryOperatorSpec.scala b/query/src/test/scala/filodb/query/exec/rangefn/BinaryOperatorSpec.scala index dffffc4b47..8e9774b1b6 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/BinaryOperatorSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/BinaryOperatorSpec.scala @@ -1,14 +1,16 @@ package filodb.query.exec.rangefn import scala.util.Random + import com.typesafe.config.{Config, ConfigFactory} import monix.execution.Scheduler.Implicits.global import monix.reactive.Observable import org.scalatest.{FunSpec, Matchers} import org.scalatest.concurrent.ScalaFutures + import filodb.core.MetricsTestData import filodb.core.query._ -import filodb.memory.format.{RowReader, ZeroCopyUTF8String} +import filodb.memory.format.ZeroCopyUTF8String import filodb.query._ import filodb.query.exec.{StaticFuncArgs, TimeFuncArgs} @@ -21,13 +23,15 @@ class BinaryOperatorSpec extends FunSpec with Matchers with ScalaFutures { val sampleBase: Array[RangeVector] = Array( new RangeVector { override def key: RangeVectorKey = ignoreKey - override def rows: Iterator[RowReader] = Seq( + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 3.3d), new TransientRow(2L, 5.1d)).iterator }, new RangeVector { override def key: RangeVectorKey = ignoreKey - override def rows: Iterator[RowReader] = Seq( + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(3L, 3239.3423d), new TransientRow(4L, 94935.1523d)).iterator }) @@ -49,7 +53,8 @@ class BinaryOperatorSpec extends FunSpec with Matchers with ScalaFutures { override def key: RangeVectorKey = ignoreKey - override def rows: Iterator[RowReader] = data.iterator + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = data.iterator }) fireBinaryOperatorTests(samples, scalar) fireComparatorOperatorTests(samples, scalar) @@ -63,19 +68,22 @@ class BinaryOperatorSpec extends FunSpec with Matchers with ScalaFutures { val samples: Array[RangeVector] = Array( new RangeVector { override def key: RangeVectorKey = ignoreKey - override def rows: Iterator[RowReader] = Seq( + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, Double.NaN), new TransientRow(2L, 5.6d)).iterator }, new RangeVector { override def key: RangeVectorKey = ignoreKey - override def rows: Iterator[RowReader] = Seq( + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 4.6d), new TransientRow(2L, 4.4d)).iterator }, new RangeVector { override def key: RangeVectorKey = ignoreKey - override def rows: Iterator[RowReader] = Seq( + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 0d), new TransientRow(2L, 5.4d)).iterator } @@ -93,7 +101,8 @@ class BinaryOperatorSpec extends FunSpec with Matchers with ScalaFutures { new RangeVector { override def key: RangeVectorKey = ignoreKey - override def rows: Iterator[RowReader] = Seq( + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 2.0d/0d), new TransientRow(2L, 4.5d), new TransientRow(2L, 0d), @@ -252,19 +261,22 @@ class BinaryOperatorSpec extends FunSpec with Matchers with ScalaFutures { val samples: Array[RangeVector] = Array( new RangeVector { override def key: RangeVectorKey = ignoreKey - override def rows: Iterator[RowReader] = Seq( + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 1), new TransientRow(2L, 2)).iterator }, new RangeVector { override def key: RangeVectorKey = ignoreKey - override def rows: Iterator[RowReader] = Seq( + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 1), new TransientRow(2L, 2)).iterator }, new RangeVector { override def key: RangeVectorKey = ignoreKey - override def rows: Iterator[RowReader] = Seq( + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 1), new TransientRow(2L, 2)).iterator } @@ -288,7 +300,8 @@ class BinaryOperatorSpec extends FunSpec with Matchers with ScalaFutures { val samples: Array[RangeVector] = Array( new RangeVector { override def key: RangeVectorKey = ignoreKey - override def rows: Iterator[RowReader] = Seq( + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 15.004124836249305), new TransientRow(2L, 2)).iterator } diff --git a/query/src/test/scala/filodb/query/exec/rangefn/InstantFunctionSpec.scala b/query/src/test/scala/filodb/query/exec/rangefn/InstantFunctionSpec.scala index be21ac981e..894cbcee18 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/InstantFunctionSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/InstantFunctionSpec.scala @@ -1,12 +1,14 @@ package filodb.query.exec.rangefn import scala.util.Random + import monix.execution.Scheduler.Implicits.global import monix.reactive.Observable import org.scalatest.concurrent.ScalaFutures + import filodb.core.{MetricsTestData, MachineMetricsData => MMD} -import filodb.core.query.{CustomRangeVectorKey, RangeParams, RangeVector, RangeVectorKey, ResultSchema, TransientRow} -import filodb.memory.format.{RowReader, ZeroCopyUTF8String, vectors => bv} +import filodb.core.query._ +import filodb.memory.format.{ZeroCopyUTF8String, vectors => bv} import filodb.query._ import filodb.query.exec.StaticFuncArgs @@ -20,13 +22,15 @@ class InstantFunctionSpec extends RawDataWindowingSpec with ScalaFutures { val sampleBase: Array[RangeVector] = Array( new RangeVector { override def key: RangeVectorKey = ignoreKey - override def rows: Iterator[RowReader] = Seq( + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 3.3d), new TransientRow(2L, 5.1d)).iterator }, new RangeVector { override def key: RangeVectorKey = ignoreKey - override def rows: Iterator[RowReader] = Seq( + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(3L, 3239.3423d), new TransientRow(4L, 94935.1523d)).iterator }) @@ -45,7 +49,8 @@ class InstantFunctionSpec extends RawDataWindowingSpec with ScalaFutures { override def key: RangeVectorKey = ignoreKey - override def rows: Iterator[RowReader] = data.iterator + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = data.iterator }) fireInstantFunctionTests(samples) } @@ -56,20 +61,23 @@ class InstantFunctionSpec extends RawDataWindowingSpec with ScalaFutures { val samples: Array[RangeVector] = Array( new RangeVector { + import filodb.core.query.NoCloseCursor._ override def key: RangeVectorKey = ignoreKey - override def rows: Iterator[RowReader] = Seq( + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, Double.NaN), new TransientRow(2L, 5.6d)).iterator }, new RangeVector { override def key: RangeVectorKey = ignoreKey - override def rows: Iterator[RowReader] = Seq( + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 4.6d), new TransientRow(2L, 4.4d)).iterator }, new RangeVector { override def key: RangeVectorKey = ignoreKey - override def rows: Iterator[RowReader] = Seq( + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 0d), new TransientRow(2L, 5.4d)).iterator } @@ -85,7 +93,8 @@ class InstantFunctionSpec extends RawDataWindowingSpec with ScalaFutures { new RangeVector { override def key: RangeVectorKey = ignoreKey - override def rows: Iterator[RowReader] = Seq( + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 2.0d/0d), new TransientRow(2L, 4.5d), new TransientRow(2L, 0d), @@ -273,7 +282,8 @@ class InstantFunctionSpec extends RawDataWindowingSpec with ScalaFutures { val samples: Array[RangeVector] = Array( new RangeVector { override def key: RangeVectorKey = ignoreKey - override def rows: Iterator[RowReader] = Seq( + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 1456790399), // 2016-02-29 23:59:59 February 29th new TransientRow(2L, 1456790400), // 2016-03-01 00:00:00 March 1st new TransientRow(3L, 1230768000), // 2009-01-01 00:00:00 just after leap second @@ -295,7 +305,8 @@ class InstantFunctionSpec extends RawDataWindowingSpec with ScalaFutures { val samples: Array[RangeVector] = Array( new RangeVector { override def key: RangeVectorKey = ignoreKey - override def rows: Iterator[RowReader] = Seq( + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, Double.NaN), new TransientRow(2L, Double.NaN) ).iterator diff --git a/query/src/test/scala/filodb/query/exec/rangefn/LabelReplaceSpec.scala b/query/src/test/scala/filodb/query/exec/rangefn/LabelReplaceSpec.scala index db0c662aad..afa4de57cd 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/LabelReplaceSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/LabelReplaceSpec.scala @@ -7,8 +7,8 @@ import org.scalatest.{FunSpec, Matchers} import org.scalatest.concurrent.ScalaFutures import filodb.core.MetricsTestData -import filodb.core.query.{CustomRangeVectorKey, QueryConfig, QueryContext, QuerySession, RangeVector, RangeVectorKey, ResultSchema, TransientRow} -import filodb.memory.format.{RowReader, ZeroCopyUTF8String} +import filodb.core.query._ +import filodb.memory.format.ZeroCopyUTF8String import filodb.query._ class LabelReplaceSpec extends FunSpec with Matchers with ScalaFutures { @@ -30,14 +30,16 @@ class LabelReplaceSpec extends FunSpec with Matchers with ScalaFutures { new RangeVector { override def key: RangeVectorKey = testKey1 - override def rows: Iterator[RowReader] = Seq( + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 3.3d), new TransientRow(2L, 5.1d)).iterator }, new RangeVector { override def key: RangeVectorKey = testKey2 - override def rows: Iterator[RowReader] = Seq( + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(3L, 100d), new TransientRow(4L, 200d)).iterator }) @@ -56,14 +58,16 @@ class LabelReplaceSpec extends FunSpec with Matchers with ScalaFutures { new RangeVector { override def key: RangeVectorKey = sampleKey1 - override def rows: Iterator[RowReader] = Seq( + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 3.3d), new TransientRow(2L, 5.1d)).iterator }, new RangeVector { override def key: RangeVectorKey = sampleKey2 - override def rows: Iterator[RowReader] = Seq( + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(3L, 100d), new TransientRow(4L, 200d)).iterator }) @@ -100,14 +104,16 @@ class LabelReplaceSpec extends FunSpec with Matchers with ScalaFutures { new RangeVector { override def key: RangeVectorKey = sampleKey1 - override def rows: Iterator[RowReader] = Seq( + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 3.3d), new TransientRow(2L, 5.1d)).iterator }, new RangeVector { override def key: RangeVectorKey = ignoreKey - override def rows: Iterator[RowReader] = Seq( + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(3L, 100d), new TransientRow(4L, 200d)).iterator }) @@ -145,14 +151,16 @@ class LabelReplaceSpec extends FunSpec with Matchers with ScalaFutures { new RangeVector { override def key: RangeVectorKey = sampleKey1 - override def rows: Iterator[RowReader] = Seq( + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 3.3d), new TransientRow(2L, 5.1d)).iterator }, new RangeVector { override def key: RangeVectorKey = ignoreKey - override def rows: Iterator[RowReader] = Seq( + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(3L, 100d), new TransientRow(4L, 200d)).iterator }) diff --git a/query/src/test/scala/filodb/query/exec/rangefn/LableJoinSpec.scala b/query/src/test/scala/filodb/query/exec/rangefn/LableJoinSpec.scala index 0429c7039c..0f0e429f20 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/LableJoinSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/LableJoinSpec.scala @@ -8,7 +8,7 @@ import org.scalatest.concurrent.ScalaFutures import filodb.core.MetricsTestData import filodb.core.query._ -import filodb.memory.format.{RowReader, ZeroCopyUTF8String} +import filodb.memory.format.ZeroCopyUTF8String import filodb.query._ class LableJoinSpec extends FunSpec with Matchers with ScalaFutures { @@ -55,14 +55,16 @@ class LableJoinSpec extends FunSpec with Matchers with ScalaFutures { new RangeVector { override def key: RangeVectorKey = testKey1 - override def rows: Iterator[RowReader] = Seq( + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 3.3d), new TransientRow(2L, 5.1d)).iterator }, new RangeVector { override def key: RangeVectorKey = testKey2 - override def rows: Iterator[RowReader] = Seq( + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(3L, 100d), new TransientRow(4L, 200d)).iterator }) @@ -71,14 +73,16 @@ class LableJoinSpec extends FunSpec with Matchers with ScalaFutures { new RangeVector { override def key: RangeVectorKey = testKey3 - override def rows: Iterator[RowReader] = Seq( + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 3.3d), new TransientRow(2L, 5.1d)).iterator }, new RangeVector { override def key: RangeVectorKey = testKey4 - override def rows: Iterator[RowReader] = Seq( + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(3L, 100d), new TransientRow(4L, 200d)).iterator }) diff --git a/query/src/test/scala/filodb/query/exec/rangefn/ScalarFunctionSpec.scala b/query/src/test/scala/filodb/query/exec/rangefn/ScalarFunctionSpec.scala index aa62e4e599..b601061931 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/ScalarFunctionSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/ScalarFunctionSpec.scala @@ -2,21 +2,22 @@ package filodb.query.exec.rangefn import java.util.concurrent.TimeUnit +import scala.concurrent.duration.FiniteDuration + import com.typesafe.config.{Config, ConfigFactory} +import monix.execution.Scheduler.Implicits.global +import monix.reactive.Observable +import org.scalatest.{FunSpec, Matchers} +import org.scalatest.concurrent.ScalaFutures + import filodb.core.MetricsTestData import filodb.core.memstore.{FixedMaxPartitionsEvictionPolicy, TimeSeriesMemStore} import filodb.core.metadata.{Dataset, DatasetOptions} import filodb.core.query._ import filodb.core.store.{InMemoryMetaStore, NullColumnStore} -import filodb.memory.format.{RowReader, ZeroCopyUTF8String} +import filodb.memory.format.ZeroCopyUTF8String +import filodb.query.{exec, QueryResult, ScalarFunctionId} import filodb.query.exec.TimeScalarGeneratorExec -import filodb.query.{QueryResult, ScalarFunctionId, exec} -import monix.execution.Scheduler.Implicits.global -import monix.reactive.Observable -import org.scalatest.concurrent.ScalaFutures -import org.scalatest.{FunSpec, Matchers} - -import scala.concurrent.duration.FiniteDuration class ScalarFunctionSpec extends FunSpec with Matchers with ScalaFutures { val timeseriesDataset = Dataset.make("timeseries", @@ -46,19 +47,22 @@ class ScalarFunctionSpec extends FunSpec with Matchers with ScalaFutures { new RangeVector { override def key: RangeVectorKey = testKey1 - override def rows: Iterator[RowReader] = Seq( + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 1d)).iterator }, new RangeVector { override def key: RangeVectorKey = testKey2 - override def rows: Iterator[RowReader] = Seq( + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 5d)).iterator }, new RangeVector { override def key: RangeVectorKey = testKey1 - override def rows: Iterator[RowReader] = Seq( + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 3d), new TransientRow(2L, 3d), new TransientRow(3L, 3d)).iterator @@ -66,25 +70,29 @@ class ScalarFunctionSpec extends FunSpec with Matchers with ScalaFutures { new RangeVector { override def key: RangeVectorKey = testKey1 - override def rows: Iterator[RowReader] = Seq( + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 2d)).iterator }, new RangeVector { override def key: RangeVectorKey = testKey2 - override def rows: Iterator[RowReader] = Seq( + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 4d)).iterator }, new RangeVector { override def key: RangeVectorKey = testKey2 - override def rows: Iterator[RowReader] = Seq( + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 6d)).iterator }, new RangeVector { override def key: RangeVectorKey = testKey1 - override def rows: Iterator[RowReader] = Seq( + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 0d)).iterator }) @@ -92,7 +100,8 @@ class ScalarFunctionSpec extends FunSpec with Matchers with ScalaFutures { new RangeVector { override def key: RangeVectorKey = testKey1 - override def rows: Iterator[RowReader] = Seq( + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 1d), new TransientRow(2L, 10d), new TransientRow(3L, 30d) diff --git a/query/src/test/scala/filodb/query/exec/rangefn/SortFunctionSpec.scala b/query/src/test/scala/filodb/query/exec/rangefn/SortFunctionSpec.scala index 660d04707b..c662fca886 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/SortFunctionSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/SortFunctionSpec.scala @@ -36,49 +36,57 @@ class SortFunctionSpec extends FunSpec with Matchers with ScalaFutures { new RangeVector { override def key: RangeVectorKey = testKey1 - override def rows: Iterator[RowReader] = Seq( + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 1d)).iterator }, new RangeVector { override def key: RangeVectorKey = testKey2 - override def rows: Iterator[RowReader] = Seq( + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 5d)).iterator }, new RangeVector { override def key: RangeVectorKey = testKey1 - override def rows: Iterator[RowReader] = Seq( + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 3d)).iterator }, new RangeVector { override def key: RangeVectorKey = testKey1 - override def rows: Iterator[RowReader] = Seq( + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 2d)).iterator }, new RangeVector { override def key: RangeVectorKey = testKey2 - override def rows: Iterator[RowReader] = Seq( + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 4d)).iterator }, new RangeVector { override def key: RangeVectorKey = testKey2 - override def rows: Iterator[RowReader] = Seq( + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 6d)).iterator }, new RangeVector { override def key: RangeVectorKey = testKey1 - override def rows: Iterator[RowReader] = Seq( + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 0d)).iterator }, new RangeVector { override def key: RangeVectorKey = testKey1 - override def rows: Iterator[RowReader] = Seq.empty[RowReader].iterator + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq.empty[RowReader].iterator } ) @@ -113,13 +121,15 @@ class SortFunctionSpec extends FunSpec with Matchers with ScalaFutures { new RangeVector { override def key: RangeVectorKey = testKey1 - override def rows: Iterator[RowReader] = Seq( + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 1d)).iterator }, new RangeVector { override def key: RangeVectorKey = testKey2 - override def rows: Iterator[RowReader] = Seq( + import filodb.core.query.NoCloseCursor._ + override def rows(): RangeVectorCursor = Seq( new TransientRow(1L, 5d)).iterator }) From a0edcf12748058aac7a3fbb0c429780cce840dad Mon Sep 17 00:00:00 2001 From: "Brian S. O'Neill" Date: Wed, 1 Jul 2020 17:19:52 -0700 Subject: [PATCH 16/22] fix(query): Add lock validation to some unit tests and apply fixes. (#807) --- .../filodb.core/query/RangeVectorCursor.scala | 23 +++++++++++++++++ .../scala/filodb.memory/data/ChunkMap.scala | 11 ++++---- .../query/exec/PeriodicSamplesMapper.scala | 25 +++++++++++++------ .../query/exec/RangeVectorTransformer.scala | 25 +++++++------------ .../query/exec/LastSampleFunctionSpec.scala | 1 + .../query/exec/WindowIteratorSpec.scala | 5 +++- .../rangefn/AggrOverTimeFunctionsSpec.scala | 7 ++++++ .../exec/rangefn/InstantFunctionSpec.scala | 18 ++++++++----- .../exec/rangefn/RateFunctionsSpec.scala | 1 + 9 files changed, 79 insertions(+), 37 deletions(-) diff --git a/core/src/main/scala/filodb.core/query/RangeVectorCursor.scala b/core/src/main/scala/filodb.core/query/RangeVectorCursor.scala index dd552e3d4d..dafc21dc22 100644 --- a/core/src/main/scala/filodb.core/query/RangeVectorCursor.scala +++ b/core/src/main/scala/filodb.core/query/RangeVectorCursor.scala @@ -40,3 +40,26 @@ object NoCloseCursor { } } +/** + * Wraps another cursor and auto-closes it when an exception is thrown. + */ +abstract class WrappedCursor(rows: RangeVectorCursor) extends RangeVectorCursor { + final def next(): RowReader = { + try { + doNext() + } catch { + case e: Throwable => { + close() + throw e + } + } + } + + def hasNext: Boolean = rows.hasNext + + def close(): Unit = rows.close() + + // Subclass must implement this method. + def doNext(): RowReader +} + diff --git a/memory/src/main/scala/filodb.memory/data/ChunkMap.scala b/memory/src/main/scala/filodb.memory/data/ChunkMap.scala index a75ce687ab..e2a78d5bfe 100644 --- a/memory/src/main/scala/filodb.memory/data/ChunkMap.scala +++ b/memory/src/main/scala/filodb.memory/data/ChunkMap.scala @@ -128,14 +128,13 @@ object ChunkMap extends StrictLogging { val numLocksReleased = ChunkMap.releaseAllSharedLocks() if (numLocksReleased > 0) { + val msg = s"Number of locks was non-zero: $numLocksReleased. " + + s"This is indicative of a possible lock acquisition/release bug." if (unitTest) { - // FIXME: Uncomment this when tests have been fixed. - //throw new Error(s"Number of locks was non-zero: $numLocksReleased") + throw new Error(msg) } - logger.error(s"Number of locks was non-zero: $numLocksReleased. " + - s"This is indicative of a possible lock acquisition/release bug.") - // FIXME: Causes failures when running the unit tests for some unknown reason. - //haltAndCatchFire() + logger.error(msg) + haltAndCatchFire() } execPlanTracker.put(t, execPlan) } diff --git a/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala b/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala index 39132d4ec2..761bee89e8 100644 --- a/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala +++ b/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala @@ -158,7 +158,7 @@ abstract class ChunkedWindowIterator[R <: MutableRowReader]( window: Long, rangeFunction: ChunkedRangeFunction[R], querySession: QuerySession) -extends RangeVectorCursor with StrictLogging { +extends WrappedCursor(rv.rows()) with StrictLogging { // Lazily open the iterator and obtain the lock. This allows one thread to create the // iterator, but the lock is owned by the thread actually performing the iteration. private lazy val windowIt = { @@ -170,12 +170,22 @@ extends RangeVectorCursor with StrictLogging { it } - def close: Unit = rv.rows().close() + private var isClosed: Boolean = false + + override def close(): Unit = { + if (!isClosed) { + isClosed = true + val wit = windowIt + wit.unlock() + wit.close() + super.close() + } + } def sampleToEmit: R override def hasNext: Boolean = windowIt.hasMoreWindows - override def next: R = { + override def doNext: R = { rangeFunction.reset() // Lazy variables have an extra lookup cost, due to a volatile bitmap field generated by @@ -196,9 +206,9 @@ extends RangeVectorCursor with StrictLogging { nextInfo.getValueVectorAccessor, nextInfo.getValueVectorAddr) qLogger.error(s"addChunks Exception: info.numRows=${nextInfo.numRows} " + - s"info.endTime=${nextInfo.endTime} curWindowEnd=${wit.curWindowEnd} tsReader=$tsReader " + - s"timestampVectorLength=${tsReader.length(nextInfo.getTsVectorAccessor, nextInfo.getTsVectorAddr)} " + - s"valueVectorLength=${valReader.length(nextInfo.getValueVectorAccessor, nextInfo.getValueVectorAddr)}", e) + s"info.endTime=${nextInfo.endTime} curWindowEnd=${wit.curWindowEnd} tsReader=$tsReader " + + s"timestampVectorLength=${tsReader.length(nextInfo.getTsVectorAccessor, nextInfo.getTsVectorAddr)} " + + s"valueVectorLength=${valReader.length(nextInfo.getValueVectorAccessor, nextInfo.getValueVectorAddr)}", e) throw e } } @@ -206,8 +216,7 @@ extends RangeVectorCursor with StrictLogging { if (!wit.hasMoreWindows) { // Release the shared lock and close the iterator, in case it also holds a lock. - wit.unlock() - wit.close() + close() } sampleToEmit diff --git a/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala b/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala index 84cb709877..dfc838e7f6 100644 --- a/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala +++ b/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala @@ -145,10 +145,9 @@ final case class InstantVectorFunctionMapper(function: InstantFunctionId, private class DoubleInstantFuncIterator(rows: RangeVectorCursor, instantFunction: DoubleInstantFunction, scalar: Seq[ScalarRangeVector], - result: TransientRow = new TransientRow()) extends - RangeVectorCursor { - final def hasNext: Boolean = rows.hasNext - final def next(): RowReader = { + result: TransientRow = new TransientRow()) + extends WrappedCursor(rows) { + override def doNext(): RowReader = { val next = rows.next() val nextVal = next.getDouble(1) val timestamp = next.getLong(0) @@ -156,32 +155,28 @@ private class DoubleInstantFuncIterator(rows: RangeVectorCursor, result.setValues(timestamp, newValue) result } - final def close(): Unit = rows.close() } private class H2DoubleInstantFuncIterator(rows: RangeVectorCursor, instantFunction: HistToDoubleIFunction, scalar: Seq[ScalarRangeVector], result: TransientRow = new TransientRow()) - extends RangeVectorCursor { - final def hasNext: Boolean = rows.hasNext - final def next(): RowReader = { + extends WrappedCursor(rows) { + override def doNext(): RowReader = { val next = rows.next() val timestamp = next.getLong(0) val newValue = instantFunction(next.getHistogram(1), scalar.map(_.getValue(timestamp))) result.setValues(timestamp, newValue) result } - final def close(): Unit = rows.close() } private class HD2DoubleInstantFuncIterator(rows: RangeVectorCursor, instantFunction: HDToDoubleIFunction, scalar: Seq[ScalarRangeVector], result: TransientRow = new TransientRow()) - extends RangeVectorCursor { - final def hasNext: Boolean = rows.hasNext - final def next(): RowReader = { + extends WrappedCursor(rows) { + override def doNext(): RowReader = { val next = rows.next() val timestamp = next.getLong(0) val newValue = instantFunction(next.getHistogram(1), @@ -189,7 +184,6 @@ private class HD2DoubleInstantFuncIterator(rows: RangeVectorCursor, result.setValues(timestamp, newValue) result } - final def close(): Unit = rows.close() } /** @@ -220,13 +214,13 @@ final case class ScalarOperationMapper(operator: BinaryOperator, private def evaluate(source: Observable[RangeVector], scalarRangeVector: ScalarRangeVector) = { source.map { rv => - val resultIterator: RangeVectorCursor = new RangeVectorCursor() { + val resultIterator: RangeVectorCursor = new WrappedCursor(rv.rows) { private val rows = rv.rows private val result = new TransientRow() override def hasNext: Boolean = rows.hasNext - override def next(): RowReader = { + override def doNext(): RowReader = { val next = rows.next() val nextVal = next.getDouble(1) val timestamp = next.getLong(0) @@ -236,7 +230,6 @@ final case class ScalarOperationMapper(operator: BinaryOperator, result.setValues(timestamp, newValue) result } - override def close(): Unit = rv.rows().close() } IteratorBackedRangeVector(rv.key, resultIterator) } diff --git a/query/src/test/scala/filodb/query/exec/LastSampleFunctionSpec.scala b/query/src/test/scala/filodb/query/exec/LastSampleFunctionSpec.scala index 9db822df28..794393aa9f 100644 --- a/query/src/test/scala/filodb/query/exec/LastSampleFunctionSpec.scala +++ b/query/src/test/scala/filodb/query/exec/LastSampleFunctionSpec.scala @@ -151,5 +151,6 @@ class LastSampleFunctionSpec extends RawDataWindowingSpec { cur = cur + step } output.hasNext shouldEqual false + output.close() } } diff --git a/query/src/test/scala/filodb/query/exec/WindowIteratorSpec.scala b/query/src/test/scala/filodb/query/exec/WindowIteratorSpec.scala index 92019388d9..770d675114 100644 --- a/query/src/test/scala/filodb/query/exec/WindowIteratorSpec.scala +++ b/query/src/test/scala/filodb/query/exec/WindowIteratorSpec.scala @@ -424,6 +424,7 @@ class WindowIteratorSpec extends RawDataWindowingSpec { Some(InternalRangeFunction.AvgOverTime), ColumnType.DoubleColumn, queryConfig, useChunked = false).asSliding, queryConfig) slidingWinIterator.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual windowResults + slidingWinIterator.close() val chunkedIt = new ChunkedWindowIteratorD(rv, 50000L, 100000, 700000L, 100000, RangeFunction(tsResSchema, @@ -459,7 +460,7 @@ class WindowIteratorSpec extends RawDataWindowingSpec { Some(InternalRangeFunction.CountOverTime), ColumnType.DoubleColumn, queryConfig, useChunked = false).asSliding, queryConfig) slidingWinIterator.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual windowResults - + slidingWinIterator.close(); val chunkedIt = new ChunkedWindowIteratorD(rv, 50000L, 100000, 700000L, 100000, RangeFunction(tsResSchema, @@ -549,6 +550,7 @@ class WindowIteratorSpec extends RawDataWindowingSpec { Some(InternalRangeFunction.MinOverTime), ColumnType.DoubleColumn, queryConfig, useChunked = false).asSliding, queryConfig) slidingWinIterator.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual windowResults + slidingWinIterator.close() val chunkedIt = new ChunkedWindowIteratorD(rv, 50000L, 100000, 700000L, 100000, RangeFunction(tsResSchema, @@ -585,6 +587,7 @@ class WindowIteratorSpec extends RawDataWindowingSpec { Some(InternalRangeFunction.MaxOverTime), ColumnType.DoubleColumn, queryConfig, useChunked = false).asSliding, queryConfig) slidingWinIterator.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual windowResults + slidingWinIterator.close() val chunkedIt = new ChunkedWindowIteratorD(rv, 50000L, 100000, 700000L, 100000, RangeFunction(tsResSchema, diff --git a/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala b/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala index 8ebce29b2f..8b1d0698fc 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala @@ -252,6 +252,7 @@ class AggrOverTimeFunctionsSpec extends RawDataWindowingSpec { info(s"iteration $x windowSize=$windowSize step=$step") val slidingIt = slidingWindowIt(data, rv, new SumOverTimeFunction(), windowSize, step) val aggregated = slidingIt.map(_.getDouble(1)).toBuffer + slidingIt.close() // drop first sample because of exclusive start aggregated shouldEqual data.sliding(windowSize, step).map(_.drop(1).sum).toBuffer @@ -313,6 +314,7 @@ class AggrOverTimeFunctionsSpec extends RawDataWindowingSpec { val minSlidingIt = slidingWindowIt(data, rv, new MinMaxOverTimeFunction(Ordering[Double].reverse), windowSize, step) val aggregated = minSlidingIt.map(_.getDouble(1)).toBuffer + minSlidingIt.close() // drop first sample because of exclusive start aggregated shouldEqual data.sliding(windowSize, step).map(_.drop(1).min).toBuffer @@ -322,6 +324,7 @@ class AggrOverTimeFunctionsSpec extends RawDataWindowingSpec { val maxSlidingIt = slidingWindowIt(data, rv, new MinMaxOverTimeFunction(Ordering[Double]), windowSize, step) val aggregated3 = maxSlidingIt.map(_.getDouble(1)).toBuffer + maxSlidingIt.close() // drop first sample because of exclusive start aggregated3 shouldEqual data.sliding(windowSize, step).map(_.drop(1).max).toBuffer @@ -343,6 +346,7 @@ class AggrOverTimeFunctionsSpec extends RawDataWindowingSpec { val countSliding = slidingWindowIt(data, rv, new CountOverTimeFunction(), windowSize, step) val aggregated1 = countSliding.map(_.getDouble(1)).toBuffer + countSliding.close() aggregated1 shouldEqual data.sliding(windowSize, step).map(_.length - 1).toBuffer val countChunked = chunkedWindowIt(data, rv, new CountOverTimeChunkedFunction(), windowSize, step) @@ -351,6 +355,7 @@ class AggrOverTimeFunctionsSpec extends RawDataWindowingSpec { val avgSliding = slidingWindowIt(data, rv, new AvgOverTimeFunction(), windowSize, step) val aggregated3 = avgSliding.map(_.getDouble(1)).toBuffer + avgSliding.close() aggregated3 shouldEqual data.sliding(windowSize, step).map(a => avg(a drop 1)).toBuffer // In sample_data2, there are no NaN's, that's why using avg function is fine @@ -376,10 +381,12 @@ class AggrOverTimeFunctionsSpec extends RawDataWindowingSpec { val varSlidingIt = slidingWindowIt(data, rv, new StdVarOverTimeFunction(), windowSize, step) val aggregated2 = varSlidingIt.map(_.getDouble(1)).toBuffer + varSlidingIt.close() aggregated2 shouldEqual data.sliding(windowSize, step).map(a => stdVar(a drop 1)).toBuffer val stdDevSlidingIt = slidingWindowIt(data, rv, new StdDevOverTimeFunction(), windowSize, step) val aggregated3 = stdDevSlidingIt.map(_.getDouble(1)).toBuffer + stdDevSlidingIt.close() aggregated3 shouldEqual data.sliding(windowSize, step).map(d => Math.sqrt(stdVar(d drop 1))).toBuffer val varFunc = new StdVarOverTimeChunkedFunctionD() diff --git a/query/src/test/scala/filodb/query/exec/rangefn/InstantFunctionSpec.scala b/query/src/test/scala/filodb/query/exec/rangefn/InstantFunctionSpec.scala index 894cbcee18..690eea588a 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/InstantFunctionSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/InstantFunctionSpec.scala @@ -326,16 +326,22 @@ class InstantFunctionSpec extends RawDataWindowingSpec with ScalaFutures { schema: ResultSchema = resultSchema): Unit = { val instantVectorFnMapper = exec.InstantVectorFunctionMapper(instantFunctionId, funcParams.map(x => StaticFuncArgs(x, RangeParams(100,10,200)))) val resultObs = instantVectorFnMapper(Observable.fromIterable(samples), querySession, 1000, schema, Nil) - val result = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) + val result = resultObs.toListL.runAsync.futureValue.map(_.rows) expectedVal.zip(result).foreach { case (ex, res) => { ex.zip(res).foreach { - case (val1, val2) => - if (val1.isInfinity) val2.isInfinity shouldEqual true - else if (val1.isNaN) val2.isNaN shouldEqual true - else val1 shouldEqual val2 +- 0.0001 + case (val1, val2) => { + val val2Num = val2.getDouble(1) + if (val1.isInfinity) val2Num.isInfinity shouldEqual true + else if (val1.isNaN) val2Num.isNaN shouldEqual true + else val1 shouldEqual val2Num +- 0.0001 + } } + // Ensure that locks are released from DoubleInstantFuncIterator. A couple of the tests + // don't feed in enough expected data for the iterator to reach the end naturally and + // close itself. + res.close(); } } } -} \ No newline at end of file +} diff --git a/query/src/test/scala/filodb/query/exec/rangefn/RateFunctionsSpec.scala b/query/src/test/scala/filodb/query/exec/rangefn/RateFunctionsSpec.scala index c6efe1f78b..a2a30c1ed9 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/RateFunctionsSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/RateFunctionsSpec.scala @@ -167,6 +167,7 @@ class RateFunctionsSpec extends RawDataWindowingSpec { val slidingRate = slidingWindowIt(data, rv, RateFunction, windowSize, step) val slidingResults = slidingRate.map(_.getDouble(1)).toBuffer + slidingRate.close() val rateChunked = chunkedWindowIt(data, rv, new ChunkedRateFunction, windowSize, step) val resultRows = rateChunked.map { r => (r.getLong(0), r.getDouble(1)) }.toBuffer From e1b684a8d36e6079bb5935415b746e103dcec920 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Mon, 6 Jul 2020 12:48:43 -0700 Subject: [PATCH 17/22] debug(memory): Add additional debug logging when allocMetadata() call fails (#808) --- .../src/main/scala/filodb.memory/Block.scala | 25 +++++++++++++++++++ .../main/scala/filodb.memory/MemFactory.scala | 2 +- .../test/scala/filodb.memory/BlockSpec.scala | 3 +++ 3 files changed, 29 insertions(+), 1 deletion(-) diff --git a/memory/src/main/scala/filodb.memory/Block.scala b/memory/src/main/scala/filodb.memory/Block.scala index 0995237637..754ab49cc9 100644 --- a/memory/src/main/scala/filodb.memory/Block.scala +++ b/memory/src/main/scala/filodb.memory/Block.scala @@ -104,6 +104,10 @@ trait ReusableMemory extends StrictLogging { } } +object Block extends StrictLogging { + val _log = logger +} + /** * A block is a reusable piece of memory beginning at the address and has a capacity. * It is capable of holding metadata also for reclaims. @@ -177,6 +181,7 @@ class Block(val address: Long, val capacity: Long, val reclaimListener: ReclaimL UnsafeUtils.setShort(UnsafeUtils.ZeroPointer, metaAddr, metaSize) metaAddr + 2 } else { + Block._log.error(s"Unexpected ERROR with allocMetadata. Block info: $detailedDebugString") throw new OutOfOffheapMemoryException(metaSize, rem) } } @@ -211,6 +216,26 @@ class Block(val address: Long, val capacity: Long, val reclaimListener: ReclaimL def debugString: String = f"Block @0x$address%016x canReclaim=$canReclaim remaining=$remaining " + s"owner: ${owner.map(_.debugString).getOrElse("--")}" + // Include detailed metadata debug info, enough to debug any block metadata allocation issues + // Meta alloc overhead, plus histogram and stats on meta allocation, plus pointer info + def detailedDebugString: String = { + val metasizeHist = collection.mutable.HashMap[Int, Int]().withDefaultValue(0) + var numMetas = 0 + + // Walk metadata blocks and collect stats + var metaPointer = address + _metaPosition + while (metaPointer < (address + capacity)) { + val metaSize = UnsafeUtils.getShort(metaPointer) + numMetas += 1 + metasizeHist(metaSize) += 1 + metaPointer += (2 + metaSize) + } + + debugString + + f"\ncapacity=$capacity position=${_position} metaPos=${_metaPosition} gap=${_metaPosition - _position}\n" + + s"# metadatas=$numMetas metadata size histogram=$metasizeHist" + } + // debug method to set memory to specific value for testing private[memory] def set(value: Byte): Unit = UnsafeUtils.unsafe.setMemory(address, capacity, value) diff --git a/memory/src/main/scala/filodb.memory/MemFactory.scala b/memory/src/main/scala/filodb.memory/MemFactory.scala index 3dbc352f74..4bbdf5ca3c 100644 --- a/memory/src/main/scala/filodb.memory/MemFactory.scala +++ b/memory/src/main/scala/filodb.memory/MemFactory.scala @@ -371,7 +371,7 @@ class BlockMemFactory(blockStore: BlockManager, def shutdown(): Unit = {} def debugString: String = - s"BlockMemFactory($bucketTime) ${tags.map { case (k, v) => s"$k=$v" }.mkString(" ")}" + s"BlockMemFactory($bucketTime, $metadataAllocSize) ${tags.map { case (k, v) => s"$k=$v" }.mkString(" ")}" } diff --git a/memory/src/test/scala/filodb.memory/BlockSpec.scala b/memory/src/test/scala/filodb.memory/BlockSpec.scala index dacc597a6b..ec8853c55b 100644 --- a/memory/src/test/scala/filodb.memory/BlockSpec.scala +++ b/memory/src/test/scala/filodb.memory/BlockSpec.scala @@ -65,6 +65,9 @@ class BlockSpec extends FlatSpec with Matchers with BeforeAndAfter with BeforeAn block.remaining shouldEqual 3822 + //XXX for debugging + println(block.detailedDebugString) + block.markReclaimable() block.reclaim() testReclaimer.reclaimedBytes shouldEqual 70 From 484f3102cc9ad8d4cec2f0b5b44495cdd170f838 Mon Sep 17 00:00:00 2001 From: Jackson Jeyapaul Date: Tue, 7 Jul 2020 09:27:31 -0700 Subject: [PATCH 18/22] misc(doc): Rename master to main in docs (#810) --- CONTRIBUTING.md | 2 +- doc/FiloDB_Taxi_Geo_demo.snb | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index 3375f6e2f8..a25c5e25ab 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -1,7 +1,7 @@ # Contribution Guidelines Thank you for thinking of contributing to FiloDB! We welcome all contributions through Github Pull Requests. -When you create a new PR, please be sure to review the guidelines below. Then create a branch based on `develop` (not `master`). +When you create a new PR, please be sure to review the guidelines below. Then create a branch based on `develop` (not `main`). ## Pull Requests Guidelines Pull requests are the only means by which you can contribute to this project, please follow the following steps when submitting pull requests : diff --git a/doc/FiloDB_Taxi_Geo_demo.snb b/doc/FiloDB_Taxi_Geo_demo.snb index 6622eaf645..e8d4e0de0f 100644 --- a/doc/FiloDB_Taxi_Geo_demo.snb +++ b/doc/FiloDB_Taxi_Geo_demo.snb @@ -31,7 +31,7 @@ "id" : "B3641CCECB6B45F2A885DC4A03E7B0E6" }, "cell_type" : "markdown", - "source" : "The public [NYC Taxi Dataset](http://www.andresmh.com/nyctaxitrips/) contains telemetry (pickup, dropoff locations, times) info on millions of taxi rides in NYC.\n\n| Medallion prefix | 1/1 - 1/6 | 1/7 - 1/12 |\n| ----------------- | ----------- | ------------ |\n| AA | records | records |\n| AB | records | records |\n\n* Partition key - `:stringPrefix medallion 2` - hash multiple drivers trips into ~300 partitions\n* Segment key - `:timeslice pickup_datetime 6d`\n\nAllows for easy filtering by individual drivers, and slicing by time.\n\nThis Notebook assumes the user has followed the FiloDB NYC Taxi [ingestion instructions](https://github.com/filodb/FiloDB/blob/master/doc/datasets_reading.md) first." + "source" : "The public [NYC Taxi Dataset](http://www.andresmh.com/nyctaxitrips/) contains telemetry (pickup, dropoff locations, times) info on millions of taxi rides in NYC.\n\n| Medallion prefix | 1/1 - 1/6 | 1/7 - 1/12 |\n| ----------------- | ----------- | ------------ |\n| AA | records | records |\n| AB | records | records |\n\n* Partition key - `:stringPrefix medallion 2` - hash multiple drivers trips into ~300 partitions\n* Segment key - `:timeslice pickup_datetime 6d`\n\nAllows for easy filtering by individual drivers, and slicing by time.\n\nThis Notebook assumes the user has followed the FiloDB NYC Taxi [ingestion instructions](https://github.com/filodb/FiloDB/blob/main/doc/datasets_reading.md) first." }, { "metadata" : { "id" : "C0F7DF68B6FA4C2D86BA6C5A96EFD1E9" From a4a16cc95d9fb3918b2d50129b6d07fb33f20fc3 Mon Sep 17 00:00:00 2001 From: jackson-paul Date: Tue, 7 Jul 2020 09:32:13 -0700 Subject: [PATCH 19/22] Update integration version to 0.9.8.integration --- version.sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/version.sbt b/version.sbt index e7363ced9f..fe1b2e3cc3 100644 --- a/version.sbt +++ b/version.sbt @@ -1 +1 @@ -version in ThisBuild := "0.9.7.integration-SNAPSHOT" +version in ThisBuild := "0.9.8.integration-SNAPSHOT" From 2600daa896127459214dad0b98db4b1ca854f333 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Tue, 7 Jul 2020 13:16:56 -0700 Subject: [PATCH 20/22] perf(sparkJobs): Reducing cass batch size to make c* queries smaller (#809) * Reduce downsample batch size from 10k to 500 to reduce cass query size - more queries instead of few very big queries. * Changed spark job to use sync cass calls since they are single threaded anyway - I was encountering unit test bugs in the observable.toIterator conversion. * Remove timed batches since batch sizes were not consistent. --- .../columnstore/CassandraColumnStore.scala | 34 +++++++------ .../columnstore/IngestionTimeIndexTable.scala | 10 ++-- .../columnstore/TimeSeriesChunksTable.scala | 17 +++++++ .../CassandraColumnStoreSpec.scala | 50 ++++++++++++++++++- core/src/main/resources/filodb-defaults.conf | 5 +- .../downsampler/chunk/DownsamplerMain.scala | 10 ++-- .../chunk/DownsamplerSettings.scala | 2 - 7 files changed, 93 insertions(+), 35 deletions(-) diff --git a/cassandra/src/main/scala/filodb.cassandra/columnstore/CassandraColumnStore.scala b/cassandra/src/main/scala/filodb.cassandra/columnstore/CassandraColumnStore.scala index 46e8b23285..012cca0d99 100644 --- a/cassandra/src/main/scala/filodb.cassandra/columnstore/CassandraColumnStore.scala +++ b/cassandra/src/main/scala/filodb.cassandra/columnstore/CassandraColumnStore.scala @@ -187,33 +187,37 @@ extends ColumnStore with CassandraChunkSource with StrictLogging { * handle this case. */ // scalastyle:off parameter.number - def getChunksByIngestionTimeRange(datasetRef: DatasetRef, - splits: Iterator[ScanSplit], - ingestionTimeStart: Long, - ingestionTimeEnd: Long, - userTimeStart: Long, - endTimeExclusive: Long, - maxChunkTime: Long, - batchSize: Int, - batchTime: FiniteDuration): Observable[Seq[RawPartData]] = { - val partKeys = Observable.fromIterator(splits).flatMap { + def getChunksByIngestionTimeRangeNoAsync(datasetRef: DatasetRef, + splits: Iterator[ScanSplit], + ingestionTimeStart: Long, + ingestionTimeEnd: Long, + userTimeStart: Long, + endTimeExclusive: Long, + maxChunkTime: Long, + batchSize: Int): Iterator[Seq[RawPartData]] = { + val partKeys = splits.flatMap { case split: CassandraTokenRangeSplit => val indexTable = getOrCreateIngestionTimeIndexTable(datasetRef) logger.debug(s"Querying cassandra for partKeys for split=$split ingestionTimeStart=$ingestionTimeStart " + s"ingestionTimeEnd=$ingestionTimeEnd") - indexTable.scanPartKeysByIngestionTime(split.tokens, ingestionTimeStart, ingestionTimeEnd) + indexTable.scanPartKeysByIngestionTimeNoAsync(split.tokens, ingestionTimeStart, ingestionTimeEnd) case split => throw new UnsupportedOperationException(s"Unknown split type $split seen") } import filodb.core.Iterators._ val chunksTable = getOrCreateChunkTable(datasetRef) - partKeys.bufferTimedAndCounted(batchTime, batchSize).map { parts => + partKeys.sliding(batchSize, batchSize).map { parts => logger.debug(s"Querying cassandra for chunks from ${parts.size} partitions userTimeStart=$userTimeStart " + s"endTimeExclusive=$endTimeExclusive maxChunkTime=$maxChunkTime") - // TODO evaluate if we can increase parallelism here. This needs to be tuneable - // based on how much faster downsampling should run, and how much additional read load cassandra can take. - chunksTable.readRawPartitionRangeBB(parts, userTimeStart - maxChunkTime, endTimeExclusive).toIterator().toSeq + // This could be more parallel, but decision was made to control parallelism at one place: In spark (via its + // parallelism configuration. Revisit if needed later. + val batchReadSpan = Kamon.spanBuilder("cassandra-per-batch-data-read-latency").start() + try { + chunksTable.readRawPartitionRangeBBNoAsync(parts, userTimeStart - maxChunkTime, endTimeExclusive) + } finally { + batchReadSpan.finish() + } } } diff --git a/cassandra/src/main/scala/filodb.cassandra/columnstore/IngestionTimeIndexTable.scala b/cassandra/src/main/scala/filodb.cassandra/columnstore/IngestionTimeIndexTable.scala index 318b1fce7c..4ae4bdfb03 100644 --- a/cassandra/src/main/scala/filodb.cassandra/columnstore/IngestionTimeIndexTable.scala +++ b/cassandra/src/main/scala/filodb.cassandra/columnstore/IngestionTimeIndexTable.scala @@ -5,7 +5,6 @@ import java.nio.ByteBuffer import scala.concurrent.{ExecutionContext, Future} import com.datastax.driver.core.{ConsistencyLevel, ResultSet, Row} -import monix.reactive.Observable import filodb.cassandra.FiloCassandraConnector import filodb.core._ @@ -101,10 +100,10 @@ sealed class IngestionTimeIndexTable(val dataset: DatasetRef, } } - def scanPartKeysByIngestionTime(tokens: Seq[(String, String)], - ingestionTimeStart: Long, - ingestionTimeEnd: Long): Observable[ByteBuffer] = { - val it = tokens.iterator.flatMap { case (start, end) => + def scanPartKeysByIngestionTimeNoAsync(tokens: Seq[(String, String)], + ingestionTimeStart: Long, + ingestionTimeEnd: Long): Iterator[ByteBuffer] = { + tokens.iterator.flatMap { case (start, end) => /* * FIXME conversion of tokens to Long works only for Murmur3Partitioner because it generates * Long based tokens. If other partitioners are used, this can potentially break. @@ -117,7 +116,6 @@ sealed class IngestionTimeIndexTable(val dataset: DatasetRef, session.execute(stmt).iterator.asScala .map { row => row.getBytes("partition") } } - Observable.fromIterator(it).handleObservableErrors } /** diff --git a/cassandra/src/main/scala/filodb.cassandra/columnstore/TimeSeriesChunksTable.scala b/cassandra/src/main/scala/filodb.cassandra/columnstore/TimeSeriesChunksTable.scala index cf21cea524..56c44768ca 100644 --- a/cassandra/src/main/scala/filodb.cassandra/columnstore/TimeSeriesChunksTable.scala +++ b/cassandra/src/main/scala/filodb.cassandra/columnstore/TimeSeriesChunksTable.scala @@ -205,6 +205,23 @@ sealed class TimeSeriesChunksTable(val dataset: DatasetRef, } yield rpd } + /** + * Not an async call - limit number of partitions queried at a time + */ + def readRawPartitionRangeBBNoAsync(partitions: Seq[ByteBuffer], + startTime: Long, + endTimeExclusive: Long): Seq[RawPartData] = { + val query = readChunkRangeCql.bind().setList(0, partitions.asJava, classOf[ByteBuffer]) + .setLong(1, chunkID(startTime, 0)) + .setLong(2, chunkID(endTimeExclusive, 0)) + session.execute(query).iterator().asScala + .map { row => (row.getBytes(0), chunkSetFromRow(row, 1)) } + .sortedGroupBy(_._1) + .map { case (partKeyBuffer, chunkSetIt) => + RawPartData(partKeyBuffer.array, chunkSetIt.map(_._2).toBuffer) + }.toSeq + } + def scanPartitionsBySplit(tokens: Seq[(String, String)]): Observable[RawPartData] = { val res: Observable[Future[Iterator[RawPartData]]] = Observable.fromIterable(tokens).map { case (start, end) => diff --git a/cassandra/src/test/scala/filodb.cassandra/columnstore/CassandraColumnStoreSpec.scala b/cassandra/src/test/scala/filodb.cassandra/columnstore/CassandraColumnStoreSpec.scala index f1466f3031..99c60fb42e 100644 --- a/cassandra/src/test/scala/filodb.cassandra/columnstore/CassandraColumnStoreSpec.scala +++ b/cassandra/src/test/scala/filodb.cassandra/columnstore/CassandraColumnStoreSpec.scala @@ -16,8 +16,8 @@ import filodb.core._ import filodb.core.binaryrecord2.RecordBuilder import filodb.core.metadata.{Dataset, Schemas} import filodb.core.store.{ChunkSet, ChunkSetInfo, ColumnStoreSpec, PartKeyRecord} -import filodb.memory.BinaryRegionLarge -import filodb.memory.format.UnsafeUtils +import filodb.memory.{BinaryRegionLarge, NativeMemoryManager} +import filodb.memory.format.{TupleRowReader, UnsafeUtils} import filodb.memory.format.ZeroCopyUTF8String._ class CassandraColumnStoreSpec extends ColumnStoreSpec { @@ -27,6 +27,21 @@ class CassandraColumnStoreSpec extends ColumnStoreSpec { lazy val colStore = new CassandraColumnStore(config, s, session) lazy val metaStore = new CassandraMetaStore(config.getConfig("cassandra"), session) + val nativeMemoryManager = new NativeMemoryManager(100000000L, Map.empty) + val promDataset = Dataset("prometheus", Schemas.gauge) + + // First create the tables in C* + override def beforeAll(): Unit = { + super.beforeAll() + colStore.initialize(promDataset.ref, 1).futureValue + colStore.truncate(promDataset.ref, 1).futureValue + } + + override def afterAll(): Unit = { + super.afterAll() + nativeMemoryManager.shutdown() + } + "getScanSplits" should "return splits from Cassandra" in { // Single split, token_start should equal token_end val singleSplits = colStore.getScanSplits(dataset.ref).asInstanceOf[Seq[CassandraTokenRangeSplit]] @@ -224,4 +239,35 @@ class CassandraColumnStoreSpec extends ColumnStoreSpec { parts(0).chunkSetsTimeOrdered should have length (1) parts(0).chunkSetsTimeOrdered(0).vectors.toSeq shouldEqual sourceChunks.head.chunks } + + "getChunksByIngestionTimeRangeNoAsync" should "batch partitions properly" in { + + val gaugeName = "my_gauge" + val seriesTags = Map("_ws_".utf8 -> "my_ws".utf8) + val firstSampleTime = 74373042000L + val partBuilder = new RecordBuilder(nativeMemoryManager) + val ingestTime = 1594130687316L + val rows = Seq(TupleRowReader((Some(firstSampleTime), Some(0.0d)))) + val chunksets = for { i <- 0 until 1050 } yield { + val partKey = partBuilder.partKeyFromObjects(Schemas.gauge, gaugeName + i, seriesTags) + ChunkSet(Schemas.gauge.data, partKey, ingestTime, rows, nativeMemoryManager) + } + colStore.write(promDataset.ref, Observable.fromIterable(chunksets)).futureValue + + val batches = colStore.getChunksByIngestionTimeRangeNoAsync( + promDataset.ref, + colStore.getScanSplits(promDataset.ref).iterator, + ingestTime - 1, + ingestTime + 1, + firstSampleTime - 1, + firstSampleTime + 1, + 10L, + 100 + ).toList + + batches.size shouldEqual 11 // 100 rows per batch, 1050 rows => 11 batches + batches.zipWithIndex.foreach { case (b, i) => + b.size shouldEqual (if (i == 10) 50 else 100) + } + } } diff --git a/core/src/main/resources/filodb-defaults.conf b/core/src/main/resources/filodb-defaults.conf index b74fc6e635..c7d1a31a4b 100644 --- a/core/src/main/resources/filodb-defaults.conf +++ b/core/src/main/resources/filodb-defaults.conf @@ -233,10 +233,7 @@ filodb { # cass-session-provider-fqcn = fqcn # Number of time series to operate on at one time. Reduce if there is much less memory available - cass-write-batch-size = 10000 - - # Maximum time to wait during cassandra reads to form a batch of partitions to downsample - cass-write-batch-time = 3s + cass-write-batch-size = 500 # amount of parallelism to introduce in the spark job. This controls number of spark partitions # increase if the number of splits seen in cassandra reads is low and spark jobs are slow. diff --git a/spark-jobs/src/main/scala/filodb/downsampler/chunk/DownsamplerMain.scala b/spark-jobs/src/main/scala/filodb/downsampler/chunk/DownsamplerMain.scala index f6e8632d49..66df533946 100644 --- a/spark-jobs/src/main/scala/filodb/downsampler/chunk/DownsamplerMain.scala +++ b/spark-jobs/src/main/scala/filodb/downsampler/chunk/DownsamplerMain.scala @@ -108,17 +108,15 @@ class Downsampler(settings: DownsamplerSettings, batchDownsampler: BatchDownsamp .mapPartitions { splitIter => Kamon.init() KamonShutdownHook.registerShutdownHook() - import filodb.core.Iterators._ val rawDataSource = batchDownsampler.rawCassandraColStore - val batchReadSpan = Kamon.spanBuilder("cassandra-raw-data-read-latency").start() - val batchIter = rawDataSource.getChunksByIngestionTimeRange(datasetRef = batchDownsampler.rawDatasetRef, + val batchIter = rawDataSource.getChunksByIngestionTimeRangeNoAsync( + datasetRef = batchDownsampler.rawDatasetRef, splits = splitIter, ingestionTimeStart = ingestionTimeStart, ingestionTimeEnd = ingestionTimeEnd, userTimeStart = userTimeStart, endTimeExclusive = userTimeEndExclusive, maxChunkTime = settings.rawDatasetIngestionConfig.storeConfig.maxChunkTime.toMillis, - batchSize = settings.batchSize, batchTime = settings.batchTime).toIterator() - batchReadSpan.finish() - batchIter // iterator of batches + batchSize = settings.batchSize) + batchIter } .foreach { rawPartsBatch => Kamon.init() diff --git a/spark-jobs/src/main/scala/filodb/downsampler/chunk/DownsamplerSettings.scala b/spark-jobs/src/main/scala/filodb/downsampler/chunk/DownsamplerSettings.scala index 4ee4f055cf..bb0b564159 100644 --- a/spark-jobs/src/main/scala/filodb/downsampler/chunk/DownsamplerSettings.scala +++ b/spark-jobs/src/main/scala/filodb/downsampler/chunk/DownsamplerSettings.scala @@ -54,8 +54,6 @@ class DownsamplerSettings(conf: Config = ConfigFactory.empty()) extends Serializ @transient lazy val batchSize = downsamplerConfig.getInt("cass-write-batch-size") - @transient lazy val batchTime = downsamplerConfig.as[FiniteDuration]("cass-write-batch-time") - @transient lazy val splitsPerNode = downsamplerConfig.getInt("splits-per-node") @transient lazy val cassWriteTimeout = downsamplerConfig.as[FiniteDuration]("cassandra-write-timeout") From 961cfbcce8c90cdf745eee933668925f3de682f7 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Tue, 7 Jul 2020 23:22:43 -0700 Subject: [PATCH 21/22] bug(cassandra): Read from ingestion index table had duplicates (#812) Scan on ingestion index table results in duplicates since there is a row per matching chunk. Remove duplicates from the query result before batching. --- .../columnstore/CassandraColumnStore.scala | 2 -- .../columnstore/IngestionTimeIndexTable.scala | 3 +-- .../columnstore/CassandraColumnStoreSpec.scala | 12 ++++++++---- core/src/main/resources/filodb-defaults.conf | 2 +- 4 files changed, 10 insertions(+), 9 deletions(-) diff --git a/cassandra/src/main/scala/filodb.cassandra/columnstore/CassandraColumnStore.scala b/cassandra/src/main/scala/filodb.cassandra/columnstore/CassandraColumnStore.scala index 012cca0d99..ec2449967a 100644 --- a/cassandra/src/main/scala/filodb.cassandra/columnstore/CassandraColumnStore.scala +++ b/cassandra/src/main/scala/filodb.cassandra/columnstore/CassandraColumnStore.scala @@ -204,8 +204,6 @@ extends ColumnStore with CassandraChunkSource with StrictLogging { case split => throw new UnsupportedOperationException(s"Unknown split type $split seen") } - import filodb.core.Iterators._ - val chunksTable = getOrCreateChunkTable(datasetRef) partKeys.sliding(batchSize, batchSize).map { parts => logger.debug(s"Querying cassandra for chunks from ${parts.size} partitions userTimeStart=$userTimeStart " + diff --git a/cassandra/src/main/scala/filodb.cassandra/columnstore/IngestionTimeIndexTable.scala b/cassandra/src/main/scala/filodb.cassandra/columnstore/IngestionTimeIndexTable.scala index 4ae4bdfb03..704640df3e 100644 --- a/cassandra/src/main/scala/filodb.cassandra/columnstore/IngestionTimeIndexTable.scala +++ b/cassandra/src/main/scala/filodb.cassandra/columnstore/IngestionTimeIndexTable.scala @@ -113,8 +113,7 @@ sealed class IngestionTimeIndexTable(val dataset: DatasetRef, end.toLong: java.lang.Long, ingestionTimeStart: java.lang.Long, ingestionTimeEnd: java.lang.Long) - session.execute(stmt).iterator.asScala - .map { row => row.getBytes("partition") } + session.execute(stmt).asScala.map { row => row.getBytes("partition") }.toSet.iterator } } diff --git a/cassandra/src/test/scala/filodb.cassandra/columnstore/CassandraColumnStoreSpec.scala b/cassandra/src/test/scala/filodb.cassandra/columnstore/CassandraColumnStoreSpec.scala index 99c60fb42e..c1318173bb 100644 --- a/cassandra/src/test/scala/filodb.cassandra/columnstore/CassandraColumnStoreSpec.scala +++ b/cassandra/src/test/scala/filodb.cassandra/columnstore/CassandraColumnStoreSpec.scala @@ -247,9 +247,12 @@ class CassandraColumnStoreSpec extends ColumnStoreSpec { val firstSampleTime = 74373042000L val partBuilder = new RecordBuilder(nativeMemoryManager) val ingestTime = 1594130687316L - val rows = Seq(TupleRowReader((Some(firstSampleTime), Some(0.0d)))) - val chunksets = for { i <- 0 until 1050 } yield { - val partKey = partBuilder.partKeyFromObjects(Schemas.gauge, gaugeName + i, seriesTags) + val chunksets = for { + i <- 0 until 1050 + partKey = partBuilder.partKeyFromObjects(Schemas.gauge, gaugeName + i, seriesTags) + c <- 0 until 3 + } yield { + val rows = Seq(TupleRowReader((Some(firstSampleTime + c), Some(0.0d)))) ChunkSet(Schemas.gauge.data, partKey, ingestTime, rows, nativeMemoryManager) } colStore.write(promDataset.ref, Observable.fromIterable(chunksets)).futureValue @@ -260,7 +263,7 @@ class CassandraColumnStoreSpec extends ColumnStoreSpec { ingestTime - 1, ingestTime + 1, firstSampleTime - 1, - firstSampleTime + 1, + firstSampleTime + 5, 10L, 100 ).toList @@ -268,6 +271,7 @@ class CassandraColumnStoreSpec extends ColumnStoreSpec { batches.size shouldEqual 11 // 100 rows per batch, 1050 rows => 11 batches batches.zipWithIndex.foreach { case (b, i) => b.size shouldEqual (if (i == 10) 50 else 100) + b.foreach(_.chunkSetsTimeOrdered.size shouldEqual 3) } } } diff --git a/core/src/main/resources/filodb-defaults.conf b/core/src/main/resources/filodb-defaults.conf index c7d1a31a4b..3c313e8d94 100644 --- a/core/src/main/resources/filodb-defaults.conf +++ b/core/src/main/resources/filodb-defaults.conf @@ -233,7 +233,7 @@ filodb { # cass-session-provider-fqcn = fqcn # Number of time series to operate on at one time. Reduce if there is much less memory available - cass-write-batch-size = 500 + cass-write-batch-size = 250 # amount of parallelism to introduce in the spark job. This controls number of spark partitions # increase if the number of splits seen in cassandra reads is low and spark jobs are slow. From 3c04bcf99890657e92af23714e9c597dfa11b022 Mon Sep 17 00:00:00 2001 From: sherali42 <43357447+sherali42@users.noreply.github.com> Date: Wed, 8 Jul 2020 11:12:00 -0700 Subject: [PATCH 22/22] misc(query): add cardinality limits to binary-join, group-by queries (#788) --- core/src/main/resources/filodb-defaults.conf | 6 ++ .../filodb.core/query/QueryContext.scala | 2 + .../main/scala/filodb/http/HttpSettings.scala | 2 + .../query/exec/AggrOverRangeVectors.scala | 17 +++- .../filodb/query/exec/BinaryJoinExec.scala | 14 ++- .../query/exec/BinaryJoinExecSpec.scala | 50 +++++++++++ .../query/exec/BinaryJoinGroupingSpec.scala | 88 ++++++++++++++++++- 7 files changed, 171 insertions(+), 8 deletions(-) diff --git a/core/src/main/resources/filodb-defaults.conf b/core/src/main/resources/filodb-defaults.conf index 3c313e8d94..6d5a7eaca3 100644 --- a/core/src/main/resources/filodb-defaults.conf +++ b/core/src/main/resources/filodb-defaults.conf @@ -150,6 +150,12 @@ filodb { # Maximum number of samples to return in a query sample-limit = 1000000 + # Binary Join Cardinality limit + join-cardinality-limit = 25000 + + # Group by Cardinality limit + group-by-cardinality-limit = 1000 + # Minimum step required for a query min-step = 5 seconds diff --git a/core/src/main/scala/filodb.core/query/QueryContext.scala b/core/src/main/scala/filodb.core/query/QueryContext.scala index bc2dc5ebd2..f15babbf3d 100644 --- a/core/src/main/scala/filodb.core/query/QueryContext.scala +++ b/core/src/main/scala/filodb.core/query/QueryContext.scala @@ -23,6 +23,8 @@ final case class QueryContext(origQueryParams: TsdbQueryParams = UnavailableProm spreadOverride: Option[SpreadProvider] = None, queryTimeoutMillis: Int = 30000, sampleLimit: Int = 1000000, + groupByCardLimit: Int = 100000, + joinQueryCardLimit: Int = 100000, shardOverrides: Option[Seq[Int]] = None, queryId: String = UUID.randomUUID().toString, submitTime: Long = System.currentTimeMillis()) diff --git a/http/src/main/scala/filodb/http/HttpSettings.scala b/http/src/main/scala/filodb/http/HttpSettings.scala index 4ac8fad6b8..39bacc438f 100644 --- a/http/src/main/scala/filodb/http/HttpSettings.scala +++ b/http/src/main/scala/filodb/http/HttpSettings.scala @@ -15,4 +15,6 @@ class HttpSettings(config: Config, val filoSettings: FilodbSettings) { lazy val queryDefaultSpread = config.getInt("filodb.spread-default") lazy val querySampleLimit = config.getInt("filodb.query.sample-limit") lazy val queryAskTimeout = config.as[FiniteDuration]("filodb.query.ask-timeout") + lazy val queryBinaryJoinCardLimit = config.getInt("filodb.query.join-cardinality-limit") + lazy val queryGroupByCardLimit = config.getInt("filodb.query.group-by-cardinality-limit") } diff --git a/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala b/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala index 6638c7ce22..fb204c266b 100644 --- a/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala +++ b/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala @@ -35,7 +35,8 @@ final case class ReduceAggregateExec(queryContext: QueryContext, val task = for { schema <- firstSchema } yield { val aggregator = RowAggregator(aggrOp, aggrParams, schema) - RangeVectorAggregator.mapReduce(aggregator, skipMapPhase = true, results, rv => rv.key) + RangeVectorAggregator.mapReduce(aggregator, skipMapPhase = true, results, rv => rv.key, + querySession.qContext.groupByCardLimit) } Observable.fromTask(task).flatten } @@ -76,10 +77,12 @@ final case class AggregateMapReduce(aggrOp: AggregationOperator, sourceSchema.fixedVectorLen.filter(_ <= querySession.queryConfig.fastReduceMaxWindows).map { numWindows => RangeVectorAggregator.fastReduce(aggregator, false, source, numWindows) }.getOrElse { - RangeVectorAggregator.mapReduce(aggregator, skipMapPhase = false, source, grouping) + RangeVectorAggregator.mapReduce(aggregator, skipMapPhase = false, source, grouping, + querySession.qContext.groupByCardLimit) } } else { - RangeVectorAggregator.mapReduce(aggregator, skipMapPhase = false, source, grouping) + RangeVectorAggregator.mapReduce(aggregator, skipMapPhase = false, source, grouping, + querySession.qContext.groupByCardLimit) } } @@ -132,11 +135,17 @@ object RangeVectorAggregator extends StrictLogging { def mapReduce(rowAgg: RowAggregator, skipMapPhase: Boolean, source: Observable[RangeVector], - grouping: RangeVector => RangeVectorKey): Observable[RangeVector] = { + grouping: RangeVector => RangeVectorKey, + cardinalityLimit: Int = Int.MaxValue): Observable[RangeVector] = { // reduce the range vectors using the foldLeft construct. This results in one aggregate per group. val task = source.toListL.map { rvs => // now reduce each group and create one result range vector per group val groupedResult = mapReduceInternal(rvs, rowAgg, skipMapPhase, grouping) + + // if group-by cardinality breaches the limit, throw exception + if (groupedResult.size > cardinalityLimit) + throw new BadQueryException(s"This query results in more than $cardinalityLimit group-by cardinality limit. " + + s"Try applying more filters") groupedResult.map { case (rvk, aggHolder) => val rowIterator = new CustomCloseCursor(aggHolder.map(_.toRowReader))(aggHolder.close()) IteratorBackedRangeVector(rvk, rowIterator) diff --git a/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala b/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala index 11efb47024..ed2404649b 100644 --- a/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala +++ b/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala @@ -61,10 +61,15 @@ final case class BinaryJoinExec(queryContext: QueryContext, protected def args: String = s"binaryOp=$binaryOp, on=$on, ignoring=$ignoring" + //scalastyle:off method.length protected[exec] def compose(childResponses: Observable[(QueryResponse, Int)], firstSchema: Task[ResultSchema], querySession: QuerySession): Observable[RangeVector] = { val taskOfResults = childResponses.map { + case (QueryResult(_, _, result), _) + if (result.size > queryContext.joinQueryCardLimit && cardinality == Cardinality.OneToOne) => + throw new BadQueryException(s"This query results in more than ${queryContext.joinQueryCardLimit} " + + s"join cardinality. Try applying more filters.") case (QueryResult(_, _, result), i) => (result, i) case (QueryError(_, ex), _) => throw ex }.toListL.map { resp => @@ -73,12 +78,10 @@ final case class BinaryJoinExec(queryContext: QueryContext, // require(resp.size == lhs.size + rhs.size, "Did not get sufficient responses for LHS and RHS") val lhsRvs = resp.filter(_._2 < lhs.size).flatMap(_._1) val rhsRvs = resp.filter(_._2 >= lhs.size).flatMap(_._1) - // figure out which side is the "one" side val (oneSide, otherSide, lhsIsOneSide) = if (cardinality == Cardinality.OneToMany) (lhsRvs, rhsRvs, true) else (rhsRvs, lhsRvs, false) - // load "one" side keys in a hashmap val oneSideMap = new mutable.HashMap[Map[Utf8Str, Utf8Str], RangeVector]() oneSide.foreach { rv => @@ -90,7 +93,6 @@ final case class BinaryJoinExec(queryContext: QueryContext, } oneSideMap.put(jk, rv) } - // keep a hashset of result range vector keys to help ensure uniqueness of result range vectors val resultKeySet = new mutable.HashSet[RangeVectorKey]() // iterate across the the "other" side which could be one or many and perform the binary operation @@ -102,6 +104,12 @@ final case class BinaryJoinExec(queryContext: QueryContext, throw new BadQueryException(s"Non-unique result vectors found for $resKey. " + s"Use grouping to create unique matching") resultKeySet.add(resKey) + + // OneToOne cardinality case is already handled. this condition handles OneToMany case + if (resultKeySet.size > queryContext.joinQueryCardLimit) + throw new BadQueryException(s"This query results in more than ${queryContext.joinQueryCardLimit} " + + s"join cardinality. Try applying more filters.") + val res = if (lhsIsOneSide) binOp(rvOne.rows, rvOther.rows) else binOp(rvOther.rows, rvOne.rows) IteratorBackedRangeVector(resKey, res) } diff --git a/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala b/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala index ce328bb810..e6d3005b7c 100644 --- a/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala @@ -9,6 +9,7 @@ import monix.execution.Scheduler.Implicits.global import monix.reactive.Observable import org.scalatest.{FunSpec, Matchers} import org.scalatest.concurrent.ScalaFutures +import org.scalatest.exceptions.TestFailedException import filodb.core.metadata.Column.ColumnType import filodb.core.query._ @@ -358,4 +359,53 @@ class BinaryJoinExecSpec extends FunSpec with Matchers with ScalaFutures { result.map(_.key).toSet.size shouldEqual 200 } + + it("should throw BadQueryException - one-to-one with ignoring - cardinality limit 1") { + val queryContext = QueryContext(joinQueryCardLimit = 1) // set join card limit to 1 + val execPlan = BinaryJoinExec(queryContext, dummyDispatcher, + Array(dummyPlan), // cannot be empty as some compose's rely on the schema + new Array[ExecPlan](1), // empty since we test compose, not execute or doExecute + BinaryOperator.ADD, + Cardinality.OneToOne, + Nil, Seq("tag2"), Nil, "__name__") + + // scalastyle:off + val lhs = QueryResult("someId", null, samplesLhsGrouping.map(rv => SerializedRangeVector(rv, schema))) + // val lhs = QueryResult("someId", null, samplesLhs.filter(rv => rv.key.labelValues.get(ZeroCopyUTF8String("tag2")).get.equals("tag1-1")).map(rv => SerializedRangeVector(rv, schema))) + val rhs = QueryResult("someId", null, samplesRhsGrouping.map(rv => SerializedRangeVector(rv, schema))) + // scalastyle:on + + // actual query results into 2 rows. since limit is 1, this results in BadQueryException + val thrown = intercept[TestFailedException] { + execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), tvSchemaTask, querySession) + .toListL.runAsync.futureValue + } + thrown.getCause.getClass shouldEqual classOf[BadQueryException] + thrown.getCause.getMessage shouldEqual "This query results in more than 1 join cardinality." + + " Try applying more filters." + } + + it("should throw BadQueryException - one-to-one with on - cardinality limit 1") { + val queryContext = QueryContext(joinQueryCardLimit = 1) // set join card limit to 1 + val execPlan = BinaryJoinExec(queryContext, dummyDispatcher, + Array(dummyPlan), // cannot be empty as some compose's rely on the schema + new Array[ExecPlan](1), // empty since we test compose, not execute or doExecute + BinaryOperator.ADD, + Cardinality.OneToOne, + Seq("tag1", "job"), Nil, Nil, "__name__") + + // scalastyle:off + val lhs = QueryResult("someId", null, samplesLhsGrouping.map(rv => SerializedRangeVector(rv, schema))) + val rhs = QueryResult("someId", null, samplesRhsGrouping.map(rv => SerializedRangeVector(rv, schema))) + // scalastyle:on + + // actual query results into 2 rows. since limit is 1, this results in BadQueryException + val thrown = intercept[TestFailedException] { + execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), tvSchemaTask, querySession) + .toListL.runAsync.futureValue + } + thrown.getCause.getClass shouldEqual classOf[BadQueryException] + thrown.getCause.getMessage shouldEqual "This query results in more than 1 join cardinality." + + " Try applying more filters." + } } diff --git a/query/src/test/scala/filodb/query/exec/BinaryJoinGroupingSpec.scala b/query/src/test/scala/filodb/query/exec/BinaryJoinGroupingSpec.scala index 3573dc11ad..59a746fc03 100644 --- a/query/src/test/scala/filodb/query/exec/BinaryJoinGroupingSpec.scala +++ b/query/src/test/scala/filodb/query/exec/BinaryJoinGroupingSpec.scala @@ -9,6 +9,7 @@ import monix.execution.Scheduler.Implicits.global import monix.reactive.Observable import org.scalatest.{FunSpec, Matchers} import org.scalatest.concurrent.ScalaFutures +import org.scalatest.exceptions.TestFailedException import filodb.core.metadata.Column.ColumnType import filodb.core.query._ @@ -118,7 +119,6 @@ class BinaryJoinGroupingSpec extends FunSpec with Matchers with ScalaFutures { ) it("should join many-to-one with on ") { - val samplesRhs2 = scala.util.Random.shuffle(sampleNodeRole.toList) // they may come out of order val execPlan = BinaryJoinExec(QueryContext(), dummyDispatcher, @@ -388,4 +388,90 @@ class BinaryJoinGroupingSpec extends FunSpec with Matchers with ScalaFutures { result.size shouldEqual 2 result.map(_.key.labelValues) sameElements(expectedLabels) shouldEqual true } + + it("should throw BadQueryException - many-to-one with on - cardinality limit 1") { + val queryContext = QueryContext(joinQueryCardLimit = 1) // set join card limit to 1 + val samplesRhs2 = scala.util.Random.shuffle(sampleNodeRole.toList) // they may come out of order + + val execPlan = BinaryJoinExec(queryContext, dummyDispatcher, + Array(dummyPlan), // cannot be empty as some compose's rely on the schema + new Array[ExecPlan](1), // empty since we test compose, not execute or doExecute + BinaryOperator.MUL, + Cardinality.ManyToOne, + Seq("instance"), Nil, Seq("role"), "__name__") + + // scalastyle:off + val lhs = QueryResult("someId", null, sampleNodeCpu.map(rv => SerializedRangeVector(rv, schema))) + val rhs = QueryResult("someId", null, samplesRhs2.map(rv => SerializedRangeVector(rv, schema))) + // scalastyle:on + + // actual query results into 2 rows. since limit is 1, this results in BadQueryException + val thrown = intercept[TestFailedException] { + execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), tvSchemaTask, querySession) + .toListL.runAsync.futureValue + } + + thrown.getCause.getClass shouldEqual classOf[BadQueryException] + thrown.getCause.getMessage shouldEqual "This query results in more than 1 join cardinality." + + " Try applying more filters." + } + + it("should throw BadQueryException - many-to-one with ignoring - cardinality limit 1") { + val queryContext = QueryContext(joinQueryCardLimit = 1) // set join card limit to 1 + val samplesRhs2 = scala.util.Random.shuffle(sampleNodeRole.toList) // they may come out of order + + val execPlan = BinaryJoinExec(queryContext, dummyDispatcher, + Array(dummyPlan), + new Array[ExecPlan](1), + BinaryOperator.MUL, + Cardinality.ManyToOne, + Nil, Seq("role", "mode"), Seq("role"), "__name__") + + // scalastyle:off + val lhs = QueryResult("someId", null, sampleNodeCpu.map(rv => SerializedRangeVector(rv, schema))) + val rhs = QueryResult("someId", null, samplesRhs2.map(rv => SerializedRangeVector(rv, schema))) + // scalastyle:on + + // actual query results into 2 rows. since limit is 1, this results in BadQueryException + val thrown = intercept[TestFailedException] { + execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), tvSchemaTask, querySession) + .toListL.runAsync.futureValue + } + + thrown.getCause.getClass shouldEqual classOf[BadQueryException] + thrown.getCause.getMessage shouldEqual "This query results in more than 1 join cardinality." + + " Try applying more filters." + } + + it("should throw BadQueryException - many-to-one with by and grouping without arguments - cardinality limit 1") { + val queryContext = QueryContext(joinQueryCardLimit = 3) // set join card limit to 3 + val agg = RowAggregator(AggregationOperator.Sum, Nil, tvSchema) + val aggMR = AggregateMapReduce(AggregationOperator.Sum, Nil, Nil, Seq("instance", "job")) + val mapped = aggMR(Observable.fromIterable(sampleNodeCpu), querySession, 1000, tvSchema) + + val resultObs4 = RangeVectorAggregator.mapReduce(agg, true, mapped, rv=>rv.key) + val samplesRhs = resultObs4.toListL.runAsync.futureValue + + val execPlan = BinaryJoinExec(queryContext, dummyDispatcher, + Array(dummyPlan), + new Array[ExecPlan](1), + BinaryOperator.DIV, + Cardinality.ManyToOne, + Seq("instance"), Nil, Nil, "__name__") + + // scalastyle:off + val lhs = QueryResult("someId", null, sampleNodeCpu.map(rv => SerializedRangeVector(rv, schema))) + val rhs = QueryResult("someId", null, samplesRhs.map(rv => SerializedRangeVector(rv, schema))) + // scalastyle:on + + // actual query results into 4 rows. since limit is 3, this results in BadQueryException + val thrown = intercept[TestFailedException] { + execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), tvSchemaTask, querySession) + .toListL.runAsync.futureValue + } + + thrown.getCause.getClass shouldEqual classOf[BadQueryException] + thrown.getCause.getMessage shouldEqual "This query results in more than 3 join cardinality." + + " Try applying more filters." + } }