From 6d05beed5a5b072c5142c6be2eef2100d07e6f0f Mon Sep 17 00:00:00 2001 From: TanviBhavsar Date: Thu, 22 Oct 2020 14:57:56 -0700 Subject: [PATCH 01/33] feat(query): skip aggregate present for shard key regex aggregate query (#896) --- .../columnstore/OdpSpec.scala | 9 +- cli/src/main/scala/filodb.cli/CliMain.scala | 7 +- .../scala/filodb.coordinator/QueryActor.scala | 4 +- .../filodb.coordinator/client/QueryOps.scala | 2 +- .../queryengine/Utils.scala | 2 +- .../HighAvailabilityPlanner.scala | 15 +- .../queryplanner/MultiPartitionPlanner.scala | 27 +-- .../queryplanner/ShardKeyRegexPlanner.scala | 3 +- .../queryplanner/SingleClusterPlanner.scala | 11 +- .../ClusterRecoverySpec.scala | 4 +- .../NodeCoordinatorActorSpec.scala | 6 +- .../client/SerializationSpec.scala | 2 +- .../HighAvailabilityPlannerSpec.scala | 75 +++++--- .../MultiPartitionPlannerSpec.scala | 179 ++++++++++-------- .../ShardKeyRegexPlannerSpec.scala | 17 +- .../SingleClusterPlannerSpec.scala | 30 ++- .../SingleClusterPlannerSplitSpec.scala | 26 +-- .../SinglePartitionPlannerSpec.scala | 6 +- .../filodb.core/query/QueryContext.scala | 34 ++-- .../scala/filodb.core/query/ResultTypes.scala | 1 + .../filodb.core/store/ChunkSetInfo.scala | 2 +- .../filodb/http/PrometheusApiRoute.scala | 4 +- .../filodb.jmh/HistogramQueryBenchmark.scala | 7 +- .../prometheus/query/PrometheusModel.scala | 16 ++ .../scala/filodb/query/PromCirceSupport.scala | 31 +++ .../filodb/query/PromQueryResponse.scala | 10 +- .../query/exec/AggrOverRangeVectors.scala | 7 +- .../filodb/query/exec/BinaryJoinExec.scala | 12 +- .../scala/filodb/query/exec/ExecPlan.scala | 16 +- .../filodb/query/exec/MetadataExecPlan.scala | 15 +- .../query/exec/MetadataRemoteExec.scala | 3 +- .../exec/MultiSchemaPartitionsExec.scala | 2 +- .../filodb/query/exec/PlanDispatcher.scala | 2 +- .../filodb/query/exec/PromQlRemoteExec.scala | 123 ++++++++---- .../query/exec/ScalarFixedDoubleExec.scala | 2 +- .../query/exec/TimeScalarGeneratorExec.scala | 2 +- .../filodb/query/PromCirceSupportSpec.scala | 112 +++++++++++ .../query/exec/BinaryJoinExecSpec.scala | 4 +- .../query/exec/BinaryJoinGroupingSpec.scala | 6 +- .../filodb/query/exec/MetadataExecSpec.scala | 6 +- .../exec/MultiSchemaPartitionsExecSpec.scala | 4 +- .../query/exec/PromQlRemoteExecSpec.scala | 12 +- .../downsampler/DownsamplerMainSpec.scala | 14 +- 43 files changed, 571 insertions(+), 301 deletions(-) create mode 100644 query/src/test/scala/filodb/query/PromCirceSupportSpec.scala diff --git a/cassandra/src/test/scala/filodb.cassandra/columnstore/OdpSpec.scala b/cassandra/src/test/scala/filodb.cassandra/columnstore/OdpSpec.scala index f78c307fa1..1fa0fb8647 100644 --- a/cassandra/src/test/scala/filodb.cassandra/columnstore/OdpSpec.scala +++ b/cassandra/src/test/scala/filodb.cassandra/columnstore/OdpSpec.scala @@ -1,14 +1,12 @@ 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 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} @@ -16,7 +14,7 @@ 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.{ColumnFilter, PlannerParams, QueryConfig, QueryContext, QuerySession} import filodb.core.query.Filter.Equals import filodb.core.store.{InMemoryMetaStore, PartKeyRecord, StoreConfig, TimeRangeChunkScan} import filodb.memory.format.ZeroCopyUTF8String._ @@ -187,8 +185,9 @@ class OdpSpec extends AnyFunSpec with Matchers with BeforeAndAfterAll with Scala 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 exec = MultiSchemaPartitionsExec(QueryContext(plannerParams = PlannerParams(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/cli/src/main/scala/filodb.cli/CliMain.scala b/cli/src/main/scala/filodb.cli/CliMain.scala index 0a7774ad2e..130be21a8a 100644 --- a/cli/src/main/scala/filodb.cli/CliMain.scala +++ b/cli/src/main/scala/filodb.cli/CliMain.scala @@ -349,9 +349,10 @@ object CliMain extends FilodbClusterNode { options: QOptions, tsdbQueryParams: TsdbQueryParams): Unit = { val ref = DatasetRef(dataset) val spreadProvider: Option[SpreadProvider] = options.spread.map(s => StaticSpreadProvider(SpreadChange(0, s))) - val qOpts = QueryContext(tsdbQueryParams, spreadProvider, options.sampleLimit) - .copy(queryTimeoutMillis = options.timeout.toMillis.toInt, - shardOverrides = options.shardOverrides) + + val qOpts = QueryContext(origQueryParams = tsdbQueryParams, plannerParams = PlannerParams(spreadOverride = + spreadProvider, sampleLimit = options.sampleLimit, queryTimeoutMillis = options.timeout.toMillis.toInt, shardOverrides = + options.shardOverrides)) println(s"Sending query command to server for $ref with options $qOpts...") println(s"Query Plan:\n$plan") options.everyN match { diff --git a/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala b/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala index 5fb1e366b8..1b9035599d 100644 --- a/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala +++ b/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala @@ -205,8 +205,8 @@ final class QueryActor(memStore: MemStore, def checkTimeout(queryContext: QueryContext, replyTo: ActorRef): Boolean = { // timeout can occur here if there is a build up in actor mailbox queue and delayed delivery val queryTimeElapsed = System.currentTimeMillis() - queryContext.submitTime - if (queryTimeElapsed >= queryContext.queryTimeoutMillis) { - replyTo ! QueryError(s"Query timeout, $queryTimeElapsed ms > ${queryContext.queryTimeoutMillis}", + if (queryTimeElapsed >= queryContext.plannerParams.queryTimeoutMillis) { + replyTo ! QueryError(s"Query timeout, $queryTimeElapsed ms > ${queryContext.plannerParams.queryTimeoutMillis}", QueryTimeoutException(queryTimeElapsed, this.getClass.getName)) false } else true diff --git a/coordinator/src/main/scala/filodb.coordinator/client/QueryOps.scala b/coordinator/src/main/scala/filodb.coordinator/client/QueryOps.scala index cd29d6d54e..055dc9254c 100644 --- a/coordinator/src/main/scala/filodb.coordinator/client/QueryOps.scala +++ b/coordinator/src/main/scala/filodb.coordinator/client/QueryOps.scala @@ -58,7 +58,7 @@ trait QueryOps extends ClientBase with StrictLogging { // NOTE: It's very important to extend the query timeout for the ask itself, because the queryTimeoutMillis is // the internal FiloDB scatter-gather timeout. We need additional time for the proper error to get transmitted // back in case of internal timeouts. - askCoordinator(qCmd, (qContext.queryTimeoutMillis + 10000).millis) { case r: QueryResponse2 => r } + askCoordinator(qCmd, (qContext.plannerParams.queryTimeoutMillis + 10000).millis) { case r: QueryResponse2 => r } } } \ No newline at end of file diff --git a/coordinator/src/main/scala/filodb.coordinator/queryengine/Utils.scala b/coordinator/src/main/scala/filodb.coordinator/queryengine/Utils.scala index 082607322e..5530b510d0 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryengine/Utils.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryengine/Utils.scala @@ -60,7 +60,7 @@ object Utils extends StrictLogging { case FilteredPartitionQuery(filters) => // get limited # of shards if shard key available, otherwise query all shards // TODO: monitor ratio of queries using shardKeyHash to queries that go to all shards - val shards = options.shardOverrides.getOrElse { + val shards = options.plannerParams.shardOverrides.getOrElse { val shardCols = dataset.options.shardKeyColumns if (shardCols.length > 0) { shardHashFromFilters(filters, shardCols, dataset) match { diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/HighAvailabilityPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/HighAvailabilityPlanner.scala index ba4a7c7108..a8c0328a2a 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/HighAvailabilityPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/HighAvailabilityPlanner.scala @@ -71,19 +71,20 @@ class HighAvailabilityPlanner(dsRef: DatasetRef, val queryParams = qContext.origQueryParams.asInstanceOf[PromQlQueryParams] // Divide by 1000 to convert millis to seconds. PromQL params are in seconds. val promQlParams = PromQlQueryParams(queryParams.promQl, - (timeRange.startMs + offsetMs) / 1000, queryParams.stepSecs, (timeRange.endMs + offsetMs) / 1000, - queryParams.spread, processFailure = false) + (timeRange.startMs + offsetMs) / 1000, queryParams.stepSecs, (timeRange.endMs + offsetMs) / 1000) + val newQueryContext = qContext.copy(origQueryParams = promQlParams, plannerParams = qContext.plannerParams. + copy(processFailure = false) ) logger.debug("PromQlExec params:" + promQlParams) val httpEndpoint = remoteHttpEndpoint + queryParams.remoteQueryPath.getOrElse("") rootLogicalPlan match { case lp: LabelValues => MetadataRemoteExec(httpEndpoint, remoteHttpTimeoutMs, - PlannerUtil.getLabelValuesUrlParams(lp, queryParams), qContext, - InProcessPlanDispatcher, dsRef, promQlParams) + PlannerUtil.getLabelValuesUrlParams(lp, queryParams), newQueryContext, + InProcessPlanDispatcher, dsRef) case lp: SeriesKeysByFilters => val urlParams = Map("match[]" -> queryParams.promQl) MetadataRemoteExec(httpEndpoint, remoteHttpTimeoutMs, - urlParams, qContext, InProcessPlanDispatcher, dsRef, promQlParams) + urlParams, newQueryContext, InProcessPlanDispatcher, dsRef) case _ => PromQlRemoteExec(httpEndpoint, remoteHttpTimeoutMs, - qContext, InProcessPlanDispatcher, dsRef, promQlParams) + newQueryContext, InProcessPlanDispatcher, dsRef) } } @@ -111,7 +112,7 @@ 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 + && !qContext.plannerParams.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 diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala index 2b1cbd5392..00b34ece4e 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala @@ -33,7 +33,7 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider if(!tsdbQueryParams.isInstanceOf[PromQlQueryParams] || // We don't know the promql issued (unusual) (tsdbQueryParams.isInstanceOf[PromQlQueryParams] - && !tsdbQueryParams.asInstanceOf[PromQlQueryParams].processMultiPartition)) // Query was part of routing + && !qContext.plannerParams.processMultiPartition)) // Query was part of routing localPartitionPlanner.materialize(logicalPlan, qContext) else logicalPlan match { @@ -51,9 +51,10 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider .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, - queryParams.remoteQueryPath, queryParams.processFailure, processMultiPartition = false, queryParams.verbose) + private def generateRemoteExecParams(queryContext: QueryContext, startMs: Long, endMs: Long) = { + val queryParams = queryContext.origQueryParams.asInstanceOf[PromQlQueryParams] + queryContext.copy(origQueryParams = queryParams.copy(startSecs = startMs/1000, endSecs = endMs / 1000), + plannerParams = queryContext.plannerParams.copy(processMultiPartition = false)) } /** @@ -116,8 +117,8 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider 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)) + PromQlRemoteExec(httpEndpoint, remoteHttpTimeoutMs, generateRemoteExecParams(qContext, startMs, endMs), + InProcessPlanDispatcher, dataset.ref) } } if (execPlans.size == 1) execPlans.head @@ -142,8 +143,9 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider 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)) + PromQlRemoteExec(httpEndpoint, remoteHttpTimeoutMs, generateRemoteExecParams(qContext, + queryParams.startSecs * 1000, queryParams.endSecs * 1000), InProcessPlanDispatcher, dataset.ref + ) } } else throw new UnsupportedOperationException("Binary Join across multiple partitions not supported") @@ -184,10 +186,11 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider 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("") + val finalQueryContext = generateRemoteExecParams( + qContext, partitionAssignment.timeRange.startMs, partitionAssignment.timeRange.endMs) + val httpEndpoint = partitionAssignment.endPoint + finalQueryContext.origQueryParams.asInstanceOf[PromQlQueryParams]. + remoteQueryPath.getOrElse("") MetadataRemoteExec(httpEndpoint, remoteHttpTimeoutMs, - urlParams, qContext, InProcessPlanDispatcher, dataset.ref, finalQueryParams) + urlParams, finalQueryContext, InProcessPlanDispatcher, dataset.ref) } } diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlanner.scala index b1ccf0ecb6..568cc9f230 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlanner.scala @@ -71,7 +71,8 @@ class ShardKeyRegexPlanner(dataset: Dataset, // For example for exp(sum(test{_ws_ = "demo", _ns_ =~ "App.*"})), sub queries should be // sum(test{_ws_ = "demo", _ns_ = "App-1"}), sum(test{_ws_ = "demo", _ns_ = "App-2"}) etc val newQueryParams = queryParams.copy(promQl = LogicalPlanParser.convertToQuery(newLogicalPlan)) - val newQueryContext = qContext.copy(origQueryParams = newQueryParams) + val newQueryContext = qContext.copy(origQueryParams = newQueryParams, plannerParams = qContext.plannerParams. + copy(skipAggregatePresent = true)) queryPlanner.materialize(logicalPlan.replaceFilters(result), newQueryContext) } } diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala index 09e4fc0739..c708e91162 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala @@ -106,12 +106,12 @@ class SingleClusterPlanner(dsRef: DatasetRef, private def shardsFromFilters(filters: Seq[ColumnFilter], qContext: QueryContext): Seq[Int] = { - val spreadProvToUse = qContext.spreadOverride.getOrElse(spreadProvider) + val spreadProvToUse = qContext.plannerParams.spreadOverride.getOrElse(spreadProvider) - require(shardColumns.nonEmpty || qContext.shardOverrides.nonEmpty, + require(shardColumns.nonEmpty || qContext.plannerParams.shardOverrides.nonEmpty, s"Dataset $dsRef does not have shard columns defined, and shard overrides were not mentioned") - qContext.shardOverrides.getOrElse { + qContext.plannerParams.shardOverrides.getOrElse { val shardVals = shardColumns.map { shardCol => // So to compute the shard hash we need shardCol == value filter (exact equals) for each shardColumn filters.find(f => f.column == shardCol) match { @@ -258,7 +258,8 @@ class SingleClusterPlanner(dsRef: DatasetRef, val reduceDispatcher = pickDispatcher(toReduceLevel2) val reducer = LocalPartitionReduceAggregateExec(qContext, reduceDispatcher, toReduceLevel2, lp.operator, lp.params) - reducer.addRangeVectorTransformer(AggregatePresenter(lp.operator, lp.params)) + if (!qContext.plannerParams.skipAggregatePresent) + reducer.addRangeVectorTransformer(AggregatePresenter(lp.operator, lp.params)) PlanResult(Seq(reducer), false) // since we have aggregated, no stitching } @@ -342,7 +343,7 @@ class SingleClusterPlanner(dsRef: DatasetRef, private def materializeRawSeries(qContext: QueryContext, lp: RawSeries): PlanResult = { - val spreadProvToUse = qContext.spreadOverride.getOrElse(spreadProvider) + val spreadProvToUse = qContext.plannerParams.spreadOverride.getOrElse(spreadProvider) val offsetMillis: Long = lp.offsetMs.getOrElse(0) val colName = lp.columns.headOption val (renamedFilters, schemaOpt) = extractSchemaFilter(renameMetricFilter(lp.filters)) diff --git a/coordinator/src/multi-jvm/scala/filodb.coordinator/ClusterRecoverySpec.scala b/coordinator/src/multi-jvm/scala/filodb.coordinator/ClusterRecoverySpec.scala index 31ff578857..212c80edc2 100644 --- a/coordinator/src/multi-jvm/scala/filodb.coordinator/ClusterRecoverySpec.scala +++ b/coordinator/src/multi-jvm/scala/filodb.coordinator/ClusterRecoverySpec.scala @@ -10,7 +10,7 @@ import com.typesafe.config.ConfigFactory import org.scalatest.time.{Millis, Seconds, Span} import filodb.core._ import filodb.core.metadata.Column.ColumnType -import filodb.core.query.{ColumnInfo, QueryContext} +import filodb.core.query.{ColumnInfo, PlannerParams, QueryContext} object ClusterRecoverySpecConfig extends MultiNodeConfig { // register the named roles (nodes) of the test @@ -145,7 +145,7 @@ abstract class ClusterRecoverySpec extends ClusterSpec(ClusterRecoverySpecConfig // val query = LogicalPlanQuery(dataset6.ref, // simpleAgg("count", childPlan=PartitionsRange.all(FilteredPartitionQuery(Nil), Seq("MonthYear")))) - val qOpt = QueryContext(shardOverrides = Some(Seq(0, 1))) + val qOpt = QueryContext(plannerParams = PlannerParams(shardOverrides = Some(Seq(0, 1)))) val q2 = LogicalPlan2Query(dataset6.ref, PeriodicSeriesWithWindowing( RawSeries(AllChunksSelector, Nil, Seq("AvgTone"), Some(300000), None), diff --git a/coordinator/src/test/scala/filodb.coordinator/NodeCoordinatorActorSpec.scala b/coordinator/src/test/scala/filodb.coordinator/NodeCoordinatorActorSpec.scala index 48b70cf1ae..71affd04f8 100644 --- a/coordinator/src/test/scala/filodb.coordinator/NodeCoordinatorActorSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/NodeCoordinatorActorSpec.scala @@ -122,7 +122,7 @@ class NodeCoordinatorActorSpec extends ActorTest(NodeCoordinatorActorSpec.getNew val timeMinSchema = ResultSchema(Seq(ColumnInfo("timestamp", TimestampColumn), ColumnInfo("min", DoubleColumn)), 1) val countSchema = ResultSchema(Seq(ColumnInfo("timestamp", TimestampColumn), ColumnInfo("count", DoubleColumn)), 1) val valueSchema = ResultSchema(Seq(ColumnInfo("timestamp", TimestampColumn), ColumnInfo("value", DoubleColumn)), 1) - val qOpt = QueryContext(shardOverrides = Some(Seq(0))) + val qOpt = QueryContext(plannerParams = PlannerParams(shardOverrides = Some(Seq(0)))) describe("QueryActor commands and responses") { import MachineMetricsData._ @@ -275,7 +275,7 @@ class NodeCoordinatorActorSpec extends ActorTest(NodeCoordinatorActorSpec.getNew // Should return results from both shards // shard 1 - timestamps 110000 -< 130000; shard 2 - timestamps 130000 <- 1400000 - val queryOpt = QueryContext(shardOverrides = Some(Seq(0, 1))) + val queryOpt = QueryContext(plannerParams = PlannerParams(shardOverrides = Some(Seq(0, 1)))) val series2 = (2 to 4).map(n => s"Series $n").toSet.asInstanceOf[Set[Any]] val multiFilter = Seq(ColumnFilter("series", Filter.In(series2))) val q2 = LogicalPlan2Query(ref, @@ -302,7 +302,7 @@ class NodeCoordinatorActorSpec extends ActorTest(NodeCoordinatorActorSpec.getNew memStore.refreshIndexForTesting(dataset1.ref) - val queryOpt = QueryContext(shardOverrides = Some(Seq(0, 1))) + val queryOpt = QueryContext(plannerParams = PlannerParams(shardOverrides = Some(Seq(0, 1)))) val series2 = (2 to 4).map(n => s"Series $n") val multiFilter = Seq(ColumnFilter("series", Filter.In(series2.toSet.asInstanceOf[Set[Any]]))) val q2 = LogicalPlan2Query(ref, RawSeries(AllChunksSelector, multiFilter, Seq("min"), Some(300000), None), diff --git a/coordinator/src/test/scala/filodb.coordinator/client/SerializationSpec.scala b/coordinator/src/test/scala/filodb.coordinator/client/SerializationSpec.scala index 876183ce46..dc9d9a5ca1 100644 --- a/coordinator/src/test/scala/filodb.coordinator/client/SerializationSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/client/SerializationSpec.scala @@ -337,7 +337,7 @@ class SerializationSpec extends ActorTest(SerializationSpecConfig.getNewSystem) qParams) val param = PromQlQueryParams("test", 1000, 200, 5000) val execPlan = engine.materialize(logicalPlan, QueryContext(origQueryParams = param, - spreadOverride = Some(new StaticSpreadProvider(SpreadChange(0, 0))))) + plannerParams = PlannerParams(spreadOverride = Some(new StaticSpreadProvider(SpreadChange(0, 0)))))) roundTrip(execPlan) shouldEqual execPlan } } diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/HighAvailabilityPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/HighAvailabilityPlannerSpec.scala index 602b43ca79..ad30fd05f9 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/HighAvailabilityPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/HighAvailabilityPlannerSpec.scala @@ -113,8 +113,9 @@ class HighAvailabilityPlannerSpec extends AnyFunSpec with Matchers { val execPlan = engine.materialize(summed, QueryContext(origQueryParams = promQlQueryParams)) execPlan.isInstanceOf[PromQlRemoteExec] shouldEqual (true) - execPlan.asInstanceOf[PromQlRemoteExec].params.startSecs shouldEqual(from/1000) - execPlan.asInstanceOf[PromQlRemoteExec].params.endSecs shouldEqual(to/1000) + val queryParams = execPlan.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams] + queryParams.startSecs shouldEqual(from/1000) + queryParams.endSecs shouldEqual(to/1000) } it("should generate RemoteExecPlan with RawSeries time according to lookBack") { @@ -162,10 +163,12 @@ class HighAvailabilityPlannerSpec extends AnyFunSpec with Matchers { l1.rangeVectorTransformers(0).asInstanceOf[PeriodicSamplesMapper].end shouldEqual (2000000) l1.rangeVectorTransformers(1).isInstanceOf[AggregateMapReduce] shouldEqual true } + val queryParams = child2.queryContext.origQueryParams. + asInstanceOf[PromQlQueryParams] - child2.params.startSecs shouldEqual from/1000 - child2.params.endSecs shouldEqual (1060000-1)/1000 - child2.params.processFailure shouldEqual(false) + queryParams.startSecs shouldEqual from/1000 + queryParams.endSecs shouldEqual (1060000-1)/1000 + child2.queryContext.plannerParams.processFailure shouldEqual(false) } it("should generate only PromQlExec when local failure starts before query time") { @@ -188,8 +191,10 @@ class HighAvailabilityPlannerSpec extends AnyFunSpec with Matchers { val execPlan = engine.materialize(summed, QueryContext(origQueryParams = promQlQueryParams)) execPlan.isInstanceOf[PromQlRemoteExec] shouldEqual (true) - execPlan.asInstanceOf[PromQlRemoteExec].params.startSecs shouldEqual(from/1000) - execPlan.asInstanceOf[PromQlRemoteExec].params.endSecs shouldEqual(to/1000) + val queryParams = execPlan.asInstanceOf[PromQlRemoteExec].queryContext.origQueryParams. + asInstanceOf[PromQlQueryParams] + queryParams.startSecs shouldEqual(from/1000) + queryParams.endSecs shouldEqual(to/1000) } it("should generate only PromQlExec when local failure timerange coincide with query time range") { @@ -209,11 +214,13 @@ class HighAvailabilityPlannerSpec extends AnyFunSpec with Matchers { val engine = new HighAvailabilityPlanner(dsRef, localPlanner, failureProvider, queryConfig) - val execPlan = engine.materialize(summed, QueryContext(origQueryParams = promQlQueryParams)) + val execPlan = engine.materialize(summed, QueryContext(origQueryParams = promQlQueryParams)) execPlan.isInstanceOf[PromQlRemoteExec] shouldEqual (true) - execPlan.asInstanceOf[PromQlRemoteExec].params.startSecs shouldEqual(from/1000) - execPlan.asInstanceOf[PromQlRemoteExec].params.endSecs shouldEqual(to/1000) + val queryParams = execPlan.asInstanceOf[PromQlRemoteExec].queryContext.origQueryParams. + asInstanceOf[PromQlQueryParams] + queryParams.startSecs shouldEqual(from/1000) + queryParams.endSecs shouldEqual(to/1000) } it("should generate only PromQlExec when local failure starts before query end time and ends after query end time") { @@ -236,8 +243,10 @@ class HighAvailabilityPlannerSpec extends AnyFunSpec with Matchers { val execPlan = engine.materialize(summed, QueryContext(origQueryParams = promQlQueryParams)) execPlan.isInstanceOf[PromQlRemoteExec] shouldEqual (true) - execPlan.asInstanceOf[PromQlRemoteExec].params.startSecs shouldEqual(from/1000) - execPlan.asInstanceOf[PromQlRemoteExec].params.endSecs shouldEqual(to/1000) + val queryParams =execPlan.asInstanceOf[PromQlRemoteExec].queryContext.origQueryParams. + asInstanceOf[PromQlQueryParams] + queryParams.startSecs shouldEqual(from/1000) + queryParams.endSecs shouldEqual(to/1000) } it("should generate PromQlExecPlan and LocalPlan with RawSeries time according to lookBack and step") { @@ -289,10 +298,12 @@ class HighAvailabilityPlannerSpec extends AnyFunSpec with Matchers { l1.rangeVectorTransformers(1).isInstanceOf[AggregateMapReduce] shouldEqual true } - child2.params.startSecs shouldEqual 900 - child2.params.endSecs shouldEqual 1020 - child2.params.stepSecs shouldEqual 60 - child2.params.processFailure shouldEqual(false) + val queryParams = child2.asInstanceOf[PromQlRemoteExec].queryContext.origQueryParams. + asInstanceOf[PromQlQueryParams] + queryParams.startSecs shouldEqual 900 + queryParams.endSecs shouldEqual 1020 + queryParams.stepSecs shouldEqual 60 + child2.asInstanceOf[PromQlRemoteExec].queryContext.plannerParams.processFailure shouldEqual(false) } it("should generate only PromQlExecPlan when second remote ends after query end time") { @@ -321,11 +332,12 @@ class HighAvailabilityPlannerSpec extends AnyFunSpec with Matchers { execPlan.isInstanceOf[PromQlRemoteExec] shouldEqual true val child = execPlan.asInstanceOf[PromQlRemoteExec] - - child.params.startSecs shouldEqual 900 - child.params.endSecs shouldEqual 1980 - child.params.stepSecs shouldEqual 60 - child.params.processFailure shouldEqual false + val queryParams = child.asInstanceOf[PromQlRemoteExec].queryContext.origQueryParams. + asInstanceOf[PromQlQueryParams] + queryParams.startSecs shouldEqual 900 + queryParams.endSecs shouldEqual 1980 + queryParams.stepSecs shouldEqual 60 + child.asInstanceOf[PromQlRemoteExec].queryContext.plannerParams.processFailure shouldEqual false } it("should not do routing for InstantQueries when there are local and remote failures") { @@ -393,10 +405,12 @@ class HighAvailabilityPlannerSpec extends AnyFunSpec with Matchers { execPlan.isInstanceOf[PromQlRemoteExec] shouldEqual true val child = execPlan.asInstanceOf[PromQlRemoteExec] - child.params.startSecs shouldEqual from - child.params.endSecs shouldEqual to - child.params.stepSecs shouldEqual step - child.params.processFailure shouldEqual false + val queryParams = child.asInstanceOf[PromQlRemoteExec].queryContext.origQueryParams. + asInstanceOf[PromQlQueryParams] + queryParams.startSecs shouldEqual from + queryParams.endSecs shouldEqual to + queryParams.stepSecs shouldEqual step + child.asInstanceOf[PromQlRemoteExec].queryContext.plannerParams.processFailure shouldEqual false } it("should work with offset") { @@ -420,8 +434,10 @@ class HighAvailabilityPlannerSpec extends AnyFunSpec with Matchers { // So PromQlExec is generated instead of local LocalPartitionDistConcatExec. PromQlExec will have original query and start time // Start time with offset will be calculated by buddy pod execPlan2.isInstanceOf[PromQlRemoteExec] shouldEqual (true) - execPlan2.asInstanceOf[PromQlRemoteExec].params.startSecs shouldEqual(700) - execPlan2.asInstanceOf[PromQlRemoteExec].params.endSecs shouldEqual(10000) + val queryParams = execPlan2.asInstanceOf[PromQlRemoteExec].queryContext.origQueryParams. + asInstanceOf[PromQlQueryParams] + queryParams.startSecs shouldEqual(700) + queryParams.endSecs shouldEqual(10000) } it("should generate PromQlExec for metadata queries") { @@ -443,8 +459,9 @@ class HighAvailabilityPlannerSpec extends AnyFunSpec with Matchers { val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) execPlan.isInstanceOf[MetadataRemoteExec] shouldEqual (true) - execPlan.asInstanceOf[MetadataRemoteExec].params.startSecs shouldEqual (from) - execPlan.asInstanceOf[MetadataRemoteExec].params.endSecs shouldEqual (to) + val queryParams = execPlan.asInstanceOf[MetadataRemoteExec].queryContext.origQueryParams.asInstanceOf[PromQlQueryParams] + queryParams.startSecs shouldEqual (from) + queryParams.endSecs shouldEqual (to) } } diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala index 66ddffc688..ced9bdacc6 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala @@ -5,11 +5,10 @@ import akka.testkit.TestProbe import com.typesafe.config.ConfigFactory import org.scalatest.funspec.AnyFunSpec import org.scalatest.matchers.should.Matchers - import filodb.coordinator.ShardMapper import filodb.core.MetricsTestData import filodb.core.metadata.Schemas -import filodb.core.query.{PromQlQueryParams, QueryConfig, QueryContext} +import filodb.core.query.{PlannerParams, PromQlQueryParams, QueryConfig, QueryContext} import filodb.prometheus.ast.TimeStepParams import filodb.prometheus.parse.Parser import filodb.query.LogicalPlan @@ -58,9 +57,10 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers { 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 promQlQueryParams = PromQlQueryParams("test{job = \"app\"}", 1000, 100, 2000) - val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams, plannerParams = + PlannerParams(processMultiPartition = true))) execPlan.isInstanceOf[LocalPartitionDistConcatExec] shouldEqual (true) execPlan.children.length shouldEqual 2 @@ -87,23 +87,23 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers { 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 promQlQueryParams = PromQlQueryParams("test{job = \"app\"}", startSeconds, step, endSeconds) - val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams, plannerParams = + PlannerParams(processMultiPartition = true))) val stitchRvsExec = execPlan.asInstanceOf[StitchRvsExec] stitchRvsExec.children.size shouldEqual (2) stitchRvsExec.children(0).isInstanceOf[PromQlRemoteExec] shouldEqual true stitchRvsExec.children(1).isInstanceOf[PromQlRemoteExec] shouldEqual true - val remoteExec1 = stitchRvsExec.children(0).asInstanceOf[PromQlRemoteExec] - remoteExec1.params.startSecs shouldEqual startSeconds - remoteExec1.params.endSecs shouldEqual (localPartitionStart - 1) - remoteExec1.params.stepSecs shouldEqual step - remoteExec1.params.processFailure shouldEqual true - remoteExec1.params.processMultiPartition shouldEqual false + val queryParams1 = remoteExec1.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams] + queryParams1.startSecs shouldEqual startSeconds + queryParams1.endSecs shouldEqual (localPartitionStart - 1) + queryParams1.stepSecs shouldEqual step + remoteExec1.queryContext.plannerParams.processFailure shouldEqual true + remoteExec1.queryContext.plannerParams.processMultiPartition shouldEqual false remoteExec1.queryEndpoint shouldEqual "remote-url" val expectedStartMs = ((startSeconds*1000) to (endSeconds*1000) by (step*1000)).find { instant => @@ -111,11 +111,12 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers { }.get val remoteExec2 = stitchRvsExec.children(1).asInstanceOf[PromQlRemoteExec] - remoteExec2.params.startSecs shouldEqual (expectedStartMs / 1000) - remoteExec2.params.endSecs shouldEqual endSeconds - remoteExec2.params.stepSecs shouldEqual step - remoteExec2.params.processFailure shouldEqual true - remoteExec2.params.processMultiPartition shouldEqual false + val queryParams2 = remoteExec2.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams] + queryParams2.startSecs shouldEqual (expectedStartMs / 1000) + queryParams2.endSecs shouldEqual endSeconds + queryParams2.stepSecs shouldEqual step + remoteExec2.queryContext.plannerParams.processFailure shouldEqual true + remoteExec2.queryContext.plannerParams.processMultiPartition shouldEqual false remoteExec2.queryEndpoint shouldEqual "remote-url2" } @@ -132,9 +133,10 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers { 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 promQlQueryParams = PromQlQueryParams("time()", 1000, 100, 2000) - val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams, plannerParams = + PlannerParams(processMultiPartition = true))) execPlan.isInstanceOf[TimeScalarGeneratorExec] shouldEqual (true) } @@ -152,10 +154,10 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers { 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 promQlQueryParams = PromQlQueryParams("test1{job = \"app\"} + test2{job = \"app\"}", 1000, 100, 2000) - val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams, plannerParams = + PlannerParams(processMultiPartition = true))) execPlan.isInstanceOf[BinaryJoinExec] shouldEqual (true) } @@ -196,14 +198,16 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers { 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 promQlQueryParams = PromQlQueryParams("test1{job = \"app\"} + test2{job = \"app\"}", 1000, 100, 10000) - val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams, plannerParams = + PlannerParams(processMultiPartition = true))) execPlan.isInstanceOf[PromQlRemoteExec] shouldEqual (true) - execPlan.asInstanceOf[PromQlRemoteExec].params.startSecs shouldEqual 1000 - execPlan.asInstanceOf[PromQlRemoteExec].params.endSecs shouldEqual 10000 + val queryParams = execPlan.asInstanceOf[PromQlRemoteExec].queryContext.origQueryParams. + asInstanceOf[PromQlQueryParams] + queryParams.startSecs shouldEqual 1000 + queryParams.endSecs shouldEqual 10000 } it ("should generate Exec plan for Metadata query") { @@ -225,17 +229,20 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers { TimeStepParams(startSeconds, step, endSeconds)) val promQlQueryParams = PromQlQueryParams( - "http_requests_total{job=\"prometheus\", method=\"GET\"}", startSeconds, step, endSeconds, - processMultiPartition = true) + "http_requests_total{job=\"prometheus\", method=\"GET\"}", startSeconds, step, endSeconds) - val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams, plannerParams = + PlannerParams(processMultiPartition = true))) 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) + val queryParams = execPlan.children(1).asInstanceOf[MetadataRemoteExec].queryContext.origQueryParams. + asInstanceOf[PromQlQueryParams] + + queryParams.startSecs shouldEqual(startSeconds) + queryParams.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 @@ -268,20 +275,21 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers { } 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 promQlQueryParams = PromQlQueryParams("test{job = \"app\"}", startSeconds, step, endSeconds) + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams, plannerParams = + PlannerParams(processMultiPartition = true))) val stitchRvsExec = execPlan.asInstanceOf[StitchRvsExec] stitchRvsExec.children.size shouldEqual (3) stitchRvsExec.children(0).isInstanceOf[PromQlRemoteExec] shouldEqual (true) stitchRvsExec.children(1).isInstanceOf[PromQlRemoteExec] shouldEqual (true) stitchRvsExec.children(2).isInstanceOf[PromQlRemoteExec] shouldEqual (true) val remoteExec1 = stitchRvsExec.children(0).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 + val queryParams1 = remoteExec1.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams] + queryParams1.startSecs shouldEqual startSeconds + queryParams1.endSecs shouldEqual 3999 + queryParams1.stepSecs shouldEqual step + remoteExec1.queryContext.plannerParams.processFailure shouldEqual true + remoteExec1.queryContext.plannerParams.processMultiPartition shouldEqual false remoteExec1.queryEndpoint shouldEqual "remote-url1" val remoteExec2 = stitchRvsExec.children(1).asInstanceOf[PromQlRemoteExec] @@ -293,19 +301,21 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers { 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 + val queryParams2 = remoteExec2.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams] + queryParams2.startSecs shouldEqual expectedStartMs1 / 1000 + queryParams2.endSecs shouldEqual 6999 + queryParams2.stepSecs shouldEqual step + remoteExec2.queryContext.plannerParams.processFailure shouldEqual true + remoteExec2.queryContext.plannerParams.processMultiPartition shouldEqual false remoteExec2.queryEndpoint shouldEqual "remote-url2" val remoteExec3 = stitchRvsExec.children(2).asInstanceOf[PromQlRemoteExec] - remoteExec3.params.startSecs shouldEqual expectedStartMs2 / 1000 - remoteExec3.params.endSecs shouldEqual endSeconds - remoteExec3.params.stepSecs shouldEqual step - remoteExec3.params.processFailure shouldEqual true - remoteExec3.params.processMultiPartition shouldEqual false + val queryParams3 = remoteExec3.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams] + queryParams3.startSecs shouldEqual expectedStartMs2 / 1000 + queryParams3.endSecs shouldEqual endSeconds + queryParams3.stepSecs shouldEqual step + remoteExec3.queryContext.plannerParams.processFailure shouldEqual true + remoteExec3.queryContext.plannerParams.processMultiPartition shouldEqual false remoteExec3.queryEndpoint shouldEqual "remote-url3" } @@ -334,10 +344,10 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers { 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 promQlQueryParams = PromQlQueryParams("test{job = \"app\"}", startSeconds, step, endSeconds) - val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams, + plannerParams = PlannerParams(processMultiPartition = true))) val stitchRvsExec = execPlan.asInstanceOf[StitchRvsExec] stitchRvsExec.children.size shouldEqual (2) @@ -347,19 +357,21 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers { // Instant/Raw queries will have same start and end point in all partitions as we want to fetch raw data val remoteExec1 = stitchRvsExec.children(0).asInstanceOf[PromQlRemoteExec] - remoteExec1.params.startSecs shouldEqual startSeconds - remoteExec1.params.endSecs shouldEqual endSeconds - remoteExec1.params.stepSecs shouldEqual step - remoteExec1.params.processFailure shouldEqual true - remoteExec1.params.processMultiPartition shouldEqual false + val queryParams1 = remoteExec1.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams] + queryParams1.startSecs shouldEqual startSeconds + queryParams1.endSecs shouldEqual endSeconds + queryParams1.stepSecs shouldEqual step + remoteExec1.queryContext.plannerParams.processFailure shouldEqual true + remoteExec1.queryContext.plannerParams.processMultiPartition shouldEqual false remoteExec1.queryEndpoint shouldEqual "remote-url1" val remoteExec2 = stitchRvsExec.children(1).asInstanceOf[PromQlRemoteExec] - remoteExec2.params.startSecs shouldEqual startSeconds - remoteExec2.params.endSecs shouldEqual endSeconds - remoteExec2.params.stepSecs shouldEqual step - remoteExec2.params.processFailure shouldEqual true - remoteExec2.params.processMultiPartition shouldEqual false + val queryParams2 = remoteExec1.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams] + queryParams2.startSecs shouldEqual startSeconds + queryParams2.endSecs shouldEqual endSeconds + queryParams2.stepSecs shouldEqual step + remoteExec2.queryContext.plannerParams.processFailure shouldEqual true + remoteExec2.queryContext.plannerParams.processMultiPartition shouldEqual false remoteExec2.queryEndpoint shouldEqual "remote-url2" } @@ -390,10 +402,10 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers { 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 promQlQueryParams = PromQlQueryParams("test{job = \"app\"}", startSeconds, step, endSeconds) - val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams, plannerParams = + PlannerParams(processMultiPartition = true))) val stitchRvsExec = execPlan.asInstanceOf[StitchRvsExec] stitchRvsExec.children.size shouldEqual (2) stitchRvsExec.children(0).isInstanceOf[PromQlRemoteExec] shouldEqual (true) @@ -401,19 +413,21 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers { val remoteExec = stitchRvsExec.children(0).asInstanceOf[PromQlRemoteExec] - remoteExec.params.startSecs shouldEqual startSeconds - remoteExec.params.endSecs shouldEqual (localPartitionStartMs - 1) / 1000 - remoteExec.params.stepSecs shouldEqual step - remoteExec.params.processFailure shouldEqual true - remoteExec.params.processMultiPartition shouldEqual false + val queryParams = remoteExec.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams] + queryParams.startSecs shouldEqual startSeconds + queryParams.endSecs shouldEqual (localPartitionStartMs - 1) / 1000 + queryParams.stepSecs shouldEqual step + remoteExec.queryContext.plannerParams.processFailure shouldEqual true + remoteExec.queryContext.plannerParams.processMultiPartition shouldEqual false remoteExec.queryEndpoint shouldEqual "remote-url" val remoteExec2 = stitchRvsExec.children(1).asInstanceOf[PromQlRemoteExec] - remoteExec2.params.startSecs shouldEqual endSeconds - remoteExec2.params.endSecs shouldEqual endSeconds - remoteExec2.params.stepSecs shouldEqual step - remoteExec2.params.processFailure shouldEqual true - remoteExec2.params.processMultiPartition shouldEqual false + val queryParams2 = remoteExec2.queryContext.origQueryParams.asInstanceOf[PromQlQueryParams] + queryParams2.startSecs shouldEqual endSeconds + queryParams2.endSecs shouldEqual endSeconds + queryParams2.stepSecs shouldEqual step + remoteExec2.queryContext.plannerParams.processFailure shouldEqual true + remoteExec2.queryContext.plannerParams.processMultiPartition shouldEqual false remoteExec2.queryEndpoint shouldEqual "remote-url" } @@ -433,13 +447,13 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers { } val engine = new MultiPartitionPlanner(partitionLocationProvider, localPlanner, "local", dataset, queryConfig) + val lp = Parser.labelValuesQueryToLogicalPlan(Seq("""__metric__"""), Some("""_ws_="demo""""), TimeStepParams(startSeconds, step, endSeconds) ) - val promQlQueryParams = PromQlQueryParams( - "", startSeconds, step, endSeconds, None, Some("/api/v2/label/values"), - processMultiPartition = true) + val promQlQueryParams = PromQlQueryParams("", startSeconds, step, endSeconds, Some("/api/v2/label/values")) - val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams, plannerParams = + PlannerParams(processMultiPartition = true))) execPlan.isInstanceOf[LabelValuesDistConcatExec] shouldEqual (true) execPlan.children(0).isInstanceOf[LabelValuesDistConcatExec] shouldEqual(true) @@ -448,7 +462,8 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers { val expectedUrlParams = Map("filter" -> """_ws_="demo"""", "labels" -> "__metric__") execPlan.children(1).asInstanceOf[MetadataRemoteExec].urlParams shouldEqual(expectedUrlParams) // Filter values // should have quotes - execPlan.children(1).asInstanceOf[MetadataRemoteExec].params.endSecs shouldEqual(localPartitionStart - 1) + execPlan.children(1).asInstanceOf[MetadataRemoteExec].queryContext.origQueryParams.asInstanceOf[PromQlQueryParams]. + endSecs shouldEqual(localPartitionStart - 1) execPlan.children(0).asInstanceOf[LabelValuesDistConcatExec].children(0).asInstanceOf[LabelValuesExec].startMs shouldEqual (localPartitionStart * 1000) execPlan.children(0).asInstanceOf[LabelValuesDistConcatExec].children(0).asInstanceOf[LabelValuesExec].endMs shouldEqual diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlannerSpec.scala index a03acc976d..edb8d3088b 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlannerSpec.scala @@ -9,7 +9,7 @@ import org.scalatest.matchers.should.Matchers 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.{ColumnFilter, PlannerParams, PromQlQueryParams, QueryConfig, QueryContext} import filodb.core.query.Filter.Equals import filodb.prometheus.ast.TimeStepParams import filodb.prometheus.parse.Parser @@ -30,7 +30,7 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture withFallback(routingConfig) private val queryConfig = new QueryConfig(config) - private val promQlQueryParams = PromQlQueryParams("sum(heap_usage)", 100, 1, 1000, None) + private val promQlQueryParams = PromQlQueryParams("sum(heap_usage)", 100, 1, 1000) private val localMapper = new ShardMapper(32) for {i <- 0 until 32} localMapper.registerNode(Seq(i), node) @@ -61,7 +61,7 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture ColumnFilter("_ns_", Equals("App-2"))))} val engine = new ShardKeyRegexPlanner( dataset, localPlanner, shardKeyMatcherFn) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = PromQlQueryParams("sum(heap_usage)", 100, 1, - 1000, None))) + 1000))) execPlan.isInstanceOf[MultiPartitionReduceAggregateExec] shouldEqual(true) execPlan.children(0).children.head.isInstanceOf[MultiSchemaPartitionsExec] execPlan.children(1).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. @@ -86,7 +86,7 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture ColumnFilter("_ns_", Equals("App-2"))))} val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = PromQlQueryParams("1 + test{_ws_ = \"demo\"," + - " _ns_ =~ \"App.*\", instance = \"Inst-1\" }", 100, 1, 1000, None))) + " _ns_ =~ \"App.*\", instance = \"Inst-1\" }", 100, 1, 1000))) execPlan.isInstanceOf[MultiPartitionDistConcatExec] shouldEqual(true) execPlan.rangeVectorTransformers(0).isInstanceOf[ScalarOperationMapper] shouldEqual true execPlan.children(0).children.head.isInstanceOf[MultiSchemaPartitionsExec] @@ -209,7 +209,7 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture it("should generate Exec plan for scalar - time()") { val lp = Parser.queryToLogicalPlan("""scalar(test{_ws_ = "demo", _ns_ =~ "App.*"}) - time()""", 1000, 1000) - val promQlQueryParams = PromQlQueryParams("""scalar(test{_ws_ = "demo", _ns_ =~ "App.*"}) - time()""", 100, 1, 1000, None) + val promQlQueryParams = PromQlQueryParams("""scalar(test{_ws_ = "demo", _ns_ =~ "App.*"}) - time()""", 100, 1, 1000) val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-2"))))} @@ -242,10 +242,11 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture TimeStepParams(1000, 20, 5000) ) val promQlQueryParams = PromQlQueryParams( - "", 1000, 20, 5000, None, Some("/api/v2/label/values"), - processMultiPartition = true) + "", 1000, 20, 5000, Some("/api/v2/label/values")) + + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams, plannerParams = + PlannerParams(processMultiPartition = true))) - val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) execPlan.isInstanceOf[LabelValuesDistConcatExec] shouldEqual (true) } } diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/SingleClusterPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/SingleClusterPlannerSpec.scala index e9b2ae0199..832f9929c5 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/SingleClusterPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/SingleClusterPlannerSpec.scala @@ -1,17 +1,15 @@ package filodb.coordinator.queryplanner import scala.concurrent.duration._ - import akka.actor.ActorSystem import akka.testkit.TestProbe import com.typesafe.config.ConfigFactory import org.scalatest.concurrent.ScalaFutures - import filodb.coordinator.ShardMapper import filodb.coordinator.client.QueryCommands.{FunctionalSpreadProvider, StaticSpreadProvider} import filodb.core.{GlobalScheduler, MetricsTestData, SpreadChange} import filodb.core.metadata.Schemas -import filodb.core.query.{ColumnFilter, Filter, PromQlQueryParams, QueryConfig, QueryContext} +import filodb.core.query.{ColumnFilter, Filter, PlannerParams, PromQlQueryParams, QueryConfig, QueryContext} import filodb.core.store.TimeRangeChunkScan import filodb.prometheus.ast.{TimeStepParams, WindowConstants} import filodb.prometheus.parse.Parser @@ -114,7 +112,7 @@ class SingleClusterPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture // materialized exec plan val execPlan = engine.materialize(logicalPlan, - QueryContext(promQlQueryParams, Some(StaticSpreadProvider(SpreadChange(0, 4))), 1000000)) + QueryContext(promQlQueryParams, plannerParams = PlannerParams(spreadOverride = Some(StaticSpreadProvider(SpreadChange(0, 4))), queryTimeoutMillis =1000000))) execPlan.isInstanceOf[BinaryJoinExec] shouldEqual true // Now there should be multiple levels of reduce because we have 16 shards @@ -142,9 +140,8 @@ class SingleClusterPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture case _ => throw new IllegalArgumentException(s"Unexpected LP $lp") } - val execPlan = engine.materialize(lp, - QueryContext(promQlQueryParams, Some(StaticSpreadProvider(SpreadChange(0, 4))), 1000000)) - + val execPlan = engine.materialize(lp, QueryContext(promQlQueryParams, plannerParams = PlannerParams(spreadOverride = + Some(StaticSpreadProvider(SpreadChange(0, 4))), queryTimeoutMillis =1000000))) info(s"First child plan: ${execPlan.children.head.printTree()}") execPlan.isInstanceOf[LocalPartitionDistConcatExec] shouldEqual true execPlan.children.foreach { l1 => @@ -185,7 +182,8 @@ class SingleClusterPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture val logicalPlan = BinaryJoin(summed1, BinaryOperator.DIV, Cardinality.OneToOne, summed2) // materialized exec plan - val execPlan = engine.materialize(logicalPlan, QueryContext(promQlQueryParams, Some(FunctionalSpreadProvider(spreadFunc)), 1000000)) + val execPlan = engine.materialize(logicalPlan, QueryContext(promQlQueryParams, plannerParams = PlannerParams + (spreadOverride = Some(FunctionalSpreadProvider(spreadFunc)), queryTimeoutMillis =1000000))) execPlan.printTree() execPlan.isInstanceOf[BinaryJoinExec] shouldEqual true @@ -201,8 +199,8 @@ class SingleClusterPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture def spread(filter: Seq[ColumnFilter]): Seq[SpreadChange] = { Seq(SpreadChange(0, 1), SpreadChange(25000000, 2)) // spread change time is in ms } - val execPlan = engine.materialize(lp, QueryContext(promQlQueryParams, - Some(FunctionalSpreadProvider(spread)), 1000000)) + val execPlan = engine.materialize(lp,QueryContext(promQlQueryParams, plannerParams = PlannerParams + (spreadOverride = Some(FunctionalSpreadProvider(spread)), queryTimeoutMillis = 1000000))) execPlan.rangeVectorTransformers.head.isInstanceOf[StitchRvsMapper] shouldEqual true } @@ -211,8 +209,8 @@ class SingleClusterPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture def spread(filter: Seq[ColumnFilter]): Seq[SpreadChange] = { Seq(SpreadChange(0, 1), SpreadChange(35000000, 2)) } - val execPlan = engine.materialize(lp, QueryContext(promQlQueryParams, - Some(FunctionalSpreadProvider(spread)), 1000000)) + val execPlan = engine.materialize(lp, QueryContext(promQlQueryParams, plannerParams = PlannerParams + (spreadOverride = Some(FunctionalSpreadProvider(spread)), queryTimeoutMillis = 1000000))) execPlan.rangeVectorTransformers.isEmpty shouldEqual true } @@ -222,8 +220,8 @@ class SingleClusterPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture def spread(filter: Seq[ColumnFilter]): Seq[SpreadChange] = { Seq(SpreadChange(0, 1), SpreadChange(25000000, 2)) } - val execPlan = engine.materialize(lp, QueryContext(promQlQueryParams, Some(FunctionalSpreadProvider(spread)), - 1000000)) + val execPlan = engine.materialize(lp, QueryContext(promQlQueryParams, plannerParams = PlannerParams + (spreadOverride = Some(FunctionalSpreadProvider(spread)), queryTimeoutMillis = 1000000))) val binaryJoinNode = execPlan.children(0) binaryJoinNode.isInstanceOf[BinaryJoinExec] shouldEqual true binaryJoinNode.children.size shouldEqual 2 @@ -236,8 +234,8 @@ class SingleClusterPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture def spread(filter: Seq[ColumnFilter]): Seq[SpreadChange] = { Seq(SpreadChange(0, 1), SpreadChange(35000000, 2)) } - val execPlan = engine.materialize(lp, QueryContext(promQlQueryParams, Some(FunctionalSpreadProvider(spread)), - 1000000)) + val execPlan = engine.materialize(lp, QueryContext(promQlQueryParams, plannerParams = PlannerParams + (spreadOverride = Some(FunctionalSpreadProvider(spread)), queryTimeoutMillis = 1000000))) val binaryJoinNode = execPlan.children(0) binaryJoinNode.isInstanceOf[BinaryJoinExec] shouldEqual true binaryJoinNode.children.foreach(_.isInstanceOf[StitchRvsExec] should not equal true) diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/SingleClusterPlannerSplitSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/SingleClusterPlannerSplitSpec.scala index 00c8f10c16..6393fa9b9f 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/SingleClusterPlannerSplitSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/SingleClusterPlannerSplitSpec.scala @@ -155,7 +155,8 @@ class SingleClusterPlannerSplitSpec extends AnyFunSpec with Matchers with ScalaF // materialized exec plan val parentExecPlan = engine.materialize(logicalPlan, - QueryContext(promQlQueryParams, Some(StaticSpreadProvider(SpreadChange(0, 4))), 1000000)) + QueryContext(promQlQueryParams, plannerParams = PlannerParams(spreadOverride = + Some(StaticSpreadProvider(SpreadChange(0, 4))), queryTimeoutMillis = 1000000))) parentExecPlan.isInstanceOf[SplitLocalPartitionDistConcatExec] shouldEqual true parentExecPlan.children.foreach { execPlan => @@ -189,7 +190,8 @@ class SingleClusterPlannerSplitSpec extends AnyFunSpec with Matchers with ScalaF } val parentExecPlan = engine.materialize(lp, - QueryContext(promQlQueryParams, Some(StaticSpreadProvider(SpreadChange(0, 4))), 1000000)) + QueryContext(promQlQueryParams, plannerParams = PlannerParams(spreadOverride = + Some(StaticSpreadProvider(SpreadChange(0, 4))), queryTimeoutMillis = 1000000))) info(s"First inner child plan: ${parentExecPlan.children.head.children.head.printTree()}") parentExecPlan.isInstanceOf[SplitLocalPartitionDistConcatExec] shouldEqual true @@ -220,7 +222,9 @@ class SingleClusterPlannerSplitSpec extends AnyFunSpec with Matchers with ScalaF val logicalPlan = BinaryJoin(summed1, BinaryOperator.DIV, Cardinality.OneToOne, summed2) // materialized exec plan - val parentExecPlan = engine.materialize(logicalPlan, QueryContext(promQlQueryParams, Some(FunctionalSpreadProvider(spreadFunc)), 1000000)) + val parentExecPlan = engine.materialize(logicalPlan, QueryContext(promQlQueryParams, plannerParams = + PlannerParams(spreadOverride = Some(FunctionalSpreadProvider(spreadFunc)), queryTimeoutMillis = 1000000))) + parentExecPlan.printTree() parentExecPlan.isInstanceOf[SplitLocalPartitionDistConcatExec] shouldEqual true @@ -253,8 +257,8 @@ class SingleClusterPlannerSplitSpec extends AnyFunSpec with Matchers with ScalaF def spread(filter: Seq[ColumnFilter]): Seq[SpreadChange] = { Seq(SpreadChange(0, 1), SpreadChange(350000, 2)) // spread change time is in ms } - val execPlan = engine.materialize(lp, QueryContext(promQlQueryParams, - Some(FunctionalSpreadProvider(spread)), 1000000)) + val execPlan = engine.materialize(lp, QueryContext(promQlQueryParams, plannerParams = PlannerParams(spreadOverride = + Some(FunctionalSpreadProvider(spread)), queryTimeoutMillis = 1000000))) execPlan.isInstanceOf[SplitLocalPartitionDistConcatExec] shouldEqual true execPlan.rangeVectorTransformers.head.isInstanceOf[StitchRvsMapper] shouldEqual true // Stitch split plans execPlan.children should have length (5) @@ -272,8 +276,8 @@ class SingleClusterPlannerSplitSpec extends AnyFunSpec with Matchers with ScalaF def spread(filter: Seq[ColumnFilter]): Seq[SpreadChange] = { Seq(SpreadChange(0, 1), SpreadChange(450000, 2)) // spread change time is in ms } - val execPlan = engine.materialize(lp, QueryContext(promQlQueryParams, - Some(FunctionalSpreadProvider(spread)), 1000000)) + val execPlan = engine.materialize(lp, QueryContext(promQlQueryParams, plannerParams = PlannerParams(spreadOverride = + Some(FunctionalSpreadProvider(spread)), queryTimeoutMillis = 1000000))) execPlan.isInstanceOf[SplitLocalPartitionDistConcatExec] shouldEqual true execPlan.rangeVectorTransformers.head.isInstanceOf[StitchRvsMapper] shouldEqual true // Stitch split plans execPlan.children should have length (5) @@ -293,8 +297,8 @@ class SingleClusterPlannerSplitSpec extends AnyFunSpec with Matchers with ScalaF def spread(filter: Seq[ColumnFilter]): Seq[SpreadChange] = { Seq(SpreadChange(0, 1), SpreadChange(350000, 2)) } - val execPlan = engine.materialize(lp, QueryContext(promQlQueryParams, Some(FunctionalSpreadProvider(spread)), - 1000000)) + val execPlan = engine.materialize(lp, QueryContext(promQlQueryParams, plannerParams = PlannerParams(spreadOverride = + Some(FunctionalSpreadProvider(spread)), queryTimeoutMillis = 1000000))) execPlan.isInstanceOf[SplitLocalPartitionDistConcatExec] shouldEqual true execPlan.rangeVectorTransformers.head.isInstanceOf[StitchRvsMapper] shouldEqual true // Stitch split plans @@ -329,8 +333,8 @@ class SingleClusterPlannerSplitSpec extends AnyFunSpec with Matchers with ScalaF def spread(filter: Seq[ColumnFilter]): Seq[SpreadChange] = { Seq(SpreadChange(0, 1), SpreadChange(450000, 2)) } - val execPlan = engine.materialize(lp, QueryContext(promQlQueryParams, Some(FunctionalSpreadProvider(spread)), - 1000000)) + val execPlan = engine.materialize(lp, QueryContext(promQlQueryParams, plannerParams = PlannerParams(spreadOverride = + Some(FunctionalSpreadProvider(spread)), queryTimeoutMillis = 1000000))) execPlan.isInstanceOf[SplitLocalPartitionDistConcatExec] shouldEqual true execPlan.rangeVectorTransformers.head.isInstanceOf[StitchRvsMapper] shouldEqual true // Stitch split plans diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/SinglePartitionPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/SinglePartitionPlannerSpec.scala index 60a7b98382..671984cb82 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/SinglePartitionPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/SinglePartitionPlannerSpec.scala @@ -171,8 +171,10 @@ class SinglePartitionPlannerSpec extends AnyFunSpec with Matchers { execPlan.children(1).isInstanceOf[PromQlRemoteExec] shouldEqual(true) // LHS should have only LHS query and RHS should have oly RHS query - execPlan.children(0).asInstanceOf[PromQlRemoteExec].params.promQl shouldEqual("""test1{job="app"}""") - execPlan.children(1).asInstanceOf[PromQlRemoteExec].params.promQl shouldEqual("""test2{job="app"}""") + execPlan.children(0).asInstanceOf[PromQlRemoteExec].queryContext.origQueryParams.asInstanceOf[PromQlQueryParams]. + promQl shouldEqual("""test1{job="app"}""") + execPlan.children(1).asInstanceOf[PromQlRemoteExec].queryContext.origQueryParams.asInstanceOf[PromQlQueryParams]. + promQl shouldEqual("""test2{job="app"}""") } } diff --git a/core/src/main/scala/filodb.core/query/QueryContext.scala b/core/src/main/scala/filodb.core/query/QueryContext.scala index f15babbf3d..98b3b4e0fe 100644 --- a/core/src/main/scala/filodb.core/query/QueryContext.scala +++ b/core/src/main/scala/filodb.core/query/QueryContext.scala @@ -11,27 +11,39 @@ trait TsdbQueryParams * This class provides PromQl query parameters * Config has routing parameters */ -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 class PromQlQueryParams(promQl: String, startSecs: Long, stepSecs: Long, endSecs: Long , remoteQueryPath: + Option[String] = None, verbose: Boolean = false) extends TsdbQueryParams + case object UnavailablePromQlQueryParams extends TsdbQueryParams +case class PlannerParams(spread: Option[Int] = None, + spreadOverride: Option[SpreadProvider] = None, + shardOverrides: Option[Seq[Int]] = None, + queryTimeoutMillis: Int = 30000, + sampleLimit: Int = 1000000, + groupByCardLimit: Int = 100000, + joinQueryCardLimit: Int = 100000, + skipAggregatePresent: Boolean = false, + processFailure: Boolean = true, + processMultiPartition: Boolean = false) +object PlannerParams { + def apply(constSpread: Option[SpreadProvider], sampleLimit: Int): PlannerParams = + PlannerParams(spreadOverride = constSpread, sampleLimit = sampleLimit) +} /** * This class provides general query processing parameters */ final case class QueryContext(origQueryParams: TsdbQueryParams = UnavailablePromQlQueryParams, - 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()) + submitTime: Long = System.currentTimeMillis(), + plannerParams: PlannerParams = PlannerParams()) object QueryContext { def apply(constSpread: Option[SpreadProvider], sampleLimit: Int): QueryContext = - QueryContext(spreadOverride = constSpread, sampleLimit = sampleLimit) + QueryContext(plannerParams = PlannerParams(constSpread, sampleLimit)) + + def apply(queryParams: TsdbQueryParams, constSpread: Option[SpreadProvider]): QueryContext = + QueryContext(origQueryParams = queryParams, plannerParams = PlannerParams(spreadOverride = constSpread)) /** * Creates a spreadFunc that looks for a particular filter with keyName Equals a value, and then maps values diff --git a/core/src/main/scala/filodb.core/query/ResultTypes.scala b/core/src/main/scala/filodb.core/query/ResultTypes.scala index daf44f9920..46eec48b56 100644 --- a/core/src/main/scala/filodb.core/query/ResultTypes.scala +++ b/core/src/main/scala/filodb.core/query/ResultTypes.scala @@ -51,6 +51,7 @@ final case class ResultSchema(columns: Seq[ColumnInfo], numRowKeyColumns: Int, def isHistDouble: Boolean = columns.length == 3 && columns(1).colType == HistogramColumn && columns(2).colType == DoubleColumn def isHistogram: Boolean = columns.length == 2 && columns(1).colType == HistogramColumn + def isAvgAggregator: Boolean = columns.length == 3 && columns(2).name.equals("count") def hasSameColumnsAs(other: ResultSchema): Boolean = { // exclude fixedVectorLen & colIDs diff --git a/core/src/main/scala/filodb.core/store/ChunkSetInfo.scala b/core/src/main/scala/filodb.core/store/ChunkSetInfo.scala index 96bde268c0..8c8065033a 100644 --- a/core/src/main/scala/filodb.core/store/ChunkSetInfo.scala +++ b/core/src/main/scala/filodb.core/store/ChunkSetInfo.scala @@ -421,7 +421,7 @@ extends Iterator[ChunkSetInfoReader] { while (curWindowEnd > lastEndTime && infos.hasNext) { val next = infos.nextInfoReader val queryTimeElapsed = System.currentTimeMillis() - queryContext.submitTime - if (queryTimeElapsed >= queryContext.queryTimeoutMillis) + if (queryTimeElapsed >= queryContext.plannerParams.queryTimeoutMillis) throw QueryTimeoutException(queryTimeElapsed, this.getClass.getName) // Add if next chunkset is within window and not empty. Otherwise keep going diff --git a/http/src/main/scala/filodb/http/PrometheusApiRoute.scala b/http/src/main/scala/filodb/http/PrometheusApiRoute.scala index 014282fcd2..e6e4106eaa 100644 --- a/http/src/main/scala/filodb/http/PrometheusApiRoute.scala +++ b/http/src/main/scala/filodb/http/PrometheusApiRoute.scala @@ -48,7 +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(query, start.toLong, step.toLong, end.toLong, spread), histMap.getOrElse(false)) + spread, PromQlQueryParams(query, start.toLong, step.toLong, end.toLong), histMap.getOrElse(false)) } } } ~ @@ -64,7 +64,7 @@ class PrometheusApiRoute(nodeCoord: ActorRef, settings: HttpSettings)(implicit a val stepLong = step.map(_.toLong).getOrElse(0L) val logicalPlan = Parser.queryToLogicalPlan(query, time.toLong, stepLong) askQueryAndRespond(dataset, logicalPlan, explainOnly.getOrElse(false), - verbose.getOrElse(false), spread, PromQlQueryParams(query, time.toLong, stepLong, time.toLong, spread), + verbose.getOrElse(false), spread, PromQlQueryParams(query, time.toLong, stepLong, time.toLong), histMap.getOrElse(false)) } } diff --git a/jmh/src/main/scala/filodb.jmh/HistogramQueryBenchmark.scala b/jmh/src/main/scala/filodb.jmh/HistogramQueryBenchmark.scala index f5c355be1e..f22a4539ba 100644 --- a/jmh/src/main/scala/filodb.jmh/HistogramQueryBenchmark.scala +++ b/jmh/src/main/scala/filodb.jmh/HistogramQueryBenchmark.scala @@ -19,7 +19,7 @@ import filodb.core.{MachineMetricsData, MetricsTestData, SpreadChange, TestData} import filodb.core.binaryrecord2.RecordBuilder import filodb.core.memstore._ import filodb.core.metadata.Schemas -import filodb.core.query.{QueryConfig, QueryContext, QuerySession} +import filodb.core.query.{PlannerParams, QueryConfig, QueryContext, QuerySession} import filodb.core.store._ import filodb.memory.MemFactory import filodb.memory.format.SeqRowReader @@ -91,8 +91,11 @@ class HistogramQueryBenchmark { // Single-threaded query test val numQueries = 500 + QueryContext(plannerParams= PlannerParams(sampleLimit = 1000)) + QueryContext(plannerParams = PlannerParams(shardOverrides = Some(Seq(0)))) + val qContext = QueryContext(Some(new StaticSpreadProvider(SpreadChange(0, 1))), 100). - copy(shardOverrides = Some(Seq(0)), queryTimeoutMillis = 60000) + copy(plannerParams = PlannerParams(shardOverrides = Some(Seq(0)), queryTimeoutMillis = 60000)) val hLogicalPlan = Parser.queryToLogicalPlan(histQuery, startTime/1000, 1000) val hExecPlan = hEngine.materialize(hLogicalPlan, qContext) val querySched = Scheduler.singleThread(s"benchmark-query") diff --git a/prometheus/src/main/scala/filodb/prometheus/query/PrometheusModel.scala b/prometheus/src/main/scala/filodb/prometheus/query/PrometheusModel.scala index c89cbebcbc..1bd34eb941 100644 --- a/prometheus/src/main/scala/filodb/prometheus/query/PrometheusModel.scala +++ b/prometheus/src/main/scala/filodb/prometheus/query/PrometheusModel.scala @@ -7,6 +7,7 @@ import filodb.core.metadata.Column.ColumnType import filodb.core.metadata.PartitionSchema import filodb.core.query.{ColumnFilter, ColumnInfo, Filter, RangeVector, RangeVectorKey} import filodb.query.{QueryResult => FiloQueryResult, _} +import filodb.query.AggregationOperator.Avg import filodb.query.exec.{ExecPlan, HistToPromSeriesMapper} object PrometheusModel { @@ -152,6 +153,21 @@ object PrometheusModel { } } + def toAvgResult(srv: RangeVector, + verbose: Boolean, + typ: QueryResultType, + processMultiPartition: Boolean = true): Result = { + val tags = srv.key.labelValues.map { case (k, v) => (k.toString, v.toString)} ++ + (if (verbose) makeVerboseLabels(srv.key) + else Map.empty) + val samples = srv.rows.map { r => AvgSampl(r.getLong(0)/1000, r.getDouble(1), + r.getLong(2)) + }.toSeq + + Result(tags, None, None, Some(AggregateResponse(Avg.entryName, samples))) + } + + def makeVerboseLabels(rvk: RangeVectorKey): Map[String, String] = { Map("_shards_" -> rvk.sourceShards.mkString(","), "_partIds_" -> rvk.partIds.mkString(","), diff --git a/query/src/main/scala/filodb/query/PromCirceSupport.scala b/query/src/main/scala/filodb/query/PromCirceSupport.scala index 99a4d09da4..fe44d11448 100644 --- a/query/src/main/scala/filodb/query/PromCirceSupport.scala +++ b/query/src/main/scala/filodb/query/PromCirceSupport.scala @@ -3,6 +3,8 @@ package filodb.query import io.circe.{Decoder, Encoder, HCursor, Json} import io.circe.syntax._ +import filodb.query.AggregationOperator.Avg + object PromCirceSupport { import cats.syntax.either._ // necessary to encode sample in promql response as an array with long and double value as string @@ -10,9 +12,21 @@ 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 h @ AvgSampl(t, v, c) => Json.fromValues(Seq(t.asJson, v.toString.asJson, c.toString.asJson)) case m @ MetadataSampl(v) => Json.fromValues(Seq(v.asJson)) } + implicit val decodeAvgSample: Decoder[AvgSampl] = new Decoder[AvgSampl] { + final def apply(c: HCursor): Decoder.Result[AvgSampl] = { + for { timestamp <- c.downArray.as[Long] + value <- c.downArray.right.as[String] + count <- c.downArray.right.right.as[Long] + } yield { + AvgSampl(timestamp, value.toDouble, count) + } + } + } + implicit val decodeFoo: Decoder[DataSampl] = new Decoder[DataSampl] { final def apply(c: HCursor): Decoder.Result[DataSampl] = { val tsResult = c.downArray.as[Long] @@ -30,4 +44,21 @@ object PromCirceSupport { } } } + + implicit val decodeAggregate: Decoder[AggregateResponse] = new Decoder[AggregateResponse] { + final def apply(c: HCursor): Decoder.Result[AggregateResponse] = { + val functionName = c.downField("function").as[String] match { + case Right(fn) => fn + case Left(ex) => throw ex + } + + val aggregateSamples = functionName match { + case Avg.entryName => c.downField ("aggregateValues").as[List[AvgSampl]] + } + + for { + sample <- aggregateSamples + } yield AggregateResponse(functionName, sample) + } + } } diff --git a/query/src/main/scala/filodb/query/PromQueryResponse.scala b/query/src/main/scala/filodb/query/PromQueryResponse.scala index 5b3ad39aca..439cf30b71 100644 --- a/query/src/main/scala/filodb/query/PromQueryResponse.scala +++ b/query/src/main/scala/filodb/query/PromQueryResponse.scala @@ -15,10 +15,15 @@ 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) +final case class Result(metric: Map[String, String], values: Option[Seq[DataSampl]], value: Option[DataSampl] = None, + aggregateResponse: Option[AggregateResponse] = None) sealed trait DataSampl +sealed trait AggregateSampl + +case class AggregateResponse(function: String, aggregateSampl: Seq[AggregateSampl]) + /** * Metric value for a given timestamp * @param timestamp in seconds since epoch @@ -29,3 +34,6 @@ final case class Sampl(timestamp: Long, value: Double) extends DataSampl final case class HistSampl(timestamp: Long, buckets: Map[String, Double]) extends DataSampl final case class MetadataSampl(values: Map[String, String]) extends DataSampl + +final case class AvgSampl(timestamp: Long, value: Double, count: Long) extends AggregateSampl + diff --git a/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala b/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala index 5825fc297a..161f255746 100644 --- a/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala +++ b/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala @@ -36,11 +36,10 @@ trait ReduceAggregateExec extends NonLeafExecPlan { yield { val aggregator = RowAggregator(aggrOp, aggrParams, schema) RangeVectorAggregator.mapReduce(aggregator, skipMapPhase = true, results, rv => rv.key, - querySession.qContext.groupByCardLimit) + querySession.qContext.plannerParams.groupByCardLimit) } Observable.fromTask(task).flatten } - } /** @@ -104,11 +103,11 @@ final case class AggregateMapReduce(aggrOp: AggregationOperator, RangeVectorAggregator.fastReduce(aggregator, false, source, numWindows) }.getOrElse { RangeVectorAggregator.mapReduce(aggregator, skipMapPhase = false, source, grouping, - querySession.qContext.groupByCardLimit) + querySession.qContext.plannerParams.groupByCardLimit) } } else { RangeVectorAggregator.mapReduce(aggregator, skipMapPhase = false, source, grouping, - querySession.qContext.groupByCardLimit) + querySession.qContext.plannerParams.groupByCardLimit) } } diff --git a/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala b/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala index a4d390a241..f8be12d668 100644 --- a/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala +++ b/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala @@ -67,9 +67,9 @@ final case class BinaryJoinExec(queryContext: QueryContext, 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.") + if (result.size > queryContext.plannerParams.joinQueryCardLimit && cardinality == Cardinality.OneToOne) => + throw new BadQueryException(s"This query results in more than ${queryContext.plannerParams.joinQueryCardLimit}" + + s" join cardinality. Try applying more filters.") case (QueryResult(_, _, result), i) => (result, i) case (QueryError(_, ex), _) => throw ex }.toListL.map { resp => @@ -109,9 +109,9 @@ final case class BinaryJoinExec(queryContext: QueryContext, 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.") + if (resultKeySet.size > queryContext.plannerParams.joinQueryCardLimit) + throw new BadQueryException(s"This query results in more than ${queryContext.plannerParams. + 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/main/scala/filodb/query/exec/ExecPlan.scala b/query/src/main/scala/filodb/query/exec/ExecPlan.scala index fe0c5e8f16..05bebc7757 100644 --- a/query/src/main/scala/filodb/query/exec/ExecPlan.scala +++ b/query/src/main/scala/filodb/query/exec/ExecPlan.scala @@ -143,7 +143,7 @@ trait ExecPlan extends QueryCommand { val finalRes = allTransformers.foldLeft((res.rvs, resSchema)) { (acc, transf) => span.mark(transf.getClass.getSimpleName) val paramRangeVector: Seq[Observable[ScalarRangeVector]] = transf.funcParams.map(_.getResult) - (transf.apply(acc._1, querySession, queryContext.sampleLimit, acc._2, + (transf.apply(acc._1, querySession, queryContext.plannerParams.sampleLimit, acc._2, paramRangeVector), transf.schema(acc._2)) } val recSchema = SerializedRangeVector.toSchema(finalRes._2.columns, finalRes._2.brSchemas) @@ -157,18 +157,18 @@ trait ExecPlan extends QueryCommand { case srv: SerializableRangeVector => numResultSamples += srv.numRowsInt // fail the query instead of limiting range vectors and returning incomplete/inaccurate results - if (enforceLimit && numResultSamples > queryContext.sampleLimit) - throw new BadQueryException(s"This query results in more than ${queryContext.sampleLimit} samples. " + - s"Try applying more filters or reduce time range.") + if (enforceLimit && numResultSamples > queryContext.plannerParams.sampleLimit) + throw new BadQueryException(s"This query results in more than ${queryContext.plannerParams. + sampleLimit} samples.Try applying more filters or reduce time range.") srv case rv: RangeVector => // materialize, and limit rows per RV val srv = SerializedRangeVector(rv, builder, recSchema, getClass.getSimpleName, span) numResultSamples += srv.numRowsInt // fail the query instead of limiting range vectors and returning incomplete/inaccurate results - if (enforceLimit && numResultSamples > queryContext.sampleLimit) - throw new BadQueryException(s"This query results in more than ${queryContext.sampleLimit} samples. " + - s"Try applying more filters or reduce time range.") + if (enforceLimit && numResultSamples > queryContext.plannerParams.sampleLimit) + throw new BadQueryException(s"This query results in more than ${queryContext.plannerParams. + sampleLimit} samples. Try applying more filters or reduce time range.") srv } .toListL @@ -184,7 +184,7 @@ trait ExecPlan extends QueryCommand { // 250 RVs * (250 bytes for RV-Key + 200 samples * 32 bytes per sample) // is < 2MB qLogger.warn(s"queryId: ${queryContext.queryId} result was large size $numBytes. May need to " + - s"tweak limits. ExecPlan was: ${printTree()} ; Limit was: ${queryContext.sampleLimit}") + s"tweak limits. ExecPlan was: ${printTree()} ; Limit was: ${queryContext.plannerParams.sampleLimit}") } span.mark(s"num-result-samples: $numResultSamples") span.mark(s"num-range-vectors: ${r.size}") diff --git a/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala b/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala index bfff4d3d90..1b921baa40 100644 --- a/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala +++ b/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala @@ -84,7 +84,7 @@ final case class PartKeysExec(queryContext: QueryContext, val rvs = source match { case memStore: MemStore => val response = memStore.partKeysWithFilters(dataset, shard, filters, - fetchFirstLastSampleTimes, end, start, queryContext.sampleLimit) + fetchFirstLastSampleTimes, end, start, queryContext.plannerParams.sampleLimit) import NoCloseCursor._ Observable.now(IteratorBackedRangeVector( new CustomRangeVectorKey(Map.empty), UTF8MapIteratorRowReader(response))) @@ -96,7 +96,7 @@ final case class PartKeysExec(queryContext: QueryContext, ExecResult(rvs, Task.eval(sch)) } - def args: String = s"shard=$shard, filters=$filters, limit=${queryContext.sampleLimit}" + def args: String = s"shard=$shard, filters=$filters, limit=${queryContext.plannerParams.sampleLimit}" } final case class LabelValuesExec(queryContext: QueryContext, @@ -118,13 +118,12 @@ final case class LabelValuesExec(queryContext: QueryContext, val memStore = source.asInstanceOf[MemStore] 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) - .map(termInfo => Map(columns.head.utf8 -> termInfo.term)) - .toIterator + case true if (columns.size == 1) => memStore.labelValues(dataset, shard, columns.head, queryContext. + plannerParams.sampleLimit).map(termInfo => Map(columns.head.utf8 -> termInfo.term)).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, endMs, startMs, - queryContext.sampleLimit) + queryContext.plannerParams.sampleLimit) } import NoCloseCursor._ Observable.now(IteratorBackedRangeVector(new CustomRangeVectorKey(Map.empty), @@ -137,6 +136,6 @@ final case class LabelValuesExec(queryContext: QueryContext, ExecResult(rvs, Task.eval(sch)) } - def args: String = s"shard=$shard, filters=$filters, col=$columns, limit=${queryContext.sampleLimit}, " + - s"startMs=$startMs, endMs=$endMs" + def args: String = s"shard=$shard, filters=$filters, col=$columns, limit=${queryContext.plannerParams.sampleLimit}," + + 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 index 667b23f398..824dcee529 100644 --- a/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala +++ b/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala @@ -17,8 +17,7 @@ case class MetadataRemoteExec(queryEndpoint: String, urlParams: Map[String, Any], queryContext: QueryContext, dispatcher: PlanDispatcher, - dataset: DatasetRef, - params: PromQlQueryParams) extends RemoteExec { + dataset: DatasetRef) extends RemoteExec { private val columns = Seq(ColumnInfo("Labels", ColumnType.MapColumn)) private val resultSchema = ResultSchema(columns, 1) diff --git a/query/src/main/scala/filodb/query/exec/MultiSchemaPartitionsExec.scala b/query/src/main/scala/filodb/query/exec/MultiSchemaPartitionsExec.scala index 5b5fb12be0..e843495176 100644 --- a/query/src/main/scala/filodb/query/exec/MultiSchemaPartitionsExec.scala +++ b/query/src/main/scala/filodb/query/exec/MultiSchemaPartitionsExec.scala @@ -46,7 +46,7 @@ final case class MultiSchemaPartitionsExec(queryContext: QueryContext, Kamon.currentSpan().mark("lookup-partitions-done") val queryTimeElapsed = System.currentTimeMillis() - queryContext.submitTime - if (queryTimeElapsed >= queryContext.queryTimeoutMillis) + if (queryTimeElapsed >= queryContext.plannerParams.queryTimeoutMillis) throw QueryTimeoutException(queryTimeElapsed, this.getClass.getName) // Find the schema if one wasn't supplied diff --git a/query/src/main/scala/filodb/query/exec/PlanDispatcher.scala b/query/src/main/scala/filodb/query/exec/PlanDispatcher.scala index 2fb68aa7b8..b948ac3ac6 100644 --- a/query/src/main/scala/filodb/query/exec/PlanDispatcher.scala +++ b/query/src/main/scala/filodb/query/exec/PlanDispatcher.scala @@ -30,7 +30,7 @@ case class ActorPlanDispatcher(target: ActorRef) extends PlanDispatcher { def dispatch(plan: ExecPlan)(implicit sched: Scheduler): Task[QueryResponse] = { val queryTimeElapsed = System.currentTimeMillis() - plan.queryContext.submitTime - val remainingTime = plan.queryContext.queryTimeoutMillis - queryTimeElapsed + val remainingTime = plan.queryContext.plannerParams.queryTimeoutMillis - queryTimeElapsed // Don't send if time left is very small if (remainingTime < 1) { Task.raiseError(QueryTimeoutException(queryTimeElapsed, this.getClass.getName)) diff --git a/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala b/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala index d20ff7e8c2..d9519b1fde 100644 --- a/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala +++ b/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala @@ -21,19 +21,19 @@ import filodb.core.store.ChunkSource import filodb.memory.format.ZeroCopyUTF8String._ import filodb.memory.format.vectors.{CustomBuckets, MutableHistogram} import filodb.query._ +import filodb.query.AggregationOperator.Avg trait RemoteExec extends LeafExecPlan with StrictLogging { - val params: PromQlQueryParams + def queryEndpoint: String - val queryEndpoint: String + def requestTimeoutMs: Long - val requestTimeoutMs: Long + def urlParams: Map[String, Any] + def promQlQueryParams: PromQlQueryParams = queryContext.origQueryParams.asInstanceOf[PromQlQueryParams] - val urlParams: Map[String, Any] - - def args: String = s"${params.toString}, queryEndpoint=$queryEndpoint, " + - s"requestTimeoutMs=$requestTimeoutMs, limit=${queryContext.sampleLimit}" + def args: String = s"${promQlQueryParams.toString}, ${queryContext.plannerParams}, queryEndpoint=$queryEndpoint, " + + s"requestTimeoutMs=$requestTimeoutMs" def limit: Int = ??? @@ -69,15 +69,17 @@ trait RemoteExec extends LeafExecPlan with StrictLogging { 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, + Map("start" -> promQlQueryParams.startSecs, + "end" -> promQlQueryParams.endSecs, + "time" -> promQlQueryParams.endSecs, + "step" -> promQlQueryParams.stepSecs, + "processFailure" -> queryContext.plannerParams.processFailure, + "processMultiPartition" -> queryContext.plannerParams.processMultiPartition, "histogramMap" -> "true", - "verbose" -> params.verbose) - if (params.spread.isDefined) finalUrlParams = finalUrlParams + ("spread" -> params.spread.get) + "skipAggregatePresent" -> queryContext.plannerParams.skipAggregatePresent, + "verbose" -> promQlQueryParams.verbose) + if (queryContext.plannerParams.spread.isDefined) finalUrlParams = finalUrlParams + ("spread" -> queryContext. + plannerParams.spread.get) logger.debug("URLParams for RemoteExec:" + finalUrlParams) finalUrlParams } @@ -88,20 +90,27 @@ case class PromQlRemoteExec(queryEndpoint: String, requestTimeoutMs: Long, queryContext: QueryContext, dispatcher: PlanDispatcher, - dataset: DatasetRef, - params: PromQlQueryParams) extends RemoteExec { + dataset: DatasetRef) extends RemoteExec { private val defaultColumns = Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), ColumnInfo("value", ColumnType.DoubleColumn)) - private val defaultRecSchema = SerializedRangeVector.toSchema(defaultColumns) - private val defaultResultSchema = ResultSchema(defaultColumns, 1) - private val histColumns = Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), - ColumnInfo("h", ColumnType.HistogramColumn)) - private val histRecSchema = SerializedRangeVector.toSchema(histColumns) - private val histResultSchema = ResultSchema(histColumns, 1) +//TODO Don't use PromQL API to talk across clusters + val columns= Map("histogram" -> Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), + ColumnInfo("h", ColumnType.HistogramColumn)), + Avg.entryName -> (defaultColumns :+ ColumnInfo("count", ColumnType.LongColumn)) , + "default" -> defaultColumns) + + val recordSchema = Map("histogram" -> SerializedRangeVector.toSchema(columns.get("histogram").get), + Avg.entryName -> SerializedRangeVector.toSchema(columns.get(Avg.entryName).get), + "default" -> SerializedRangeVector.toSchema(columns.get("default").get)) + + val resultSchema = Map("histogram" -> ResultSchema(columns.get("histogram").get, 1), + Avg.entryName -> ResultSchema(columns.get(Avg.entryName).get, 1), + "default" -> ResultSchema(columns.get("default").get, 1)) + private val builder = SerializedRangeVector.newBuilder() - override val urlParams = Map("query" -> params.promQl) + override val urlParams = Map("query" -> promQlQueryParams.promQl) override def sendHttpRequest(execPlan2Span: Span, httpTimeoutMs: Long) (implicit sched: Scheduler): Future[QueryResponse] = { @@ -127,20 +136,36 @@ case class PromQlRemoteExec(queryEndpoint: String, logger.debug("PromQlRemoteExec generating empty QueryResult as result is empty") QueryResult(id, ResultSchema.empty, Seq.empty) } else { - val samples = data.result.head.values.getOrElse(Seq(data.result.head.value.get)) - if (samples.isEmpty) { - logger.debug("PromQlRemoteExec generating empty QueryResult as samples is empty") - QueryResult(id, ResultSchema.empty, Seq.empty) - } else { - // Passing histogramMap = true so DataSampl will be HistSampl for histograms - if (samples.head.isInstanceOf[HistSampl]) genHistQueryResult(data, id) - else genDefaultQueryResult(data, id) + if (data.result.head.aggregateResponse.isDefined) genAggregateResult(data, id) + else { + val samples = data.result.head.values.getOrElse(Seq(data.result.head.value.get)) + if (samples.isEmpty) { + logger.debug("PromQlRemoteExec generating empty QueryResult as samples is empty") + QueryResult(id, ResultSchema.empty, Seq.empty) + } else { + samples.head match { + // Passing histogramMap = true so DataSampl will be HistSampl for histograms + case HistSampl(timestamp, buckets) => genHistQueryResult(data, id) + case _ => genDefaultQueryResult(data, id) + } + } } } span.finish() queryResponse } + def genAggregateResult(data: Data, id: String): QueryResult = { + + val aggregateResponse = data.result.head.aggregateResponse.get + if (aggregateResponse.aggregateSampl.isEmpty) QueryResult(id, ResultSchema.empty, Seq.empty) + else { + aggregateResponse.aggregateSampl.head match { + case AvgSampl(timestamp, value, count) => genAvgQueryResult(data, id) + } + } + } + def genDefaultQueryResult(data: Data, id: String): QueryResult = { val rangeVectors = data.result.map { r => val samples = r.values.getOrElse(Seq(r.value.get)) @@ -158,14 +183,13 @@ case class PromQlRemoteExec(queryEndpoint: String, row } } - override def numRows: Option[Int] = Option(samples.size) } - SerializedRangeVector(rv, builder, defaultRecSchema, "PromQlRemoteExec-default") + SerializedRangeVector(rv, builder, recordSchema.get("default").get, "PromQlRemoteExec-default") // TODO: Handle stitching with verbose flag } - QueryResult(id, defaultResultSchema, rangeVectors) + QueryResult(id, resultSchema.get("default").get, rangeVectors) } def genHistQueryResult(data: Data, id: String): QueryResult = { @@ -196,10 +220,35 @@ case class PromQlRemoteExec(queryEndpoint: String, override def numRows: Option[Int] = Option(samples.size) } - SerializedRangeVector(rv, builder, histRecSchema, "PromQlRemoteExec-hist") + SerializedRangeVector(rv, builder, recordSchema.get("histogram").get, "PromQlRemoteExec-hist") // TODO: Handle stitching with verbose flag } - QueryResult(id, histResultSchema, rangeVectors) + QueryResult(id, resultSchema.get("histogram").get, rangeVectors) + } + + def genAvgQueryResult(data: Data, id: String): QueryResult = { + val rangeVectors = data.result.map { d => + val rv = new RangeVector { + val row = new AvgAggTransientRow() + + override def key: RangeVectorKey = CustomRangeVectorKey(d.metric.map(m => m._1.utf8 -> m._2.utf8)) + + override def rows(): RangeVectorCursor = { + import NoCloseCursor._ + d.aggregateResponse.get.aggregateSampl.iterator.collect { case a: AvgSampl => + row.setLong(0, a.timestamp * 1000) + row.setDouble(1, a.value) + row.setLong(2, a.count) + row + } + } + override def numRows: Option[Int] = Option(d.aggregateResponse.get.aggregateSampl.size) + } + SerializedRangeVector(rv, builder, recordSchema.get(Avg.entryName).get, "PromQlRemoteExec-avg") + } + + // TODO: Handle stitching with verbose flag + QueryResult(id, resultSchema.get(Avg.entryName).get, rangeVectors) } } diff --git a/query/src/main/scala/filodb/query/exec/ScalarFixedDoubleExec.scala b/query/src/main/scala/filodb/query/exec/ScalarFixedDoubleExec.scala index a6dd0fa3c9..16ed0971a2 100644 --- a/query/src/main/scala/filodb/query/exec/ScalarFixedDoubleExec.scala +++ b/query/src/main/scala/filodb/query/exec/ScalarFixedDoubleExec.scala @@ -63,7 +63,7 @@ case class ScalarFixedDoubleExec(queryContext: QueryContext, .start() rangeVectorTransformers.foldLeft((Observable.fromIterable(rangeVectors), resultSchema)) { (acc, transf) => val paramRangeVector: Seq[Observable[ScalarRangeVector]] = transf.funcParams.map(_.getResult) - (transf.apply(acc._1, querySession, queryContext.sampleLimit, acc._2, + (transf.apply(acc._1, querySession, queryContext.plannerParams.sampleLimit, acc._2, paramRangeVector), transf.schema(acc._2)) }._1.toListL.map({ span.finish() diff --git a/query/src/main/scala/filodb/query/exec/TimeScalarGeneratorExec.scala b/query/src/main/scala/filodb/query/exec/TimeScalarGeneratorExec.scala index 49d9212385..982884eb7c 100644 --- a/query/src/main/scala/filodb/query/exec/TimeScalarGeneratorExec.scala +++ b/query/src/main/scala/filodb/query/exec/TimeScalarGeneratorExec.scala @@ -72,7 +72,7 @@ case class TimeScalarGeneratorExec(queryContext: QueryContext, rangeVectorTransformers.foldLeft((Observable.fromIterable(rangeVectors), resultSchema)) { (acc, transf) => span.mark(transf.getClass.getSimpleName) val paramRangeVector: Seq[Observable[ScalarRangeVector]] = transf.funcParams.map(_.getResult) - (transf.apply(acc._1, querySession, queryContext.sampleLimit, acc._2, + (transf.apply(acc._1, querySession, queryContext.plannerParams.sampleLimit, acc._2, paramRangeVector), transf.schema(acc._2)) }._1.toListL.map({ span.finish() diff --git a/query/src/test/scala/filodb/query/PromCirceSupportSpec.scala b/query/src/test/scala/filodb/query/PromCirceSupportSpec.scala new file mode 100644 index 0000000000..d2c76c4f21 --- /dev/null +++ b/query/src/test/scala/filodb/query/PromCirceSupportSpec.scala @@ -0,0 +1,112 @@ +package filodb.query + +import io.circe.parser +import io.circe.generic.auto._ +//import io.circe.syntax.EncoderOps +import org.scalatest.concurrent.ScalaFutures +import org.scalatest.funspec.AnyFunSpec +import org.scalatest.matchers.should.Matchers + +class PromCirceSupportSpec extends AnyFunSpec with Matchers with ScalaFutures { + + // 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._ + + it("should parse Sampl") { + val inputString = + """ + |[ + | [ + | 1600102672, + | "1.2" + | ], + | [ + | 1600102687, + | "3.1" + | ] + |] + |""".stripMargin + + parseAndValidate(inputString, List(Sampl(1600102672,1.2), Sampl(1600102687,3.1))) + } + + it("should parse aggregateResponse") { + val input = """[{ + | "status": "success", + | "data": { + | "resultType": "matrix", + | "result": [{ + | "metric": { + | + | }, + | "aggregateResponse": { + | "aggregateValues": [ + | [ + | 1601491649, + | "15.186417982460787", + | 5 + | ], + | [ + | 1601491679, + | "14.891293858511071", + | 6 + | ], + | [ + | 1601491709, + | "14.843819532173134", + | 7 + | ], + | [ + | 1601491719, + | "NaN", + | 7 + | ] + | + | ], + | "function": "avg" + | } + | }] + | }, + | "errorType": null, + | "error": null + |}]""".stripMargin + val expectedResult =List(AvgSampl(1601491649,15.186417982460787,5), + AvgSampl(1601491679,14.891293858511071,6), AvgSampl(1601491709,14.843819532173134,7), AvgSampl(1601491719, + Double.NaN, 7)) + + parser.decode[List[SuccessResponse]](input) match { + case Right(successResponse) => val aggregateResponse = successResponse.head.data.result.head.aggregateResponse.get + aggregateResponse.function shouldEqual("avg") + aggregateResponse.aggregateSampl.map(_.asInstanceOf[AvgSampl]).zip(expectedResult).foreach { + case (res, ex) => if (res.value.isNaN) { + ex.value.isNaN shouldEqual(true) + ex.count shouldEqual(res.count) + ex.timestamp shouldEqual(ex.timestamp) + } else ex shouldEqual(res) + } + case Left(ex) => println(ex) + } + } + + def parseAndValidate(input: String, expectedResult: List[DataSampl]): Unit = { + parser.decode[List[DataSampl]](input) match { + case Right(samples) => + samples.zip(expectedResult).foreach { + case (val1: Sampl, val2: Sampl) => { + val1.timestamp shouldEqual(val2.timestamp) + if (val1.value.isNaN) val2.value.isNaN shouldEqual true + else val1.value shouldEqual val2.value + } + + case (val1: HistSampl, val2: HistSampl) => { + val1.timestamp shouldEqual(val2.timestamp) + val1.buckets shouldEqual val2.buckets + } + + case _ => samples.sameElements(expectedResult) + } + case Left(ex) => throw ex + } + } +} diff --git a/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala b/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala index 82dcf6d8f3..066f50cfa0 100644 --- a/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala @@ -481,7 +481,7 @@ class BinaryJoinExecSpec extends AnyFunSpec with Matchers with ScalaFutures { } it("should throw BadQueryException - one-to-one with ignoring - cardinality limit 1") { - val queryContext = QueryContext(joinQueryCardLimit = 1) // set join card limit to 1 + val queryContext = QueryContext(plannerParams= PlannerParams(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 @@ -506,7 +506,7 @@ class BinaryJoinExecSpec extends AnyFunSpec with Matchers with ScalaFutures { } it("should throw BadQueryException - one-to-one with on - cardinality limit 1") { - val queryContext = QueryContext(joinQueryCardLimit = 1) // set join card limit to 1 + val queryContext = QueryContext(plannerParams = PlannerParams(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 diff --git a/query/src/test/scala/filodb/query/exec/BinaryJoinGroupingSpec.scala b/query/src/test/scala/filodb/query/exec/BinaryJoinGroupingSpec.scala index e946486123..72385d668b 100644 --- a/query/src/test/scala/filodb/query/exec/BinaryJoinGroupingSpec.scala +++ b/query/src/test/scala/filodb/query/exec/BinaryJoinGroupingSpec.scala @@ -391,7 +391,7 @@ class BinaryJoinGroupingSpec extends AnyFunSpec with Matchers with ScalaFutures } it("should throw BadQueryException - many-to-one with on - cardinality limit 1") { - val queryContext = QueryContext(joinQueryCardLimit = 1) // set join card limit to 1 + val queryContext = QueryContext(plannerParams= PlannerParams(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, @@ -418,7 +418,7 @@ class BinaryJoinGroupingSpec extends AnyFunSpec with Matchers with ScalaFutures } it("should throw BadQueryException - many-to-one with ignoring - cardinality limit 1") { - val queryContext = QueryContext(joinQueryCardLimit = 1) // set join card limit to 1 + val queryContext = QueryContext(plannerParams= PlannerParams(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, @@ -445,7 +445,7 @@ class BinaryJoinGroupingSpec extends AnyFunSpec with Matchers with ScalaFutures } 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 queryContext = QueryContext(plannerParams= PlannerParams(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) diff --git a/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala b/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala index e5803b17b6..915742765b 100644 --- a/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala @@ -2,7 +2,6 @@ package filodb.query.exec import scala.collection.mutable.ArrayBuffer import scala.concurrent.duration._ - import com.typesafe.config.ConfigFactory import monix.eval.Task import monix.execution.Scheduler @@ -10,13 +9,12 @@ import monix.execution.Scheduler.Implicits.global import org.scalatest.BeforeAndAfterAll import org.scalatest.concurrent.ScalaFutures import org.scalatest.time.{Millis, Seconds, Span} - import filodb.core.MetricsTestData._ import filodb.core.TestData import filodb.core.binaryrecord2.BinaryRecordRowReader import filodb.core.memstore.{FixedMaxPartitionsEvictionPolicy, SomeData, TimeSeriesMemStore} import filodb.core.metadata.Schemas -import filodb.core.query.{ColumnFilter, Filter, QueryConfig, QueryContext, QuerySession, SerializedRangeVector} +import filodb.core.query.{ColumnFilter, Filter, PlannerParams, QueryConfig, QueryContext, QuerySession, SerializedRangeVector} import filodb.core.store.{InMemoryMetaStore, NullColumnStore} import filodb.memory.format.{SeqRowReader, ZeroCopyUTF8String} import filodb.query._ @@ -142,7 +140,7 @@ class MetadataExecSpec extends AnyFunSpec with Matchers with ScalaFutures with B val filters = Seq (ColumnFilter("job", Filter.Equals("myCoolService".utf8))) //Reducing limit results in truncated metadata response - val execPlan = PartKeysExec(QueryContext(sampleLimit = limit-1), dummyDispatcher, + val execPlan = PartKeysExec(QueryContext(plannerParams= PlannerParams(sampleLimit = limit - 1)), dummyDispatcher, timeseriesDataset.ref, 0, filters, false, now-5000, now) val resp = execPlan.execute(memStore, querySession).runAsync.futureValue diff --git a/query/src/test/scala/filodb/query/exec/MultiSchemaPartitionsExecSpec.scala b/query/src/test/scala/filodb/query/exec/MultiSchemaPartitionsExecSpec.scala index 63a7bcbbec..99c33dc0f9 100644 --- a/query/src/test/scala/filodb/query/exec/MultiSchemaPartitionsExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/MultiSchemaPartitionsExecSpec.scala @@ -448,8 +448,8 @@ class MultiSchemaPartitionsExecSpec extends AnyFunSpec with Matchers with ScalaF // Query returns n ("numRawSamples") samples - Applying Limit (n-1) to fail the query execution // with ResponseTooLargeException - val execPlan = MultiSchemaPartitionsExec(QueryContext(sampleLimit = 999), dummyDispatcher, - dsRef, 0, filters, AllChunkScan) + val execPlan = MultiSchemaPartitionsExec(QueryContext(plannerParams= PlannerParams(sampleLimit = 999)), + dummyDispatcher, dsRef, 0, filters, AllChunkScan) val resp = execPlan.execute(memStore, querySession).runAsync.futureValue val result = resp.asInstanceOf[QueryError] diff --git a/query/src/test/scala/filodb/query/exec/PromQlRemoteExecSpec.scala b/query/src/test/scala/filodb/query/exec/PromQlRemoteExecSpec.scala index 180f01015c..bb56d2397b 100644 --- a/query/src/test/scala/filodb/query/exec/PromQlRemoteExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/PromQlRemoteExecSpec.scala @@ -25,11 +25,11 @@ class PromQlRemoteExecSpec extends AnyFunSpec with Matchers with ScalaFutures { (implicit sched: Scheduler): Task[QueryResponse] = ??? } - val queryContext = QueryContext() val params = PromQlQueryParams("", 0, 0 , 0) + val queryContext = QueryContext(origQueryParams = params) it ("should convert matrix Data to QueryResponse ") { val expectedResult = List((1000000, 1.0), (2000000, 2.0), (3000000, 3.0)) - val exec = PromQlRemoteExec("", 60000, queryContext, dummyDispatcher, timeseriesDataset.ref, params) + val exec = PromQlRemoteExec("", 60000, queryContext, dummyDispatcher, timeseriesDataset.ref) 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()) @@ -42,7 +42,7 @@ class PromQlRemoteExecSpec extends AnyFunSpec with Matchers with ScalaFutures { it ("should convert vector Data to QueryResponse ") { val expectedResult = List((1000000, 1.0)) - val exec = PromQlRemoteExec("", 60000, queryContext, dummyDispatcher, timeseriesDataset.ref, params) + val exec = PromQlRemoteExec("", 60000, queryContext, dummyDispatcher, timeseriesDataset.ref) 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 @@ -55,7 +55,7 @@ class PromQlRemoteExecSpec extends AnyFunSpec with Matchers with ScalaFutures { it ("should convert vector Data to QueryResponse for MetadataQuery") { val exec = MetadataRemoteExec("", 60000, Map.empty, - queryContext, dummyDispatcher, timeseriesDataset.ref, params) + queryContext, dummyDispatcher, timeseriesDataset.ref) 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()) @@ -67,7 +67,7 @@ class PromQlRemoteExecSpec extends AnyFunSpec with Matchers with ScalaFutures { } it ("should convert vector Data to QueryResponse for Metadata series query") { - val exec = MetadataRemoteExec("", 60000, Map.empty, queryContext, dummyDispatcher, timeseriesDataset.ref, params) + val exec = MetadataRemoteExec("", 60000, Map.empty, queryContext, dummyDispatcher, timeseriesDataset.ref) 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()) @@ -79,7 +79,7 @@ class PromQlRemoteExecSpec extends AnyFunSpec with Matchers with ScalaFutures { } it ("should convert histogram to QueryResponse ") { - val exec = PromQlRemoteExec("", 60000, queryContext, dummyDispatcher, timeseriesDataset.ref, params) + val exec = PromQlRemoteExec("", 60000, queryContext, dummyDispatcher, timeseriesDataset.ref) val result = query.Result (Map("instance" -> "inst1"), None, Some(HistSampl(1000, Map("1" -> 2, "+Inf" -> 3)))) val res = exec.toQueryResponse(Data("vector", Seq(result)), "id", Kamon.currentSpan()) res.isInstanceOf[QueryResult] shouldEqual true diff --git a/spark-jobs/src/test/scala/filodb/downsampler/DownsamplerMainSpec.scala b/spark-jobs/src/test/scala/filodb/downsampler/DownsamplerMainSpec.scala index 2c9bae0476..7d0ff114c6 100644 --- a/spark-jobs/src/test/scala/filodb/downsampler/DownsamplerMainSpec.scala +++ b/spark-jobs/src/test/scala/filodb/downsampler/DownsamplerMainSpec.scala @@ -665,8 +665,8 @@ class DownsamplerMainSpec extends AnyFunSpec with Matchers with BeforeAndAfterAl Seq(gaugeName, gaugeLowFreqName, counterName, histName).foreach { metricName => val queryFilters = colFilters :+ ColumnFilter("_metric_", Equals(metricName)) - val exec = MultiSchemaPartitionsExec(QueryContext(sampleLimit = 1000), InProcessPlanDispatcher, - batchDownsampler.rawDatasetRef, 0, queryFilters, AllChunkScan) + val exec = MultiSchemaPartitionsExec(QueryContext(plannerParams = PlannerParams(sampleLimit = 1000)), + InProcessPlanDispatcher, batchDownsampler.rawDatasetRef, 0, queryFilters, AllChunkScan) val querySession = QuerySession(QueryContext(), queryConfig) val queryScheduler = Scheduler.fixedPool(s"$QuerySchedName", 3) @@ -693,8 +693,9 @@ class DownsamplerMainSpec extends AnyFunSpec with Matchers with BeforeAndAfterAl val colFilters = seriesTags.map { case (t, v) => ColumnFilter(t.toString, Equals(v.toString)) }.toSeq val queryFilters = colFilters :+ ColumnFilter("_metric_", Equals(untypedName)) - val exec = MultiSchemaPartitionsExec(QueryContext(sampleLimit = 1000), InProcessPlanDispatcher, - batchDownsampler.rawDatasetRef, 0, queryFilters, AllChunkScan) + val exec = MultiSchemaPartitionsExec(QueryContext(plannerParams + = PlannerParams(sampleLimit = 1000)), InProcessPlanDispatcher, batchDownsampler.rawDatasetRef, 0, + queryFilters, AllChunkScan) val querySession = QuerySession(QueryContext(), queryConfig) val queryScheduler = Scheduler.fixedPool(s"$QuerySchedName", 3) @@ -715,9 +716,8 @@ class DownsamplerMainSpec extends AnyFunSpec with Matchers with BeforeAndAfterAl downsampleTSStore.recoverIndex(batchDownsampler.rawDatasetRef, 0).futureValue 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 = 1000), InProcessPlanDispatcher, - batchDownsampler.rawDatasetRef, 0, queryFilters, AllChunkScan, - colName = Option("sum")) + val exec = MultiSchemaPartitionsExec(QueryContext(plannerParams = PlannerParams(sampleLimit = 1000)), + InProcessPlanDispatcher, batchDownsampler.rawDatasetRef, 0, queryFilters, AllChunkScan, colName = Option("sum")) val querySession = QuerySession(QueryContext(), queryConfig) val queryScheduler = Scheduler.fixedPool(s"$QuerySchedName", 3) val res = exec.execute(downsampleTSStore, querySession)(queryScheduler) From f4d79de9f206fb801fad03ca7889255ae10ff63b Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Mon, 26 Oct 2020 13:12:59 -0700 Subject: [PATCH 02/33] bug(query): Small misc low-risk cleanup changes (#934) --- .../filodb.core/memstore/TimeSeriesShard.scala | 7 +++---- .../main/scala/filodb.memory/MemFactory.scala | 5 +++++ .../main/scala/filodb/query/exec/ExecPlan.scala | 16 +++++++++------- .../filodb/query/exec/MetadataExecPlan.scala | 2 -- .../filodb/query/exec/SelectChunkInfosExec.scala | 2 -- .../query/exec/TimeScalarGeneratorExec.scala | 1 - 6 files changed, 17 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala index b84183dbf2..baff77d164 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala @@ -282,12 +282,11 @@ class TimeSeriesShard(val ref: DatasetRef, // 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)" + + Shutdown.haltAndCatchFire( new RuntimeException(f"POSSIBLE CORRUPTION DURING onReclaim(" + + f"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 + s"partID=$partID ${partition.stringPartition}")) } partition.removeChunksAt(chunkID) logger.debug(s"Reclaiming chunk chunkID=$chunkID from shard=$shardNum " + diff --git a/memory/src/main/scala/filodb.memory/MemFactory.scala b/memory/src/main/scala/filodb.memory/MemFactory.scala index a3325aef04..f6d59d8938 100644 --- a/memory/src/main/scala/filodb.memory/MemFactory.scala +++ b/memory/src/main/scala/filodb.memory/MemFactory.scala @@ -317,6 +317,11 @@ class BlockMemFactory(blockStore: BlockManager, fullBlocksToBeMarkedAsReclaimable.clear() } + /** + * If current block has the capacity, no-op + * Otherwise, move to next block. As a result, "currentBlock" + * now points to a new block + */ protected def ensureCapacity(forSize: Long): Block = synchronized { var block = accessCurrentBlock() if (block.hasCapacity(forSize)) { diff --git a/query/src/main/scala/filodb/query/exec/ExecPlan.scala b/query/src/main/scala/filodb/query/exec/ExecPlan.scala index 05bebc7757..7da3572161 100644 --- a/query/src/main/scala/filodb/query/exec/ExecPlan.scala +++ b/query/src/main/scala/filodb/query/exec/ExecPlan.scala @@ -4,6 +4,7 @@ import scala.collection.mutable.ArrayBuffer import scala.util.control.NonFatal import kamon.Kamon +import kamon.metric.MeasurementUnit import monix.eval.Task import monix.execution.Scheduler import monix.reactive.Observable @@ -115,7 +116,8 @@ trait ExecPlan extends QueryCommand { // kamon uses thread-locals. Kamon.runWithSpan(span, true) { val doEx = doExecute(source, querySession) - Kamon.histogram("query-execute-time-elapsed-step1-done") + Kamon.histogram("query-execute-time-elapsed-step1-done", + MeasurementUnit.time.milliseconds) .withTag("plan", getClass.getSimpleName) .record(System.currentTimeMillis - startExecute) doEx @@ -124,7 +126,8 @@ trait ExecPlan extends QueryCommand { // Step 2: Run connect monix pipeline to transformers, materialize the result def step2(res: ExecResult) = res.schema.map { resSchema => - Kamon.histogram("query-execute-time-elapsed-step2-start") + Kamon.histogram("query-execute-time-elapsed-step2-start", + MeasurementUnit.time.milliseconds) .withTag("plan", getClass.getSimpleName) .record(System.currentTimeMillis - startExecute) val span = Kamon.spanBuilder(s"execute-step2-${getClass.getSimpleName}") @@ -147,7 +150,8 @@ trait ExecPlan extends QueryCommand { paramRangeVector), transf.schema(acc._2)) } val recSchema = SerializedRangeVector.toSchema(finalRes._2.columns, finalRes._2.brSchemas) - Kamon.histogram("query-execute-time-elapsed-step2-transformer-pipeline-setup") + Kamon.histogram("query-execute-time-elapsed-step2-transformer-pipeline-setup", + MeasurementUnit.time.milliseconds) .withTag("plan", getClass.getSimpleName) .record(System.currentTimeMillis - startExecute) val builder = SerializedRangeVector.newBuilder() @@ -173,7 +177,8 @@ trait ExecPlan extends QueryCommand { } .toListL .map { r => - Kamon.histogram("query-execute-time-elapsed-step2-result-materialized") + Kamon.histogram("query-execute-time-elapsed-step2-result-materialized", + MeasurementUnit.time.milliseconds) .withTag("plan", getClass.getSimpleName) .record(System.currentTimeMillis - startExecute) val numBytes = builder.allContainers.map(_.numBytes).sum @@ -387,7 +392,6 @@ abstract class NonLeafExecPlan extends ExecPlan { querySession: QuerySession) (implicit sched: Scheduler): ExecResult = { val parentSpan = Kamon.currentSpan() - parentSpan.mark("create-child-tasks") // whether child tasks need to be executed sequentially. // parallelism 1 means, only one worker thread to process underlying tasks. @@ -418,9 +422,7 @@ abstract class NonLeafExecPlan extends ExecPlan { val outputSchema = processedTasks.collect { case (QueryResult(_, schema, _), _) => schema }.firstOptionL.map(_.getOrElse(ResultSchema.empty)) - parentSpan.mark("output-compose") val outputRvs = compose(processedTasks, outputSchema, querySession) - parentSpan.mark("return-results") ExecResult(outputRvs, outputSchema) } diff --git a/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala b/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala index 1b921baa40..839b998408 100644 --- a/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala +++ b/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala @@ -91,7 +91,6 @@ final case class PartKeysExec(queryContext: QueryContext, case other => Observable.empty } - Kamon.currentSpan().mark("creating-resultschema") val sch = ResultSchema(Seq(ColumnInfo("Labels", ColumnType.MapColumn)), 1) ExecResult(rvs, Task.eval(sch)) } @@ -131,7 +130,6 @@ final case class LabelValuesExec(queryContext: QueryContext, } else { Observable.empty } - parentSpan.mark("creating-resultschema") val sch = ResultSchema(Seq(ColumnInfo("Labels", ColumnType.MapColumn)), 1) ExecResult(rvs, Task.eval(sch)) } diff --git a/query/src/main/scala/filodb/query/exec/SelectChunkInfosExec.scala b/query/src/main/scala/filodb/query/exec/SelectChunkInfosExec.scala index 915624457d..27e51f3b6c 100644 --- a/query/src/main/scala/filodb/query/exec/SelectChunkInfosExec.scala +++ b/query/src/main/scala/filodb/query/exec/SelectChunkInfosExec.scala @@ -1,6 +1,5 @@ package filodb.query.exec -import kamon.Kamon import monix.eval.Task import monix.execution.Scheduler @@ -53,7 +52,6 @@ final case class SelectChunkInfosExec(queryContext: QueryContext, val dataColumn = dataSchema.data.columns(colID) val partCols = dataSchema.partitionInfos val numGroups = source.groupsInDataset(dataset) - Kamon.currentSpan().mark("creating-scanpartitions") val rvs = source.scanPartitions(dataset, lookupRes, Seq.empty, querySession) .filter(_.hasChunks(chunkMethod)) .map { partition => diff --git a/query/src/main/scala/filodb/query/exec/TimeScalarGeneratorExec.scala b/query/src/main/scala/filodb/query/exec/TimeScalarGeneratorExec.scala index 982884eb7c..220d8b3c5f 100644 --- a/query/src/main/scala/filodb/query/exec/TimeScalarGeneratorExec.scala +++ b/query/src/main/scala/filodb/query/exec/TimeScalarGeneratorExec.scala @@ -70,7 +70,6 @@ case class TimeScalarGeneratorExec(queryContext: QueryContext, .tag("query-id", queryContext.queryId) .start() rangeVectorTransformers.foldLeft((Observable.fromIterable(rangeVectors), resultSchema)) { (acc, transf) => - span.mark(transf.getClass.getSimpleName) val paramRangeVector: Seq[Observable[ScalarRangeVector]] = transf.funcParams.map(_.getResult) (transf.apply(acc._1, querySession, queryContext.plannerParams.sampleLimit, acc._2, paramRangeVector), transf.schema(acc._2)) From 58cc70f042fa2da77ea1663e17b02fdf769c5c87 Mon Sep 17 00:00:00 2001 From: Jackson Jeyapaul Date: Mon, 26 Oct 2020 18:34:39 -0700 Subject: [PATCH 03/33] feat(query): Allow custom AsyncHttpClientConfig in RemoteExec (#933) --- .../HighAvailabilityPlanner.scala | 11 +- .../queryplanner/MultiPartitionPlanner.scala | 12 +- .../queryplanner/PlannerMaterializer.scala | 6 +- .../HighAvailabilityPlannerSpec.scala | 4 +- .../MultiPartitionPlannerSpec.scala | 1 + .../query/exec/MetadataRemoteExec.scala | 8 +- .../filodb/query/exec/PromQlRemoteExec.scala | 125 +----------- .../scala/filodb/query/exec/RemoteExec.scala | 187 ++++++++++++++++++ .../query/exec/PromQlRemoteExecSpec.scala | 13 +- 9 files changed, 224 insertions(+), 143 deletions(-) create mode 100644 query/src/main/scala/filodb/query/exec/RemoteExec.scala diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/HighAvailabilityPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/HighAvailabilityPlanner.scala index a8c0328a2a..ff7b64709f 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/HighAvailabilityPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/HighAvailabilityPlanner.scala @@ -21,7 +21,9 @@ import filodb.query.exec._ class HighAvailabilityPlanner(dsRef: DatasetRef, localPlanner: QueryPlanner, failureProvider: FailureProvider, - queryConfig: QueryConfig) extends QueryPlanner with StrictLogging { + queryConfig: QueryConfig, + remoteExecHttpClient: RemoteExecHttpClient = RemoteHttpClient.defaultClient) + extends QueryPlanner with StrictLogging { import net.ceedubs.ficus.Ficus._ import LogicalPlanUtils._ @@ -79,12 +81,13 @@ class HighAvailabilityPlanner(dsRef: DatasetRef, rootLogicalPlan match { case lp: LabelValues => MetadataRemoteExec(httpEndpoint, remoteHttpTimeoutMs, PlannerUtil.getLabelValuesUrlParams(lp, queryParams), newQueryContext, - InProcessPlanDispatcher, dsRef) + InProcessPlanDispatcher, dsRef, remoteExecHttpClient) case lp: SeriesKeysByFilters => val urlParams = Map("match[]" -> queryParams.promQl) MetadataRemoteExec(httpEndpoint, remoteHttpTimeoutMs, - urlParams, newQueryContext, InProcessPlanDispatcher, dsRef) + urlParams, newQueryContext, InProcessPlanDispatcher, + dsRef, remoteExecHttpClient) case _ => PromQlRemoteExec(httpEndpoint, remoteHttpTimeoutMs, - newQueryContext, InProcessPlanDispatcher, dsRef) + newQueryContext, InProcessPlanDispatcher, dsRef, remoteExecHttpClient) } } diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala index 00b34ece4e..720a7dbeaf 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala @@ -20,7 +20,9 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider localPartitionPlanner: QueryPlanner, localPartitionName: String, dataset: Dataset, - queryConfig: QueryConfig) extends QueryPlanner with StrictLogging { + queryConfig: QueryConfig, + remoteExecHttpClient: RemoteExecHttpClient = RemoteHttpClient.defaultClient) + extends QueryPlanner with StrictLogging { import net.ceedubs.ficus.Ficus._ @@ -118,7 +120,7 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider else { val httpEndpoint = p.endPoint + queryParams.remoteQueryPath.getOrElse("") PromQlRemoteExec(httpEndpoint, remoteHttpTimeoutMs, generateRemoteExecParams(qContext, startMs, endMs), - InProcessPlanDispatcher, dataset.ref) + InProcessPlanDispatcher, dataset.ref, remoteExecHttpClient) } } if (execPlans.size == 1) execPlans.head @@ -144,8 +146,8 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider else { val httpEndpoint = partitions.head.endPoint + queryParams.remoteQueryPath.getOrElse("") PromQlRemoteExec(httpEndpoint, remoteHttpTimeoutMs, generateRemoteExecParams(qContext, - queryParams.startSecs * 1000, queryParams.endSecs * 1000), InProcessPlanDispatcher, dataset.ref - ) + queryParams.startSecs * 1000, queryParams.endSecs * 1000), + InProcessPlanDispatcher, dataset.ref, remoteExecHttpClient) } } else throw new UnsupportedOperationException("Binary Join across multiple partitions not supported") @@ -191,6 +193,6 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider val httpEndpoint = partitionAssignment.endPoint + finalQueryContext.origQueryParams.asInstanceOf[PromQlQueryParams]. remoteQueryPath.getOrElse("") MetadataRemoteExec(httpEndpoint, remoteHttpTimeoutMs, - urlParams, finalQueryContext, InProcessPlanDispatcher, dataset.ref) + urlParams, finalQueryContext, InProcessPlanDispatcher, dataset.ref, remoteExecHttpClient) } } diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/PlannerMaterializer.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/PlannerMaterializer.scala index b5ed10a9ca..a0d0a0548c 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/PlannerMaterializer.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/PlannerMaterializer.scala @@ -2,6 +2,8 @@ package filodb.coordinator.queryplanner import java.util.concurrent.ThreadLocalRandom +import com.typesafe.scalalogging.StrictLogging + import filodb.core.metadata.{DatasetOptions, Schemas} import filodb.core.query.{PromQlQueryParams, QueryContext, RangeParams} import filodb.prometheus.ast.Vectors.PromMetricLabel @@ -141,7 +143,8 @@ trait PlannerMaterializer { } } -object PlannerUtil { +object PlannerUtil extends StrictLogging { + /** * Returns URL params for label values which is used to create Metadata remote exec plan */ @@ -152,4 +155,5 @@ object PlannerUtil { head}$quote"""}.mkString(",") Map("filter" -> filters, "labels" -> lp.labelNames.mkString(",")) } + } diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/HighAvailabilityPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/HighAvailabilityPlannerSpec.scala index ad30fd05f9..a2689a31f8 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/HighAvailabilityPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/HighAvailabilityPlannerSpec.scala @@ -31,8 +31,8 @@ class HighAvailabilityPlannerSpec extends AnyFunSpec with Matchers { private val dsRef = dataset.ref private val schemas = Schemas(dataset.schema) - private val routingConfigString = "routing {\n remote {\n " + - " http {\n endpoint = localhost\n timeout = 10000\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) diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala index ced9bdacc6..33fac5fe45 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala @@ -5,6 +5,7 @@ import akka.testkit.TestProbe import com.typesafe.config.ConfigFactory import org.scalatest.funspec.AnyFunSpec import org.scalatest.matchers.should.Matchers + import filodb.coordinator.ShardMapper import filodb.core.MetricsTestData import filodb.core.metadata.Schemas diff --git a/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala b/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala index 824dcee529..567c2e4c89 100644 --- a/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala +++ b/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala @@ -1,9 +1,10 @@ package filodb.query.exec +import scala.concurrent.Future + 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 @@ -17,7 +18,8 @@ case class MetadataRemoteExec(queryEndpoint: String, urlParams: Map[String, Any], queryContext: QueryContext, dispatcher: PlanDispatcher, - dataset: DatasetRef) extends RemoteExec { + dataset: DatasetRef, + remoteExecHttpClient: RemoteExecHttpClient) extends RemoteExec { private val columns = Seq(ColumnInfo("Labels", ColumnType.MapColumn)) private val resultSchema = ResultSchema(columns, 1) @@ -26,7 +28,7 @@ case class MetadataRemoteExec(queryEndpoint: String, override def sendHttpRequest(execPlan2Span: Span, httpTimeoutMs: Long) (implicit sched: Scheduler): Future[QueryResponse] = { - PromRemoteExec.httpMetadataGet(queryEndpoint, httpTimeoutMs, queryContext.submitTime, getUrlParams()) + remoteExecHttpClient.httpMetadataGet(queryEndpoint, httpTimeoutMs, queryContext.submitTime, getUrlParams()) .map { response => response.unsafeBody match { case Left(error) => QueryError(queryContext.queryId, error.error) diff --git a/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala b/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala index d9519b1fde..e4c6fa3945 100644 --- a/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala +++ b/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala @@ -1,96 +1,25 @@ 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 import kamon.Kamon import kamon.trace.Span -import monix.eval.Task import monix.execution.Scheduler import filodb.core.DatasetRef import filodb.core.metadata.Column.ColumnType import filodb.core.query._ -import filodb.core.store.ChunkSource import filodb.memory.format.ZeroCopyUTF8String._ import filodb.memory.format.vectors.{CustomBuckets, MutableHistogram} import filodb.query._ import filodb.query.AggregationOperator.Avg -trait RemoteExec extends LeafExecPlan with StrictLogging { - - def queryEndpoint: String - - def requestTimeoutMs: Long - - def urlParams: Map[String, Any] - def promQlQueryParams: PromQlQueryParams = queryContext.origQueryParams.asInstanceOf[PromQlQueryParams] - - def args: String = s"${promQlQueryParams.toString}, ${queryContext.plannerParams}, queryEndpoint=$queryEndpoint, " + - s"requestTimeoutMs=$requestTimeoutMs" - - 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, requestTimeoutMs)) - } - } - - def sendHttpRequest(execPlan2Span: Span, httpTimeoutMs: Long) - (implicit sched: Scheduler): Future[QueryResponse] - - def getUrlParams(): Map[String, Any] = { - var finalUrlParams = urlParams ++ - Map("start" -> promQlQueryParams.startSecs, - "end" -> promQlQueryParams.endSecs, - "time" -> promQlQueryParams.endSecs, - "step" -> promQlQueryParams.stepSecs, - "processFailure" -> queryContext.plannerParams.processFailure, - "processMultiPartition" -> queryContext.plannerParams.processMultiPartition, - "histogramMap" -> "true", - "skipAggregatePresent" -> queryContext.plannerParams.skipAggregatePresent, - "verbose" -> promQlQueryParams.verbose) - if (queryContext.plannerParams.spread.isDefined) finalUrlParams = finalUrlParams + ("spread" -> queryContext. - plannerParams.spread.get) - logger.debug("URLParams for RemoteExec:" + finalUrlParams) - finalUrlParams - } - -} - case class PromQlRemoteExec(queryEndpoint: String, requestTimeoutMs: Long, queryContext: QueryContext, dispatcher: PlanDispatcher, - dataset: DatasetRef) extends RemoteExec { + dataset: DatasetRef, + remoteExecHttpClient: RemoteExecHttpClient) extends RemoteExec { private val defaultColumns = Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), ColumnInfo("value", ColumnType.DoubleColumn)) @@ -114,7 +43,7 @@ case class PromQlRemoteExec(queryEndpoint: String, override def sendHttpRequest(execPlan2Span: Span, httpTimeoutMs: Long) (implicit sched: Scheduler): Future[QueryResponse] = { - PromRemoteExec.httpGet(queryEndpoint, requestTimeoutMs, queryContext.submitTime, getUrlParams()) + remoteExecHttpClient.httpGet(queryEndpoint, requestTimeoutMs, queryContext.submitTime, getUrlParams()) .map { response => response.unsafeBody match { case Left(error) => QueryError(queryContext.queryId, error.error) @@ -252,51 +181,3 @@ case class PromQlRemoteExec(queryEndpoint: String, } } - -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/main/scala/filodb/query/exec/RemoteExec.scala b/query/src/main/scala/filodb/query/exec/RemoteExec.scala new file mode 100644 index 0000000000..e2e9e0f81e --- /dev/null +++ b/query/src/main/scala/filodb/query/exec/RemoteExec.scala @@ -0,0 +1,187 @@ +package filodb.query.exec + +import java.util.concurrent.TimeUnit + +import scala.concurrent.Future +import scala.concurrent.duration.FiniteDuration +import scala.sys.ShutdownHookThread + +import com.softwaremill.sttp.{DeserializationError, Response, SttpBackendOptions} +import com.softwaremill.sttp.SttpBackendOptions.ProxyType.{Http, Socks} +import com.softwaremill.sttp.asynchttpclient.future.AsyncHttpClientFutureBackend +import com.softwaremill.sttp.circe.asJson +import com.typesafe.scalalogging.StrictLogging +import kamon.Kamon +import kamon.trace.Span +import monix.eval.Task +import monix.execution.Scheduler +import org.asynchttpclient.{AsyncHttpClientConfig, DefaultAsyncHttpClientConfig} +import org.asynchttpclient.proxy.ProxyServer + +import filodb.core.query.{PromQlQueryParams, QuerySession} +import filodb.core.store.ChunkSource +import filodb.query.{BadQueryException, MetadataSuccessResponse, PromCirceSupport, QueryResponse, SuccessResponse} + +trait RemoteExec extends LeafExecPlan with StrictLogging { + + def queryEndpoint: String + + def remoteExecHttpClient: RemoteExecHttpClient + + def requestTimeoutMs: Long + + def urlParams: Map[String, Any] + + def promQlQueryParams: PromQlQueryParams = queryContext.origQueryParams.asInstanceOf[PromQlQueryParams] + + def args: String = s"${promQlQueryParams.toString}, ${queryContext.plannerParams}, queryEndpoint=$queryEndpoint, " + + s"requestTimeoutMs=$requestTimeoutMs" + + 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, requestTimeoutMs)) + } + } + + def sendHttpRequest(execPlan2Span: Span, httpTimeoutMs: Long) + (implicit sched: Scheduler): Future[QueryResponse] + + def getUrlParams(): Map[String, Any] = { + var finalUrlParams = urlParams ++ + Map("start" -> promQlQueryParams.startSecs, + "end" -> promQlQueryParams.endSecs, + "time" -> promQlQueryParams.endSecs, + "step" -> promQlQueryParams.stepSecs, + "processFailure" -> queryContext.plannerParams.processFailure, + "processMultiPartition" -> queryContext.plannerParams.processMultiPartition, + "histogramMap" -> "true", + "skipAggregatePresent" -> queryContext.plannerParams.skipAggregatePresent, + "verbose" -> promQlQueryParams.verbose) + if (queryContext.plannerParams.spread.isDefined) finalUrlParams = finalUrlParams + ("spread" -> queryContext. + plannerParams.spread.get) + logger.debug("URLParams for RemoteExec:" + finalUrlParams) + finalUrlParams + } + +} + +/** + * A trait for remoteExec GET Queries. + */ +trait RemoteExecHttpClient extends StrictLogging { + + def httpGet(httpEndpoint: String, httpTimeoutMs: Long, submitTime: Long, urlParams: Map[String, Any]) + (implicit scheduler: Scheduler): + Future[Response[scala.Either[DeserializationError[io.circe.Error], SuccessResponse]]] + + def httpMetadataGet(httpEndpoint: String, httpTimeoutMs: Long, submitTime: Long, urlParams: Map[String, Any]) + (implicit scheduler: Scheduler): + Future[Response[scala.Either[DeserializationError[io.circe.Error], MetadataSuccessResponse]]] + +} + +class RemoteHttpClient private(asyncHttpClientConfig: AsyncHttpClientConfig) extends RemoteExecHttpClient { + + 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._ + private implicit val backend = AsyncHttpClientFutureBackend.usingConfig(asyncHttpClientConfig) + + 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() + } +} + +object RemoteHttpClient { + + import scala.collection.JavaConverters._ + + /** + * A default prom remote http client backend from DefaultPromRemoteHttpClientFactory. + */ + def configBuilder(): DefaultAsyncHttpClientConfig.Builder = { + // A copy of private AsyncHttpClientBackend.defaultClient. + var configBuilder = new DefaultAsyncHttpClientConfig.Builder() + .setConnectTimeout(SttpBackendOptions.Default.connectionTimeout.toMillis.toInt) + configBuilder = SttpBackendOptions.Default.proxy match { + case None => configBuilder + case Some(p) => + val proxyType: org.asynchttpclient.proxy.ProxyType = + p.proxyType match { + case Socks => org.asynchttpclient.proxy.ProxyType.SOCKS_V5 + case Http => org.asynchttpclient.proxy.ProxyType.HTTP + } + + configBuilder.setProxyServer( + new ProxyServer.Builder(p.host, p.port) + .setProxyType(proxyType) // Fix issue #145 + .setNonProxyHosts(p.nonProxyHosts.asJava) + .build()) + } + configBuilder + } + + val defaultClient = RemoteHttpClient(configBuilder().build()) + + def apply(asyncHttpClientConfig: AsyncHttpClientConfig): RemoteHttpClient = + new RemoteHttpClient(asyncHttpClientConfig) + +} diff --git a/query/src/test/scala/filodb/query/exec/PromQlRemoteExecSpec.scala b/query/src/test/scala/filodb/query/exec/PromQlRemoteExecSpec.scala index bb56d2397b..8c1c4f94cd 100644 --- a/query/src/test/scala/filodb/query/exec/PromQlRemoteExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/PromQlRemoteExecSpec.scala @@ -9,7 +9,7 @@ import org.scalatest.matchers.should.Matchers import filodb.core.metadata.{Dataset, DatasetOptions} import filodb.core.query.{PromQlQueryParams, QueryContext} -import filodb.memory.format.vectors.{MutableHistogram} +import filodb.memory.format.vectors.MutableHistogram import filodb.query import filodb.query.{Data, HistSampl, QueryResponse, QueryResult, Sampl} @@ -29,7 +29,7 @@ class PromQlRemoteExecSpec extends AnyFunSpec with Matchers with ScalaFutures { val queryContext = QueryContext(origQueryParams = params) it ("should convert matrix Data to QueryResponse ") { val expectedResult = List((1000000, 1.0), (2000000, 2.0), (3000000, 3.0)) - val exec = PromQlRemoteExec("", 60000, queryContext, dummyDispatcher, timeseriesDataset.ref) + val exec = PromQlRemoteExec("", 60000, queryContext, dummyDispatcher, timeseriesDataset.ref, RemoteHttpClient.defaultClient) 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()) @@ -42,7 +42,7 @@ class PromQlRemoteExecSpec extends AnyFunSpec with Matchers with ScalaFutures { it ("should convert vector Data to QueryResponse ") { val expectedResult = List((1000000, 1.0)) - val exec = PromQlRemoteExec("", 60000, queryContext, dummyDispatcher, timeseriesDataset.ref) + val exec = PromQlRemoteExec("", 60000, queryContext, dummyDispatcher, timeseriesDataset.ref, RemoteHttpClient.defaultClient) 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 @@ -55,7 +55,7 @@ class PromQlRemoteExecSpec extends AnyFunSpec with Matchers with ScalaFutures { it ("should convert vector Data to QueryResponse for MetadataQuery") { val exec = MetadataRemoteExec("", 60000, Map.empty, - queryContext, dummyDispatcher, timeseriesDataset.ref) + queryContext, dummyDispatcher, timeseriesDataset.ref, RemoteHttpClient.defaultClient) 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()) @@ -67,7 +67,8 @@ class PromQlRemoteExecSpec extends AnyFunSpec with Matchers with ScalaFutures { } it ("should convert vector Data to QueryResponse for Metadata series query") { - val exec = MetadataRemoteExec("", 60000, Map.empty, queryContext, dummyDispatcher, timeseriesDataset.ref) + val exec = MetadataRemoteExec("", 60000, Map.empty, queryContext, + dummyDispatcher, timeseriesDataset.ref, RemoteHttpClient.defaultClient) 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()) @@ -79,7 +80,7 @@ class PromQlRemoteExecSpec extends AnyFunSpec with Matchers with ScalaFutures { } it ("should convert histogram to QueryResponse ") { - val exec = PromQlRemoteExec("", 60000, queryContext, dummyDispatcher, timeseriesDataset.ref) + val exec = PromQlRemoteExec("", 60000, queryContext, dummyDispatcher, timeseriesDataset.ref, RemoteHttpClient.defaultClient) val result = query.Result (Map("instance" -> "inst1"), None, Some(HistSampl(1000, Map("1" -> 2, "+Inf" -> 3)))) val res = exec.toQueryResponse(Data("vector", Seq(result)), "id", Kamon.currentSpan()) res.isInstanceOf[QueryResult] shouldEqual true From a90a7fd8d891de6591d1c7911238b4f71fde991c Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Wed, 28 Oct 2020 10:05:34 -0700 Subject: [PATCH 04/33] bug(query): Remove span per result RV; add spans/markers to track child plans (#935) * Removed scan span which was per result RV. It created too many spans - was not intended earlier. * Added spans and markers to track scans, and child result retrieval. * Fixed config for local zipkin testing --- conf/timeseries-filodb-server.conf | 27 +++++-------- .../scala/filodb.core/query/RangeVector.scala | 11 +----- .../filodb.core/query/RangeVectorSpec.scala | 2 +- .../filodb/query/exec/BinaryJoinExec.scala | 8 +++- .../scala/filodb/query/exec/ExecPlan.scala | 38 ++++++++++++------- .../query/exec/MetadataRemoteExec.scala | 2 +- .../filodb/query/exec/PromQlRemoteExec.scala | 6 +-- .../query/exec/RangeVectorTransformer.scala | 2 +- .../filodb/query/exec/SetOperatorExec.scala | 8 +++- .../query/exec/AggrOverRangeVectorsSpec.scala | 2 +- scripts/schema-truncate.sh | 2 +- 11 files changed, 56 insertions(+), 52 deletions(-) diff --git a/conf/timeseries-filodb-server.conf b/conf/timeseries-filodb-server.conf index e458a4ee3f..2800835194 100644 --- a/conf/timeseries-filodb-server.conf +++ b/conf/timeseries-filodb-server.conf @@ -48,24 +48,15 @@ kamon { port = 9095 } modules { - metriclog-reporter { - enabled = true - name = "MetricLog Reporter" - description = "Log all Metrics" - factory = "filodb.coordinator.KamonLogger$MetricsLogFactory" - } - spanlog-reporter { - enabled = true - name = "SpanLog Reporter" - description = "Log all traced Spans" - factory = "filodb.coordinator.KamonLogger$SpanLogFactory" - } - status-page { - enabled = false - } - zipkin-reporter { - enabled = false - } + zipkin-reporter.enabled = false + prometheus-reporter.enabled = false + status-page.enabled = false + } + + zipkin { + url = "https://localhost:9411/api/v2/spans" + max.requests = 128 + message.max.bytes = 131072 } metric.tick-interval = 60s diff --git a/core/src/main/scala/filodb.core/query/RangeVector.scala b/core/src/main/scala/filodb.core/query/RangeVector.scala index f07e595b23..9bc146ae5b 100644 --- a/core/src/main/scala/filodb.core/query/RangeVector.scala +++ b/core/src/main/scala/filodb.core/query/RangeVector.scala @@ -7,7 +7,6 @@ import scala.collection.Iterator import com.typesafe.scalalogging.StrictLogging import debox.Buffer import kamon.Kamon -import kamon.trace.Span import org.joda.time.DateTime import filodb.core.binaryrecord2.{MapItemConsumer, RecordBuilder, RecordContainer, RecordSchema} @@ -317,12 +316,7 @@ object SerializedRangeVector extends StrictLogging { // scalastyle:off null def apply(rv: RangeVector, builder: RecordBuilder, - schema: RecordSchema, - execPlanName: String, - span: Span = null): SerializedRangeVector = { - var spanBldr = Kamon.spanBuilder(s"execplan-scan-latency-$execPlanName") - if (span != null) { spanBldr = spanBldr.asChildOf(span) } - val scanSpan = spanBldr.start() + schema: RecordSchema): SerializedRangeVector = { var numRows = 0 val oldContainerOpt = builder.currentContainer val startRecordNo = oldContainerOpt.map(_.numRecords).getOrElse(0) @@ -345,7 +339,6 @@ object SerializedRangeVector extends StrictLogging { case None => builder.allContainers case Some(firstContainer) => builder.allContainers.dropWhile(_ != firstContainer) } - scanSpan.finish() new SerializedRangeVector(rv.key, numRows, containers, schema, startRecordNo) } // scalastyle:on null @@ -356,7 +349,7 @@ object SerializedRangeVector extends StrictLogging { */ def apply(rv: RangeVector, cols: Seq[ColumnInfo]): SerializedRangeVector = { val schema = toSchema(cols) - apply(rv, newBuilder(), schema, "Test-Only-Plan") + apply(rv, newBuilder(), schema) } // TODO: make this configurable.... diff --git a/core/src/test/scala/filodb.core/query/RangeVectorSpec.scala b/core/src/test/scala/filodb.core/query/RangeVectorSpec.scala index 4bdeaf56aa..8c15be7390 100644 --- a/core/src/test/scala/filodb.core/query/RangeVectorSpec.scala +++ b/core/src/test/scala/filodb.core/query/RangeVectorSpec.scala @@ -50,7 +50,7 @@ class RangeVectorSpec extends AnyFunSpec with Matchers { val builder = SerializedRangeVector.newBuilder() // Sharing one builder across multiple input RangeVectors - val srvs = rvs.map(rv => SerializedRangeVector(rv, builder, schema, "Unit-test")) + val srvs = rvs.map(rv => SerializedRangeVector(rv, builder, schema)) // Now verify each of them val observedTs = srvs(0).rows.toSeq.map(_.getLong(0)) diff --git a/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala b/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala index f8be12d668..ba992a2317 100644 --- a/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala +++ b/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala @@ -3,6 +3,7 @@ package filodb.query.exec import scala.collection.mutable import kamon.Kamon +import kamon.metric.MeasurementUnit import monix.eval.Task import monix.reactive.Observable @@ -65,6 +66,7 @@ final case class BinaryJoinExec(queryContext: QueryContext, protected[exec] def compose(childResponses: Observable[(QueryResponse, Int)], firstSchema: Task[ResultSchema], querySession: QuerySession): Observable[RangeVector] = { + val span = Kamon.currentSpan() val taskOfResults = childResponses.map { case (QueryResult(_, _, result), _) if (result.size > queryContext.plannerParams.joinQueryCardLimit && cardinality == Cardinality.OneToOne) => @@ -73,9 +75,11 @@ final case class BinaryJoinExec(queryContext: QueryContext, case (QueryResult(_, _, result), i) => (result, i) case (QueryError(_, ex), _) => throw ex }.toListL.map { resp => - Kamon.histogram("query-execute-time-elapsed-step2-child-results-available") + span.mark("binary-join-child-results-available") + Kamon.histogram("query-execute-time-elapsed-step1-child-results-available", + MeasurementUnit.time.milliseconds) .withTag("plan", getClass.getSimpleName) - .record(System.currentTimeMillis - queryContext.submitTime) + .record(Math.max(0, System.currentTimeMillis - queryContext.submitTime)) // NOTE: We can't require this any more, as multischema queries may result in not a QueryResult if the // filter returns empty results. The reason is that the schema will be undefined. // require(resp.size == lhs.size + rhs.size, "Did not get sufficient responses for LHS and RHS") diff --git a/query/src/main/scala/filodb/query/exec/ExecPlan.scala b/query/src/main/scala/filodb/query/exec/ExecPlan.scala index 7da3572161..2e3c84539b 100644 --- a/query/src/main/scala/filodb/query/exec/ExecPlan.scala +++ b/query/src/main/scala/filodb/query/exec/ExecPlan.scala @@ -119,17 +119,16 @@ trait ExecPlan extends QueryCommand { Kamon.histogram("query-execute-time-elapsed-step1-done", MeasurementUnit.time.milliseconds) .withTag("plan", getClass.getSimpleName) - .record(System.currentTimeMillis - startExecute) + .record(Math.max(0, System.currentTimeMillis - startExecute)) doEx } } // Step 2: Run connect monix pipeline to transformers, materialize the result def step2(res: ExecResult) = res.schema.map { resSchema => - Kamon.histogram("query-execute-time-elapsed-step2-start", - MeasurementUnit.time.milliseconds) + Kamon.histogram("query-execute-time-elapsed-step2-start", MeasurementUnit.time.milliseconds) .withTag("plan", getClass.getSimpleName) - .record(System.currentTimeMillis - startExecute) + .record(Math.max(0, System.currentTimeMillis - startExecute)) val span = Kamon.spanBuilder(s"execute-step2-${getClass.getSimpleName}") .asChildOf(parentSpan) .tag("query-id", queryContext.queryId) @@ -144,7 +143,6 @@ trait ExecPlan extends QueryCommand { Task.eval(QueryResult(queryContext.queryId, resSchema, Nil)) } else { val finalRes = allTransformers.foldLeft((res.rvs, resSchema)) { (acc, transf) => - span.mark(transf.getClass.getSimpleName) val paramRangeVector: Seq[Observable[ScalarRangeVector]] = transf.funcParams.map(_.getResult) (transf.apply(acc._1, querySession, queryContext.plannerParams.sampleLimit, acc._2, paramRangeVector), transf.schema(acc._2)) @@ -153,10 +151,12 @@ trait ExecPlan extends QueryCommand { Kamon.histogram("query-execute-time-elapsed-step2-transformer-pipeline-setup", MeasurementUnit.time.milliseconds) .withTag("plan", getClass.getSimpleName) - .record(System.currentTimeMillis - startExecute) + .record(Math.max(0, System.currentTimeMillis - startExecute)) + span.mark("step2-transformer-pipeline-setup") val builder = SerializedRangeVector.newBuilder() @volatile var numResultSamples = 0 // BEWARE - do not modify concurrently!! finalRes._1 + .doOnStart(_ => span.mark("before-first-materialized-result-rv")) .map { case srv: SerializableRangeVector => numResultSamples += srv.numRowsInt @@ -167,7 +167,7 @@ trait ExecPlan extends QueryCommand { srv case rv: RangeVector => // materialize, and limit rows per RV - val srv = SerializedRangeVector(rv, builder, recSchema, getClass.getSimpleName, span) + val srv = SerializedRangeVector(rv, builder, recSchema) numResultSamples += srv.numRowsInt // fail the query instead of limiting range vectors and returning incomplete/inaccurate results if (enforceLimit && numResultSamples > queryContext.plannerParams.sampleLimit) @@ -175,12 +175,13 @@ trait ExecPlan extends QueryCommand { sampleLimit} samples. Try applying more filters or reduce time range.") srv } + .doOnTerminate(_ => span.mark("after-last-materialized-result-rv")) .toListL .map { r => Kamon.histogram("query-execute-time-elapsed-step2-result-materialized", - MeasurementUnit.time.milliseconds) + MeasurementUnit.time.milliseconds) .withTag("plan", getClass.getSimpleName) - .record(System.currentTimeMillis - startExecute) + .record(Math.max(0, System.currentTimeMillis - startExecute)) val numBytes = builder.allContainers.map(_.numBytes).sum SerializedRangeVector.queryResultBytes.record(numBytes) span.mark(s"num-bytes: $numBytes") @@ -393,6 +394,11 @@ abstract class NonLeafExecPlan extends ExecPlan { (implicit sched: Scheduler): ExecResult = { val parentSpan = Kamon.currentSpan() + val span = Kamon.spanBuilder(s"execute-step1-child-result-composition-${getClass.getSimpleName}") + .asChildOf(parentSpan) + .tag("query-id", queryContext.queryId) + .start() + // whether child tasks need to be executed sequentially. // parallelism 1 means, only one worker thread to process underlying tasks. val parallelism: Int = if (parallelChildTasks) @@ -404,13 +410,16 @@ abstract class NonLeafExecPlan extends ExecPlan { // NOTE: It's really important to preserve the "index" of the child task, as joins depend on it val childTasks = Observable.fromIterable(children.zipWithIndex) .mapAsync(parallelism) { case (plan, i) => - dispatchRemotePlan(plan, parentSpan).map((_, i)) + dispatchRemotePlan(plan, span).map((_, i)) } // The first valid schema is returned as the Task. If all results are empty, then return // an empty schema. Validate that the other schemas are the same. Skip over empty schemas. var sch = ResultSchema.empty - val processedTasks = childTasks.collect { + val processedTasks = childTasks + .doOnStart(_ => span.mark("first-child-result-received")) + .doOnTerminate(_ => span.mark("last-child-result-received")) + .collect { case (res @ QueryResult(_, schema, _), i) if schema != ResultSchema.empty => sch = reduceSchemas(sch, res) (res, i.toInt) @@ -422,8 +431,11 @@ abstract class NonLeafExecPlan extends ExecPlan { val outputSchema = processedTasks.collect { case (QueryResult(_, schema, _), _) => schema }.firstOptionL.map(_.getOrElse(ResultSchema.empty)) - val outputRvs = compose(processedTasks, outputSchema, querySession) - ExecResult(outputRvs, outputSchema) + Kamon.runWithSpan(span, false) { + val outputRvs = compose(processedTasks, outputSchema, querySession) + .doOnTerminate(_ => span.finish()) + ExecResult(outputRvs, outputSchema) + } } /** diff --git a/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala b/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala index 567c2e4c89..132cbdcdf2 100644 --- a/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala +++ b/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala @@ -49,7 +49,7 @@ case class MetadataRemoteExec(queryEndpoint: String, val rangeVector = IteratorBackedRangeVector(new CustomRangeVectorKey(Map.empty), new UTF8MapIteratorRowReader(iteratorMap.toIterator)) - val srvSeq = Seq(SerializedRangeVector(rangeVector, builder, recordSchema, this.getClass.getSimpleName, span)) + val srvSeq = Seq(SerializedRangeVector(rangeVector, builder, recordSchema)) span.finish() QueryResult(id, resultSchema, srvSeq) diff --git a/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala b/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala index e4c6fa3945..4df8902783 100644 --- a/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala +++ b/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala @@ -115,7 +115,7 @@ case class PromQlRemoteExec(queryEndpoint: String, override def numRows: Option[Int] = Option(samples.size) } - SerializedRangeVector(rv, builder, recordSchema.get("default").get, "PromQlRemoteExec-default") + SerializedRangeVector(rv, builder, recordSchema.get("default").get) // TODO: Handle stitching with verbose flag } QueryResult(id, resultSchema.get("default").get, rangeVectors) @@ -149,7 +149,7 @@ case class PromQlRemoteExec(queryEndpoint: String, override def numRows: Option[Int] = Option(samples.size) } - SerializedRangeVector(rv, builder, recordSchema.get("histogram").get, "PromQlRemoteExec-hist") + SerializedRangeVector(rv, builder, recordSchema.get("histogram").get) // TODO: Handle stitching with verbose flag } QueryResult(id, resultSchema.get("histogram").get, rangeVectors) @@ -173,7 +173,7 @@ case class PromQlRemoteExec(queryEndpoint: String, } override def numRows: Option[Int] = Option(d.aggregateResponse.get.aggregateSampl.size) } - SerializedRangeVector(rv, builder, recordSchema.get(Avg.entryName).get, "PromQlRemoteExec-avg") + SerializedRangeVector(rv, builder, recordSchema.get(Avg.entryName).get) } // TODO: Handle stitching with verbose flag diff --git a/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala b/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala index cf2e9be525..06506327c7 100644 --- a/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala +++ b/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala @@ -279,7 +279,7 @@ final case class SortFunctionMapper(function: SortFunctionId) extends RangeVecto // Create SerializedRangeVector so that sorting does not consume rows iterator val resultRv = source.toListL.map { rvs => - rvs.map(SerializedRangeVector(_, builder, recSchema, getClass.getSimpleName)). + rvs.map(SerializedRangeVector(_, builder, recSchema)). sortBy { rv => if (rv.rows.hasNext) rv.rows.next().getDouble(1) else Double.NaN }(ordering) diff --git a/query/src/main/scala/filodb/query/exec/SetOperatorExec.scala b/query/src/main/scala/filodb/query/exec/SetOperatorExec.scala index d210bf150c..f8c1318359 100644 --- a/query/src/main/scala/filodb/query/exec/SetOperatorExec.scala +++ b/query/src/main/scala/filodb/query/exec/SetOperatorExec.scala @@ -4,6 +4,7 @@ import scala.collection.mutable import scala.collection.mutable.ListBuffer import kamon.Kamon +import kamon.metric.MeasurementUnit import monix.eval.Task import monix.reactive.Observable @@ -52,13 +53,16 @@ final case class SetOperatorExec(queryContext: QueryContext, protected[exec] def compose(childResponses: Observable[(QueryResponse, Int)], firstSchema: Task[ResultSchema], querySession: QuerySession): Observable[RangeVector] = { + val span = Kamon.currentSpan() val taskOfResults = childResponses.map { case (QueryResult(_, schema, result), i) => (schema, result, i) case (QueryError(_, ex), _) => throw ex }.toListL.map { resp => - Kamon.histogram("query-execute-time-elapsed-step2-child-results-available") + span.mark("binary-join-child-results-available") + Kamon.histogram("query-execute-time-elapsed-step1-child-results-available", + MeasurementUnit.time.milliseconds) .withTag("plan", getClass.getSimpleName) - .record(System.currentTimeMillis - queryContext.submitTime) + .record(Math.max(0, System.currentTimeMillis - queryContext.submitTime)) // NOTE: We can't require this any more, as multischema queries may result in not a QueryResult if the // filter returns empty results. The reason is that the schema will be undefined. // require(resp.size == lhs.size + rhs.size, "Did not get sufficient responses for LHS and RHS") diff --git a/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala b/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala index 80c1e2f921..d33e86d497 100644 --- a/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala +++ b/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala @@ -287,7 +287,7 @@ class AggrOverRangeVectorsSpec extends RawDataWindowingSpec with ScalaFutures { val recSchema = SerializedRangeVector.toSchema(Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), ColumnInfo("tdig", ColumnType.StringColumn))) val builder = SerializedRangeVector.newBuilder() - val srv = SerializedRangeVector(result7(0), builder, recSchema, "Unit-Test") + val srv = SerializedRangeVector(result7(0), builder, recSchema) val resultObs7b = RangeVectorAggregator.present(agg7, Observable.now(srv), 1000) val finalResult = resultObs7b.toListL.runAsync.futureValue diff --git a/scripts/schema-truncate.sh b/scripts/schema-truncate.sh index 82e37d2050..fdad7c306d 100755 --- a/scripts/schema-truncate.sh +++ b/scripts/schema-truncate.sh @@ -39,7 +39,7 @@ done if [[ "${KEYSP}" != "${FILO_DOWNSAMPLE_KEYSPACE}" ]]; then cat << EOF -TRUNCATE ${KEYSP}.${DSET}_pks_by_update_time +TRUNCATE ${KEYSP}.${DSET}_pks_by_update_time; EOF fi From 89027c3488c4fd5a704150537fb12c6e5c219a8b Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Wed, 28 Oct 2020 12:05:37 -0700 Subject: [PATCH 05/33] debug(query): Marking spans when dispatch of child plan is done (#936) --- query/src/main/scala/filodb/query/exec/ExecPlan.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/query/src/main/scala/filodb/query/exec/ExecPlan.scala b/query/src/main/scala/filodb/query/exec/ExecPlan.scala index 2e3c84539b..0fb9b32400 100644 --- a/query/src/main/scala/filodb/query/exec/ExecPlan.scala +++ b/query/src/main/scala/filodb/query/exec/ExecPlan.scala @@ -410,7 +410,9 @@ abstract class NonLeafExecPlan extends ExecPlan { // NOTE: It's really important to preserve the "index" of the child task, as joins depend on it val childTasks = Observable.fromIterable(children.zipWithIndex) .mapAsync(parallelism) { case (plan, i) => - dispatchRemotePlan(plan, span).map((_, i)) + val task = dispatchRemotePlan(plan, span).map((_, i)) + span.mark(s"plan-dispatched-${plan.getClass.getSimpleName}") + task } // The first valid schema is returned as the Task. If all results are empty, then return From d5692acaedfb9ad4814bc3cf70ee3c4e70effe44 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Wed, 28 Oct 2020 17:12:58 -0700 Subject: [PATCH 06/33] bug(memory): Remove Time-ordered-blocks (#918) Not reclaiming ODP blocks quickly leads to more holes. Hole expansion leads to ODP amplification. In addition there are bugs where Time ordered blocks are indeed being reclaimed before they were designed to be reclaimed. Hence simplifying by: * Remove time ordered blocks. Replace ODP block mem factories per time bucket with just one block-mem-factory. * Reclaim logic will reclaim all ODP blocks first (except currently used block) before reclaiming any ingestion block. Concurrency between ODP queries and reclaims are protected with reclaim lock - this way paged data is not reclaimed before query completes. * The intention here is to keep ODPed data until memory saturates. Once that happens, ODPed data stays in memory momentarily until query completes. It will then be reclaimed quickly. * Hole expansion logic continues to help by discarding chunk map pointers when newer paged data is reclaimed before older paged data (this can occur due to query order). * Added much needed unit test for BlockMemFactory - validates logic of marking as reclaimable and tests reclaim order --- .../downsample/OffHeapMemory.scala | 2 +- .../memstore/DemandPagedChunkStore.scala | 50 +----- .../memstore/TimeSeriesShard.scala | 5 +- .../src/test/scala/filodb.core/TestData.scala | 4 +- .../downsample/ShardDownsamplerSpec.scala | 2 +- .../memstore/DemandPagedChunkStoreSpec.scala | 6 - .../memstore/PartitionSetSpec.scala | 2 +- .../memstore/TimeSeriesPartitionSpec.scala | 14 +- .../src/main/scala/filodb.memory/Block.scala | 10 -- .../scala/filodb.memory/BlockManager.scala | 168 +++++------------- .../filodb.memory/BlockMemFactoryPool.scala | 2 +- .../main/scala/filodb.memory/MemFactory.scala | 8 +- .../filodb.memory/BlockMemFactorySpec.scala | 97 +++++++++- .../test/scala/filodb.memory/BlockSpec.scala | 8 +- ...ageAlignedBlockManagerConcurrentSpec.scala | 6 +- .../PageAlignedBlockManagerSpec.scala | 80 ++++----- .../format/vectors/IntBinaryVectorTest.scala | 2 +- .../rangefn/AggrOverTimeFunctionsSpec.scala | 4 +- 18 files changed, 205 insertions(+), 265 deletions(-) diff --git a/core/src/main/scala/filodb.core/downsample/OffHeapMemory.scala b/core/src/main/scala/filodb.core/downsample/OffHeapMemory.scala index 2d6992b915..cab49ba3a3 100644 --- a/core/src/main/scala/filodb.core/downsample/OffHeapMemory.scala +++ b/core/src/main/scala/filodb.core/downsample/OffHeapMemory.scala @@ -25,7 +25,7 @@ class OffHeapMemory(schemas: Seq[Schema], }, numPagesPerBlock = 50) - val blockMemFactory = new BlockMemFactory(blockStore, None, maxMetaSize, kamonTags, false) + val blockMemFactory = new BlockMemFactory(blockStore, maxMetaSize, kamonTags, false) val nativeMemoryManager = new NativeMemoryManager(nativeMemSize, kamonTags) diff --git a/core/src/main/scala/filodb.core/memstore/DemandPagedChunkStore.scala b/core/src/main/scala/filodb.core/memstore/DemandPagedChunkStore.scala index 14048ba2ba..6516bd33b4 100644 --- a/core/src/main/scala/filodb.core/memstore/DemandPagedChunkStore.scala +++ b/core/src/main/scala/filodb.core/memstore/DemandPagedChunkStore.scala @@ -3,11 +3,9 @@ package filodb.core.memstore import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer -import scala.concurrent.duration._ import com.typesafe.scalalogging.StrictLogging import monix.eval.Task -import org.jctools.maps.NonBlockingHashMapLong import spire.syntax.cfor._ import filodb.core.store._ @@ -30,39 +28,18 @@ import filodb.memory.format.UnsafeUtils * * @param tsShard the TimeSeriesShard containing the time series for the given shard * @param blockManager The block manager to be used for block allocation - * @param chunkRetentionHours number of hours chunks need to be retained for. Beyond this time, ODP blocks will be - * marked as reclaimable even if they are not full, so they can be reused for newer data. */ class DemandPagedChunkStore(tsShard: TimeSeriesShard, - val blockManager: BlockManager, - chunkRetentionHours: Int) + val blockManager: BlockManager) extends RawToPartitionMaker with StrictLogging { - val flushIntervalMillis = tsShard.storeConfig.flushInterval.toMillis - val retentionMillis = chunkRetentionHours * (1.hour.toMillis) - - // block factories for each time bucket - private val memFactories = new NonBlockingHashMapLong[BlockMemFactory](chunkRetentionHours, false) import TimeSeriesShard._ - import collection.JavaConverters._ private val baseContext = Map("dataset" -> tsShard.ref.toString, "shard" -> tsShard.shardNum.toString) - - private def getMemFactory(bucket: Long): BlockMemFactory = { - val factory = memFactories.get(bucket) - if (factory == UnsafeUtils.ZeroPointer) { - val newFactory = new BlockMemFactory(blockManager, - Some(bucket), - tsShard.maxMetaSize, - baseContext ++ Map("bucket" -> bucket.toString), - markFullBlocksAsReclaimable = true) - memFactories.put(bucket, newFactory) - newFactory - } else { - factory - } - } + val memFactory = new BlockMemFactory(blockManager, + tsShard.maxMetaSize, baseContext ++ Map("odp" -> "true"), + markFullBlocksAsReclaimable = true) /** * Stores raw chunks into offheap memory and populates chunks into partition @@ -84,7 +61,6 @@ extends RawToPartitionMaker with StrictLogging { // possible to guard against this by forcing an allocation, but it doesn't make sense // to allocate a block just for storing an unnecessary metadata entry. if (!rawVectors.isEmpty) { - val memFactory = getMemFactory(timeBucketForChunkSet(infoBytes)) val chunkID = ChunkSetInfo.getChunkID(infoBytes) if (!tsPart.chunkmapContains(chunkID)) { @@ -119,13 +95,6 @@ extends RawToPartitionMaker with StrictLogging { } //scalastyle:on - /** - * For a given chunkset, this method calculates the time bucket the chunks fall in. - * It is used in deciding which BlockMemFactory to use while allocating off-heap memory for this chunk. - */ - private def timeBucketForChunkSet(infoBytes: Array[Byte]): Long = - (ChunkSetInfo.getEndTime(infoBytes) / flushIntervalMillis) * flushIntervalMillis - /** * Copies the onHeap contents read from ColStore into off-heap using the given memFactory. * If an exception is thrown by this method, the tail of chunkPtrs sequence isn't filled in. @@ -144,15 +113,4 @@ extends RawToPartitionMaker with StrictLogging { } } - /** - * Ensures the oldest ODP time buckets, blocks, and BlockMemFactory's are reclaimable and cleaned up - * so we don't leak memory and blocks. Call this ideally every flushInterval. - */ - def cleanupOldestBuckets(): Unit = { - blockManager.markBucketedBlocksReclaimable(System.currentTimeMillis - retentionMillis) - // Now, iterate through memFactories and clean up ones with no blocks - memFactories.keySet.asScala.foreach { bucket => - if (!blockManager.hasTimeBucket(bucket)) memFactories.remove(bucket) - } - } } diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala index baff77d164..72c9392299 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala @@ -330,9 +330,9 @@ class TimeSeriesShard(val ref: DatasetRef, // Each shard has a single ingestion stream at a time. This BlockMemFactory is used for buffer overflow encoding // strictly during ingest() and switchBuffers(). - private[core] val overflowBlockFactory = new BlockMemFactory(blockStore, None, maxMetaSize, + private[core] val overflowBlockFactory = new BlockMemFactory(blockStore, maxMetaSize, shardTags ++ Map("overflow" -> "true"), true) - val partitionMaker = new DemandPagedChunkStore(this, blockStore, chunkRetentionHours) + val partitionMaker = new DemandPagedChunkStore(this, blockStore) private val partKeyBuilder = new RecordBuilder(MemFactory.onHeapFactory, reuseOneContainer = true) private val partKeyArray = partKeyBuilder.allContainers.head.base.asInstanceOf[Array[Byte]] @@ -947,7 +947,6 @@ class TimeSeriesShard(val ref: DatasetRef, logger.info(s"Flush of dataset=$ref shard=$shardNum group=${flushGroup.groupNum} " + s"flushWatermark=${flushGroup.flushWatermark} response=$resp offset=${_offset}") } - partitionMaker.cleanupOldestBuckets() // Some partitions might be evictable, see if need to free write buffer memory checkEnableAddPartitions() updateGauges() diff --git a/core/src/test/scala/filodb.core/TestData.scala b/core/src/test/scala/filodb.core/TestData.scala index 2552a4f7a8..ccd932c1ae 100644 --- a/core/src/test/scala/filodb.core/TestData.scala +++ b/core/src/test/scala/filodb.core/TestData.scala @@ -398,9 +398,9 @@ object MachineMetricsData { val histPartKey = histKeyBuilder.partKeyFromObjects(histDataset.schema, "request-latency", extraTags) val blockStore = new PageAlignedBlockManager(100 * 1024 * 1024, new MemoryStats(Map("test"-> "test")), null, 16) - val histIngestBH = new BlockMemFactory(blockStore, None, histDataset.schema.data.blockMetaSize, + val histIngestBH = new BlockMemFactory(blockStore, histDataset.schema.data.blockMetaSize, dummyContext, true) - val histMaxBH = new BlockMemFactory(blockStore, None, histMaxDS.schema.data.blockMetaSize, + val histMaxBH = new BlockMemFactory(blockStore, histMaxDS.schema.data.blockMetaSize, dummyContext, true) private val histBufferPool = new WriteBufferPool(TestData.nativeMem, histDataset.schema.data, TestData.storeConf) diff --git a/core/src/test/scala/filodb.core/downsample/ShardDownsamplerSpec.scala b/core/src/test/scala/filodb.core/downsample/ShardDownsamplerSpec.scala index bf0cf8fc85..eded32d4ec 100644 --- a/core/src/test/scala/filodb.core/downsample/ShardDownsamplerSpec.scala +++ b/core/src/test/scala/filodb.core/downsample/ShardDownsamplerSpec.scala @@ -48,7 +48,7 @@ class ShardDownsamplerSpec extends AnyFunSpec with Matchers with BeforeAndAfterA val customSchema = customDataset.schema private val blockStore = MMD.blockStore - protected val ingestBlockHolder = new BlockMemFactory(blockStore, None, promDataset.schema.data.blockMetaSize, + protected val ingestBlockHolder = new BlockMemFactory(blockStore, promDataset.schema.data.blockMetaSize, MMD.dummyContext, true) val storeConf = TestData.storeConf.copy(maxChunksSize = 200) diff --git a/core/src/test/scala/filodb.core/memstore/DemandPagedChunkStoreSpec.scala b/core/src/test/scala/filodb.core/memstore/DemandPagedChunkStoreSpec.scala index 6943defe80..703c80bbcd 100644 --- a/core/src/test/scala/filodb.core/memstore/DemandPagedChunkStoreSpec.scala +++ b/core/src/test/scala/filodb.core/memstore/DemandPagedChunkStoreSpec.scala @@ -58,13 +58,7 @@ class DemandPagedChunkStoreSpec extends AnyFunSpec with AsyncTest { tsPartition.numChunks shouldEqual 10 // write buffers + 9 chunks above } - pageManager.numTimeOrderedBlocks should be > 1 pageManager.numFreeBlocks should be >= (initFreeBlocks - 12) - val buckets = pageManager.timeBuckets - buckets.foreach { b => pageManager.hasTimeBucket(b) shouldEqual true } - - // Now, reclaim four time buckets, even if they are not full - pageManager.markBucketedBlocksReclaimable(buckets(4)) // try and ODP more data. Load older data than chunk retention, should still be able to load val data2 = linearMultiSeries(start - 2.hours.toMillis, timeStep=100000).take(20) diff --git a/core/src/test/scala/filodb.core/memstore/PartitionSetSpec.scala b/core/src/test/scala/filodb.core/memstore/PartitionSetSpec.scala index f267896384..eec350fbf2 100644 --- a/core/src/test/scala/filodb.core/memstore/PartitionSetSpec.scala +++ b/core/src/test/scala/filodb.core/memstore/PartitionSetSpec.scala @@ -32,7 +32,7 @@ class PartitionSetSpec extends MemFactoryCleanupTest with ScalaFutures { private val blockStore = new PageAlignedBlockManager(100 * 1024 * 1024, new MemoryStats(Map("test"-> "test")), reclaimer, 1) protected val bufferPool = new WriteBufferPool(memFactory, dataset2.schema.data, TestData.storeConf) - private val ingestBlockHolder = new BlockMemFactory(blockStore, None, dataset2.schema.data.blockMetaSize, + private val ingestBlockHolder = new BlockMemFactory(blockStore, dataset2.schema.data.blockMetaSize, dummyContext, true) val builder = new RecordBuilder(memFactory) diff --git a/core/src/test/scala/filodb.core/memstore/TimeSeriesPartitionSpec.scala b/core/src/test/scala/filodb.core/memstore/TimeSeriesPartitionSpec.scala index e5f6e5e851..7c1584af53 100644 --- a/core/src/test/scala/filodb.core/memstore/TimeSeriesPartitionSpec.scala +++ b/core/src/test/scala/filodb.core/memstore/TimeSeriesPartitionSpec.scala @@ -71,7 +71,7 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { private val blockStore = new PageAlignedBlockManager(200 * 1024 * 1024, new MemoryStats(Map("test"-> "test")), reclaimer, 1) - protected val ingestBlockHolder = new BlockMemFactory(blockStore, None, schema1.data.blockMetaSize, + protected val ingestBlockHolder = new BlockMemFactory(blockStore, schema1.data.blockMetaSize, dummyContext, true) before { @@ -114,7 +114,7 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { val data1 = part.timeRangeRows(AllChunkScan, Array(1)).map(_.getDouble(0)).toBuffer data1 shouldEqual (minData take 10) - val blockHolder = new BlockMemFactory(blockStore, None, schema1.data.blockMetaSize, dummyContext) + val blockHolder = new BlockMemFactory(blockStore, schema1.data.blockMetaSize, dummyContext) // Task needs to fully iterate over the chunks, to release the shared lock. val flushFut = Future(part.makeFlushChunks(blockHolder).toBuffer) data.drop(10).zipWithIndex.foreach { case (r, i) => part.ingest(0, r, ingestBlockHolder) } @@ -172,7 +172,7 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { part.switchBuffers(ingestBlockHolder) part.appendingChunkLen shouldEqual 0 - val blockHolder = new BlockMemFactory(blockStore, None, schema1.data.blockMetaSize, dummyContext) + val blockHolder = new BlockMemFactory(blockStore, schema1.data.blockMetaSize, dummyContext) // Task needs to fully iterate over the chunks, to release the shared lock. val flushFut = Future(part.makeFlushChunks(blockHolder).toBuffer) data.drop(10).zipWithIndex.foreach { case (r, i) => part.ingest(0, r, ingestBlockHolder) } @@ -229,7 +229,7 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { // First 10 rows ingested. Now flush in a separate Future while ingesting 6 more rows part.switchBuffers(ingestBlockHolder) myBufferPool.poolSize shouldEqual origPoolSize // current chunks become null, no new allocation yet - val blockHolder = new BlockMemFactory(blockStore, None, schema1.data.blockMetaSize, dummyContext) + val blockHolder = new BlockMemFactory(blockStore, schema1.data.blockMetaSize, dummyContext) // Task needs to fully iterate over the chunks, to release the shared lock. val flushFut = Future(part.makeFlushChunks(blockHolder).toBuffer) data.drop(10).take(6).zipWithIndex.foreach { case (r, i) => part.ingest(0, r, ingestBlockHolder) } @@ -260,7 +260,7 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { // Now, switch buffers and flush again, ingesting 5 more rows // There should now be 3 chunks total, the current write buffers plus the two flushed ones part.switchBuffers(ingestBlockHolder) - val holder2 = new BlockMemFactory(blockStore, None, schema1.data.blockMetaSize, dummyContext) + val holder2 = new BlockMemFactory(blockStore, schema1.data.blockMetaSize, dummyContext) // Task needs to fully iterate over the chunks, to release the shared lock. val flushFut2 = Future(part.makeFlushChunks(holder2).toBuffer) data.drop(16).zipWithIndex.foreach { case (r, i) => part.ingest(0, r, ingestBlockHolder) } @@ -289,7 +289,7 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { // Now, switch buffers and flush. Appenders will be empty. part.switchBuffers(ingestBlockHolder) - val blockHolder = new BlockMemFactory(blockStore, None, schema1.data.blockMetaSize, dummyContext) + val blockHolder = new BlockMemFactory(blockStore, schema1.data.blockMetaSize, dummyContext) val chunkSets = part.makeFlushChunks(blockHolder) chunkSets.isEmpty shouldEqual false part.numChunks shouldEqual 1 @@ -379,7 +379,7 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { // Now simulate a flush, verify that both chunksets flushed // Now, switch buffers and flush. Appenders will be empty. part.switchBuffers(ingestBlockHolder) - val blockHolder = new BlockMemFactory(blockStore, None, schema1.data.blockMetaSize, dummyContext) + val blockHolder = new BlockMemFactory(blockStore, schema1.data.blockMetaSize, dummyContext) val chunkSets = part.makeFlushChunks(blockHolder).toSeq chunkSets should have length (2) part.numChunks shouldEqual 2 diff --git a/memory/src/main/scala/filodb.memory/Block.scala b/memory/src/main/scala/filodb.memory/Block.scala index 4e6831e72a..153e32a185 100644 --- a/memory/src/main/scala/filodb.memory/Block.scala +++ b/memory/src/main/scala/filodb.memory/Block.scala @@ -107,16 +107,6 @@ class Block(val address: Long, val capacity: Long, val reclaimListener: ReclaimL owner = None } - /** - * Marks this block as reclaimable if unowned, or if the owner hasn't used the block in a while. - */ - def tryMarkReclaimable(): Unit = { - owner match { - case None => markReclaimable - case Some(bmf) => bmf.tryMarkReclaimable - } - } - /** * Marks this memory as free. Also zeroes all the bytes from the beginning address until capacity */ diff --git a/memory/src/main/scala/filodb.memory/BlockManager.scala b/memory/src/main/scala/filodb.memory/BlockManager.scala index eff2d484e0..9d027b7a10 100644 --- a/memory/src/main/scala/filodb.memory/BlockManager.scala +++ b/memory/src/main/scala/filodb.memory/BlockManager.scala @@ -1,12 +1,11 @@ package filodb.memory -import java.lang.{Long => jLong} -import java.util import java.util.concurrent.TimeUnit import java.util.concurrent.locks.ReentrantLock import com.kenai.jffi.{MemoryIO, PageManager} import com.typesafe.scalalogging.StrictLogging +import java.util import kamon.Kamon import kamon.metric.Counter import kamon.tag.TagSet @@ -28,25 +27,20 @@ trait BlockManager { */ def numFreeBlocks: Int - /** - * @return true if the time bucket has blocks allocated - */ - def hasTimeBucket(bucket: Long): Boolean - /** * @param memorySize The size of memory in bytes for which blocks are to be allocated - * @param bucketTime the timebucket (timestamp) from which to allocate block(s), or None for the general list + * @param odp true if this is for paging ODPed chunks * @param owner the BlockMemFactory that will be owning this block, until reclaim. Used for debugging. * @return A sequence of blocks totaling up in memory requested or empty if unable to allocate */ - def requestBlocks(memorySize: Long, bucketTime: Option[Long], owner: Option[BlockMemFactory] = None): Seq[Block] + def requestBlocks(memorySize: Long, odp: Boolean, owner: Option[BlockMemFactory] = None): Seq[Block] /** - * @param bucketTime the timebucket from which to allocate block(s), or None for the general list + * @param odp true if requesting for ODP * @param owner the BlockMemFactory that will be owning this block, until reclaim. Used for debugging. * @return One block of memory */ - def requestBlock(bucketTime: Option[Long], owner: Option[BlockMemFactory] = None): Option[Block] + def requestBlock(odp: Boolean, owner: Option[BlockMemFactory] = None): Option[Block] /** * Attempts to reclaim as many blocks as necessary to ensure that enough free blocks are @@ -90,11 +84,6 @@ trait BlockManager { */ def releaseBlocks(): Unit - /** - * Marks all time-bucketed blocks in buckets up to upTo as reclaimable - */ - def markBucketedBlocksReclaimable(upTo: Long): Unit - /** * @return Memory stats for recording */ @@ -102,13 +91,14 @@ trait BlockManager { } class MemoryStats(tags: Map[String, String]) { - val usedBlocksMetric = Kamon.gauge("blockstore-used-blocks").withTags(TagSet.from(tags)) + val usedIngestionBlocksMetric = Kamon.gauge("blockstore-used-ingestion-blocks").withTags(TagSet.from(tags)) val freeBlocksMetric = Kamon.gauge("blockstore-free-blocks").withTags(TagSet.from(tags)) val requestedBlocksMetric = Kamon.counter("blockstore-blocks-requested").withTags(TagSet.from(tags)) - val usedBlocksTimeOrderedMetric = Kamon.gauge("blockstore-used-time-ordered-blocks").withTags(TagSet.from(tags)) - val timeOrderedBlocksReclaimedMetric = Kamon.counter("blockstore-time-ordered-blocks-reclaimed") + val usedOdpBlocksMetric = Kamon.gauge("blockstore-used-odp-blocks").withTags(TagSet.from(tags)) + val odpBlocksReclaimedMetric = Kamon.counter("blockstore-odp-blocks-reclaimed") + .withTags(TagSet.from(tags)) + val ingestionBlocksReclaimedMetric = Kamon.counter("blockstore-ingestion-blocks-reclaimed") .withTags(TagSet.from(tags)) - val blocksReclaimedMetric = Kamon.counter("blockstore-blocks-reclaimed").withTags(TagSet.from(tags)) /** * How much time a thread was potentially stalled while attempting to ensure @@ -153,8 +143,8 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, protected var firstPageAddress: Long = 0L protected val freeBlocks: util.ArrayDeque[Block] = allocate() - protected[memory] val usedBlocks: util.ArrayDeque[Block] = new util.ArrayDeque[Block]() - protected[memory] val usedBlocksTimeOrdered = new util.TreeMap[Long, util.ArrayDeque[Block]] + protected[memory] val usedIngestionBlocks: util.ArrayDeque[Block] = new util.ArrayDeque[Block]() + protected[memory] val usedOdpBlocks: util.ArrayDeque[Block] = new util.ArrayDeque[Block]() val reclaimLog = new collection.mutable.Queue[ReclaimEvent] protected val lock = new ReentrantLock() @@ -164,12 +154,12 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, override def blockSizeInBytes: Long = PageManager.getInstance().pageSize() * numPagesPerBlock - def usedMemory: Long = usedBlocks.size * blockSizeInBytes + def usedMemory: Long = usedIngestionBlocks.size * blockSizeInBytes override def numFreeBlocks: Int = freeBlocks.size - override def requestBlock(bucketTime: Option[Long], bmf: Option[BlockMemFactory] = None): Option[Block] = { - val blocks = requestBlocks(blockSizeInBytes, bucketTime, bmf) + override def requestBlock(odp: Boolean, bmf: Option[BlockMemFactory] = None): Option[Block] = { + val blocks = requestBlocks(blockSizeInBytes, odp, bmf) blocks.size match { case 0 => None case 1 => Some(blocks.head) @@ -177,41 +167,33 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, } } - /* Used in tests for assertion */ - def usedBlocksSize(bucketTime: Option[Long]): Int = { - bucketTime match { - case Some(t) => usedBlocksTimeOrdered.get(t).size() - case None => usedBlocks.size() - } - } - /** * Allocates requested number of blocks. If enough blocks are not available, * then uses the ReclaimPolicy to check if blocks can be reclaimed * Uses a lock to ensure that concurrent requests are safe. * - * If bucketTime is provided, a MemoryRequestException is thrown when no blocks are - * currently available. In other words, time ordered block allocation doesn't force + * If odp is true, a MemoryRequestException is thrown when sufficient blocks are not + * currently free. In other words, ODP block request doesn't attempt * reclamation. Instead, a background task must be running which calls ensureFreeBlocks. - * Time ordered blocks are used for on-demand-paging only (ODP), initiated by a query, and + * ODP blocks are used for on-demand-paging only (ODP), initiated by a query, and * reclamation during ODP can end up causing the query results to have "holes". Throwing an * exception isn't a perfect solution, but it can suffice until a proper block pinning * mechanism is in place. Queries which fail with this exception can retry, perhaps after * calling ensureFreeBLocks explicitly. */ override def requestBlocks(memorySize: Long, - bucketTime: Option[Long], - bmf: Option[BlockMemFactory] = None): Seq[Block] = { + odp: Boolean, + ownerBmf: Option[BlockMemFactory] = None): Seq[Block] = { val num: Int = Math.ceil(memorySize / blockSizeInBytes).toInt stats.requestedBlocksMetric.increment(num) lock.lock() try { if (freeBlocks.size < num) { - if (bucketTime.isEmpty) { + if (!odp) { tryReclaimOnDemand(num) } else { - val msg = s"Unable to allocate time ordered block(s) without forcing a reclamation: " + + val msg = s"Unable to allocate ODP block(s) without forcing a reclamation: " + s"num_blocks=$num num_bytes=$memorySize freeBlocks=${freeBlocks.size}" throw new MemoryRequestException(msg) } @@ -221,8 +203,8 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, val allocated = new Array[Block](num) (0 until num).foreach { i => val block = freeBlocks.remove() - if (bmf.nonEmpty) block.setOwner(bmf.get) - use(block, bucketTime) + if (ownerBmf.nonEmpty) block.setOwner(ownerBmf.get) + use(block, odp) allocated(i) = block } allocated @@ -375,18 +357,14 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, blocks } - protected def use(block: Block, bucketTime: Option[Long]) = { + protected def use(block: Block, odp: Boolean) = { block.markInUse - bucketTime match { - case Some(bucket) => val blockList = Option(usedBlocksTimeOrdered.get(bucket)).getOrElse { - val list = new util.ArrayDeque[Block]() - usedBlocksTimeOrdered.put(bucket, list) - list - } - blockList.add(block) - stats.usedBlocksTimeOrderedMetric.update(numTimeOrderedBlocks) - case None => usedBlocks.add(block) - stats.usedBlocksMetric.update(usedBlocks.size()) + if (odp) { + usedOdpBlocks.add(block) + stats.usedOdpBlocksMetric.update(usedOdpBlocks.size()) + } else { + usedIngestionBlocks.add(block) + stats.usedIngestionBlocksMetric.update(usedIngestionBlocks.size()) } stats.freeBlocksMetric.update(freeBlocks.size()) } @@ -397,56 +375,26 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, reclaimLog += event } - //scalastyle:off protected[memory] def tryReclaim(num: Int): Int = { var reclaimed = 0 - - // First reclaim time-ordered blocks which are marked as reclaimable. - reclaimTimeOrdered(false); - + reclaimFrom(usedOdpBlocks, stats.odpBlocksReclaimedMetric) + if (reclaimed < num) reclaimFrom(usedIngestionBlocks, stats.ingestionBlocksReclaimedMetric) + // if we do not get required blocks even after reclaim call if (reclaimed < num) { - // Not enough reclaimed, so try reclaiming non-time-ordered blocks which are marked as reclaimable. - reclaimFrom(usedBlocks, stats.blocksReclaimedMetric, false) - - if (reclaimed < num) { - // Still not enough? Forcibly reclaim time-ordered blocks. - reclaimTimeOrdered(true); - - if (reclaimed < num) { - // Still not enough, but forcibly reclaiming non-time-ordered blocks is dangerous. - logger.warn(s"$num blocks to reclaim but only reclaimed $reclaimed. usedblocks=${usedBlocks.size} " + - s"usedBlocksTimeOrdered=${usedBlocksTimeOrdered.asScala.toList.map{case(n, l) => (n, l.size)}}") - } - } + logger.warn(s"$num blocks to reclaim but only reclaimed $reclaimed. " + + s"usedIngestionBlocks=${usedIngestionBlocks.size} usedOdpBlocks=${usedOdpBlocks.size()}") } - def reclaimTimeOrdered(forced: Boolean): Unit = { - val timeOrderedListIt = usedBlocksTimeOrdered.entrySet.iterator - while ( reclaimed < num && - timeOrderedListIt.hasNext ) { - val entry = timeOrderedListIt.next - val prevReclaimed = reclaimed - val removed = reclaimFrom(entry.getValue, stats.timeOrderedBlocksReclaimedMetric, forced) - if (removed.nonEmpty) { - logger.info(s"timeBlockReclaim: Reclaimed ${removed.length} time ordered blocks " + - s"from list at t=${entry.getKey} (${(System.currentTimeMillis - entry.getKey)/3600000} hrs ago) " + - s"\nReclaimed blocks: ${removed.map(b => jLong.toHexString(b.address)).mkString(" ")}") - } - // If the block list is now empty, remove it from tree map - if (entry.getValue.isEmpty) timeOrderedListIt.remove() - } - } - - def reclaimFrom(list: util.ArrayDeque[Block], reclaimedCounter: Counter, forced: Boolean): Seq[Block] = { + def reclaimFrom(list: util.ArrayDeque[Block], reclaimedCounter: Counter): Seq[Block] = { val entries = list.iterator val removed = new collection.mutable.ArrayBuffer[Block] while (entries.hasNext && reclaimed < num) { val block = entries.next - if (forced || block.canReclaim) { + if (block.canReclaim) { entries.remove() removed += block addToReclaimLog(block) - block.reclaim(forced) + block.reclaim() block.clearOwner() freeBlocks.add(block) stats.freeBlocksMetric.update(freeBlocks.size()) @@ -458,41 +406,15 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, } reclaimed } - //scalastyle:on - - def numTimeOrderedBlocks: Int = usedBlocksTimeOrdered.values.asScala.map(_.size).sum - - def timeBuckets: Seq[Long] = usedBlocksTimeOrdered.keySet.asScala.toSeq - - def markBucketedBlocksReclaimable(upTo: Long): Unit = { - lock.lock() - try { - logger.info(s"timeBlockReclaim: Marking ($upTo) - this is -${(System.currentTimeMillis - upTo)/3600000}hrs") - val keys = usedBlocksTimeOrdered.headMap(upTo).keySet.asScala - logger.info(s"timeBlockReclaim: Marking lists $keys as reclaimable") - usedBlocksTimeOrdered.headMap(upTo).values.asScala.foreach { list => - list.asScala.foreach(_.tryMarkReclaimable) - } - } finally { - lock.unlock() - } - } - - def hasTimeBucket(bucket: Long): Boolean = { - lock.lock() - val result = usedBlocksTimeOrdered.containsKey(bucket) - lock.unlock() - result - } /** * Used during testing only to try and reclaim all existing blocks */ def reclaimAll(): Unit = { logger.warn(s"Reclaiming all used blocks -- THIS BETTER BE A TEST!!!") - markBucketedBlocksReclaimable(Long.MaxValue) - usedBlocks.asScala.foreach(_.markReclaimable) - tryReclaim(usedBlocks.size + numTimeOrderedBlocks) + usedIngestionBlocks.asScala.foreach(_.markReclaimable()) + usedOdpBlocks.asScala.foreach(_.markReclaimable()) + tryReclaim(usedIngestionBlocks.size + usedOdpBlocks.size()) } /** @@ -505,9 +427,7 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, def timeBlocksForPtr(ptr: BinaryRegion.NativePointer): Seq[Block] = { lock.lock() try { - usedBlocksTimeOrdered.entrySet.iterator.asScala.flatMap { entry => - BlockDetective.containsPtr(ptr, entry.getValue) - }.toBuffer + BlockDetective.containsPtr(ptr, usedOdpBlocks) } finally { lock.unlock() } @@ -527,5 +447,5 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, } } - override def finalize(): Unit = releaseBlocks + override def finalize(): Unit = releaseBlocks() } diff --git a/memory/src/main/scala/filodb.memory/BlockMemFactoryPool.scala b/memory/src/main/scala/filodb.memory/BlockMemFactoryPool.scala index 736744f91f..ee186e218c 100644 --- a/memory/src/main/scala/filodb.memory/BlockMemFactoryPool.scala +++ b/memory/src/main/scala/filodb.memory/BlockMemFactoryPool.scala @@ -27,7 +27,7 @@ class BlockMemFactoryPool(blockStore: BlockManager, factoryPool.dequeue } else { logger.debug(s"Nothing in BlockMemFactory pool. Creating a new one") - new BlockMemFactory(blockStore, None, metadataAllocSize, baseTags) + new BlockMemFactory(blockStore, metadataAllocSize, baseTags) } fact.tags = baseTags ++ moreTags fact diff --git a/memory/src/main/scala/filodb.memory/MemFactory.scala b/memory/src/main/scala/filodb.memory/MemFactory.scala index f6d59d8938..78159b619f 100644 --- a/memory/src/main/scala/filodb.memory/MemFactory.scala +++ b/memory/src/main/scala/filodb.memory/MemFactory.scala @@ -202,14 +202,12 @@ object BlockMemFactory { * * @param blockStore The BlockManager which is used to request more blocks when the current * block is full. - * @param bucketTime the timebucket (timestamp) from which to allocate block(s), or None for the general list * @param metadataAllocSize the additional size in bytes to ensure is free for writing metadata, per chunk * @param tags a set of keys/values to identify the purpose of this MemFactory for debugging * @param markFullBlocksAsReclaimable Immediately mark and fully used block as reclaimable. * Typically true during on-demand paging of optimized chunks from persistent store */ class BlockMemFactory(blockStore: BlockManager, - bucketTime: Option[Long], metadataAllocSize: Int, var tags: Map[String, String], markFullBlocksAsReclaimable: Boolean = false) extends MemFactory with StrictLogging { @@ -217,12 +215,13 @@ class BlockMemFactory(blockStore: BlockManager, val optionSelf = Some(this) // tracks fully populated blocks not marked reclaimable yet (typically waiting for flush) + // NOT used in ODP block mem factories where markFullBlocksAsReclaimable = true val fullBlocksToBeMarkedAsReclaimable = ListBuffer[Block]() // tracks block currently being populated var currentBlock = requestBlock() - private def requestBlock() = blockStore.requestBlock(bucketTime, optionSelf).get + private def requestBlock() = blockStore.requestBlock(markFullBlocksAsReclaimable, optionSelf).get // tracks blocks that should share metadata private val metadataSpan: ListBuffer[Block] = ListBuffer[Block]() @@ -385,7 +384,8 @@ class BlockMemFactory(blockStore: BlockManager, def shutdown(): Unit = {} def debugString: String = - s"BlockMemFactory($bucketTime, $metadataAllocSize) ${tags.map { case (k, v) => s"$k=$v" }.mkString(" ")}" + s"BlockMemFactory($markFullBlocksAsReclaimable, $metadataAllocSize) " + + s"${tags.map { case (k, v) => s"$k=$v" }.mkString(" ")}" } diff --git a/memory/src/test/scala/filodb.memory/BlockMemFactorySpec.scala b/memory/src/test/scala/filodb.memory/BlockMemFactorySpec.scala index c79f14c66e..11ebc7e7fc 100644 --- a/memory/src/test/scala/filodb.memory/BlockMemFactorySpec.scala +++ b/memory/src/test/scala/filodb.memory/BlockMemFactorySpec.scala @@ -15,7 +15,7 @@ class BlockMemFactorySpec extends AnyFlatSpec with Matchers { it should "Mark all blocks of BlockMemFactory as reclaimable when used as done in ingestion pipeline" in { val stats = new MemoryStats(Map("test1" -> "test1")) val blockManager = new PageAlignedBlockManager(2048 * 1024, stats, testReclaimer, 1) - val bmf = new BlockMemFactory(blockManager, None, 50, Map("test" -> "val"), false) + val bmf = new BlockMemFactory(blockManager, 50, Map("test" -> "val"), false) // simulate encoding of multiple ts partitions in flush group @@ -37,13 +37,102 @@ class BlockMemFactorySpec extends AnyFlatSpec with Matchers { } // only the current block is not reclaimable - blockManager.usedBlocks.asScala.count(!_.canReclaim) shouldEqual 1 + blockManager.usedIngestionBlocks.asScala.count(!_.canReclaim) shouldEqual 1 - blockManager.usedBlocks.size shouldEqual 12 + blockManager.usedIngestionBlocks.size shouldEqual 12 blockManager.tryReclaim(3) shouldEqual 3 - blockManager.usedBlocks.size shouldEqual 9 // 3 are reclaimed + blockManager.usedIngestionBlocks.size shouldEqual 9 // 3 are reclaimed blockManager.releaseBlocks() } + it should "Mark all blocks of BlockMemFactory as reclaimable when used in ODP by DemandPagedChunkStore" in { + val stats = new MemoryStats(Map("test1" -> "test1")) + val blockManager = new PageAlignedBlockManager(2048 * 1024, stats, testReclaimer, 1) + + // create block mem factories for different time buckets + val bmf = new BlockMemFactory(blockManager, 50, Map("test" -> "val"), true) + + // simulate paging in chunks from cassandra + for {tsParts <- 0 to 10} { + bmf.startMetaSpan() + for {chunks <- 0 to 3} { + bmf.allocateOffheap(1000) + } + bmf.endMetaSpan(d => {}, 45) + } + + // we dont track full blocks in ODP mode + bmf.fullBlocksToBeMarkedAsReclaimable.isEmpty shouldEqual true + + // usedBlocks is not used for ODP mode + blockManager.usedIngestionBlocks.isEmpty shouldEqual true + + // time ordered blocks is used in ODP mode + // 11 blocks are used, out of which 10 are reclaimable, except the "current block" + blockManager.usedOdpBlocks.asScala.count(_.canReclaim) shouldEqual 10 + bmf.currentBlock.canReclaim shouldEqual false // current blocks cannot be reclaimed + + // There should be 11 time ordered blocks used before reclaim + blockManager.usedOdpBlocks.asScala.size shouldEqual 11 + blockManager.tryReclaim(5) shouldEqual 5 + // after reclaiming 5 blocks, only 11-5 == 6 time ordered blocks should be used + blockManager.usedOdpBlocks.asScala.size shouldEqual 6 + + // even if you try to reclaim all of them, you cannot reclaim current block + blockManager.tryReclaim(7) shouldEqual 5 + blockManager.usedOdpBlocks.asScala.size shouldEqual 1 + + blockManager.releaseBlocks() + } + + + it should "Reclaim Ingestion and ODP blocks in right order when used together" in { + val stats = new MemoryStats(Map("test1" -> "test1")) + val blockManager = new PageAlignedBlockManager(2048 * 1024, stats, testReclaimer, 1) + + val ingestionFactory = new BlockMemFactory(blockManager, 50, Map("test" -> "val"), false) + + // create block mem factories for different time buckets + val odpFactory = new BlockMemFactory(blockManager, 50, Map("test" -> "val"), true) + + // simulate encoding of multiple ts partitions in flush group + for { flushGroup <- 0 to 1 } { + for {tsParts <- 0 to 5} { + ingestionFactory.startMetaSpan() + for {chunks <- 0 to 3} { + ingestionFactory.allocateOffheap(1000) + } + ingestionFactory.endMetaSpan(d => {}, 45) + } + // full blocks are tracked as they are allocated + flushGroup match { + case 0 => ingestionFactory.fullBlocksToBeMarkedAsReclaimable.size shouldEqual 5 + case 1 => ingestionFactory.fullBlocksToBeMarkedAsReclaimable.size shouldEqual 6 + } + // full blocks are marked as reclaimable + ingestionFactory.markFullBlocksReclaimable() + } + + // simulate paging in chunks from cassandra + for {tsParts <- 0 to 10} { + odpFactory.startMetaSpan() + for {chunks <- 0 to 3} { + odpFactory.allocateOffheap(1000) + } + odpFactory.endMetaSpan(d => {}, 45) + } + + // here are the use block counts before reclaim call + blockManager.usedOdpBlocks.size shouldEqual 11 + blockManager.usedIngestionBlocks.size shouldEqual 12 + blockManager.tryReclaim(15) shouldEqual 15 + + // after reclaim, only 1 odp block + blockManager.usedOdpBlocks.asScala.size shouldEqual 1 + + // ingestion blocks should be reclaimed only if we cannot get reclaim ODP blocks. + blockManager.usedIngestionBlocks.asScala.size shouldEqual 7 + + } } diff --git a/memory/src/test/scala/filodb.memory/BlockSpec.scala b/memory/src/test/scala/filodb.memory/BlockSpec.scala index 0a6bcc2049..4c4d4e82f8 100644 --- a/memory/src/test/scala/filodb.memory/BlockSpec.scala +++ b/memory/src/test/scala/filodb.memory/BlockSpec.scala @@ -22,7 +22,7 @@ class BlockSpec extends AnyFlatSpec with Matchers with BeforeAndAfter with Befor } it should "allocate metadata and report remaining bytes accurately" in { - val block = blockManager.requestBlock(None).get + val block = blockManager.requestBlock(false).get block.capacity shouldEqual 4096 block.remaining shouldEqual 4096 @@ -36,7 +36,7 @@ class BlockSpec extends AnyFlatSpec with Matchers with BeforeAndAfter with Befor } it should "return null when allocate metadata if not enough space" in { - val block = blockManager.requestBlock(None).get + val block = blockManager.requestBlock(false).get block.capacity shouldEqual 4096 block.remaining shouldEqual 4096 @@ -47,13 +47,13 @@ class BlockSpec extends AnyFlatSpec with Matchers with BeforeAndAfter with Befor } it should "not reclaim when block has not been marked reclaimable" in { - val block = blockManager.requestBlock(None).get + val block = blockManager.requestBlock(false).get intercept[IllegalStateException] { block.reclaim() } } it should "call reclaimListener with address of all allocated metadatas" in { - val block = blockManager.requestBlock(None).get + val block = blockManager.requestBlock(false).get block.capacity shouldEqual 4096 block.remaining shouldEqual 4096 diff --git a/memory/src/test/scala/filodb.memory/PageAlignedBlockManagerConcurrentSpec.scala b/memory/src/test/scala/filodb.memory/PageAlignedBlockManagerConcurrentSpec.scala index ee48cfd77a..3130c1368a 100644 --- a/memory/src/test/scala/filodb.memory/PageAlignedBlockManagerConcurrentSpec.scala +++ b/memory/src/test/scala/filodb.memory/PageAlignedBlockManagerConcurrentSpec.scala @@ -23,7 +23,7 @@ with ConductorFixture with Matchers with BeforeAndAfterAll { threadNamed("Random guy") { //1 page - val blocks = blockManager.requestBlocks(pageSize, None) + val blocks = blockManager.requestBlocks(pageSize, false) blocks.size should be(1) val block = blocks.head block.position(block.position() + 1) @@ -31,7 +31,7 @@ with ConductorFixture with Matchers with BeforeAndAfterAll { } threadNamed("Another dude") { //2 page - val blocks = blockManager.requestBlocks(2 * pageSize, None) + val blocks = blockManager.requestBlocks(2 * pageSize, false) blocks.size should be(2) val block = blocks.head block.position(block.position() + 1) @@ -39,7 +39,7 @@ with ConductorFixture with Matchers with BeforeAndAfterAll { } threadNamed("Yet another dude") { //3 page - val blocks = blockManager.requestBlocks(3 * pageSize, None) + val blocks = blockManager.requestBlocks(3 * pageSize, false) blocks.size should be(3) val block = blocks.head block.position(block.position() + 1) diff --git a/memory/src/test/scala/filodb.memory/PageAlignedBlockManagerSpec.scala b/memory/src/test/scala/filodb.memory/PageAlignedBlockManagerSpec.scala index 59afa16b6e..4b82874cca 100644 --- a/memory/src/test/scala/filodb.memory/PageAlignedBlockManagerSpec.scala +++ b/memory/src/test/scala/filodb.memory/PageAlignedBlockManagerSpec.scala @@ -35,7 +35,7 @@ class PageAlignedBlockManagerSpec extends AnyFlatSpec with Matchers with BeforeA // val fbm = freeBlocksMetric(stats) // fbm.max should be(512) val blockSize = blockManager.blockSizeInBytes - val blocks = blockManager.requestBlocks(blockSize * 10, None) + val blocks = blockManager.requestBlocks(blockSize * 10, false) blocks.size should be(10) // val ubm = usedBlocksMetric(stats) // ubm.max should be(10) @@ -61,7 +61,7 @@ class PageAlignedBlockManagerSpec extends AnyFlatSpec with Matchers with BeforeA val blockSize = blockManager.blockSizeInBytes // val fbm = freeBlocksMetric(stats) // fbm.max should be(2) - val firstRequest = blockManager.requestBlocks(blockSize * 2, None) + val firstRequest = blockManager.requestBlocks(blockSize * 2, false) //used 2 out of 2 firstRequest.size should be(2) // val ubm = usedBlocksMetric(stats) @@ -69,7 +69,7 @@ class PageAlignedBlockManagerSpec extends AnyFlatSpec with Matchers with BeforeA //cannot fulfill // val fbm2 = freeBlocksMetric(stats) // fbm2.min should be(0) - val secondRequest = blockManager.requestBlocks(blockSize * 2, None) + val secondRequest = blockManager.requestBlocks(blockSize * 2, false) secondRequest should be(Seq.empty) blockManager.releaseBlocks() @@ -80,14 +80,14 @@ class PageAlignedBlockManagerSpec extends AnyFlatSpec with Matchers with BeforeA val stats = new MemoryStats(Map("test4" -> "test4")) val blockManager = new PageAlignedBlockManager(2 * pageSize, stats, testReclaimer, 1) val blockSize = blockManager.blockSizeInBytes - val firstRequest = blockManager.requestBlocks(blockSize * 2, None) + val firstRequest = blockManager.requestBlocks(blockSize * 2, false) //used 2 out of 2 firstRequest.size should be(2) //simulate writing to the block firstRequest.head.position(blockSize.toInt - 1) //mark them as reclaimable firstRequest.foreach(_.markReclaimable()) - val secondRequest = blockManager.requestBlocks(blockSize * 2, None) + val secondRequest = blockManager.requestBlocks(blockSize * 2, false) // val brm = reclaimedBlocksMetric(stats) // brm.count should be(2) //this request will fulfill @@ -102,11 +102,11 @@ class PageAlignedBlockManagerSpec extends AnyFlatSpec with Matchers with BeforeA val stats = new MemoryStats(Map("test5" -> "test5")) val blockManager = new PageAlignedBlockManager(4 * pageSize, stats, testReclaimer, 1) val blockSize = blockManager.blockSizeInBytes - val firstRequest = blockManager.requestBlocks(blockSize * 2, None) + val firstRequest = blockManager.requestBlocks(blockSize * 2, false) //used 2 out of 4 firstRequest.size should be(2) //only 2 left - cannot fulfill request - val secondRequest = blockManager.requestBlocks(blockSize * 3, None) + val secondRequest = blockManager.requestBlocks(blockSize * 3, false) // val brm = reclaimedBlocksMetric(stats) // brm.count should be(0) secondRequest should be(Seq.empty) @@ -176,28 +176,23 @@ class PageAlignedBlockManagerSpec extends AnyFlatSpec with Matchers with BeforeA // This block manager has 5 blocks capacity val blockManager = new PageAlignedBlockManager(5 * pageSize, stats, testReclaimer, 1) - blockManager.usedBlocks.size() shouldEqual 0 - blockManager.numTimeOrderedBlocks shouldEqual 0 - blockManager.usedBlocksTimeOrdered.size shouldEqual 0 - - // first allocate non-time ordered block - blockManager.requestBlock(None).map(_.markReclaimable).isDefined shouldEqual true - blockManager.usedBlocks.size shouldEqual 1 - - blockManager.requestBlock(Some(1000L)).map(_.markReclaimable).isDefined shouldEqual true - blockManager.requestBlock(Some(1000L)).map(_.markReclaimable).isDefined shouldEqual true - blockManager.requestBlock(Some(1000L)).isDefined shouldEqual true - blockManager.usedBlocksTimeOrdered.get(1000L).size() shouldEqual 3 + blockManager.usedIngestionBlocks.size() shouldEqual 0 + blockManager.usedOdpBlocks.size() shouldEqual 0 - blockManager.requestBlock(Some(9000L)).map(_.markReclaimable).isDefined shouldEqual true - blockManager.usedBlocksTimeOrdered.get(9000L).size() shouldEqual 1 + // first allocate 1 regular block + blockManager.requestBlock(false).isDefined shouldEqual true + blockManager.usedIngestionBlocks.size shouldEqual 1 - blockManager.numTimeOrderedBlocks shouldEqual 4 - blockManager.usedBlocksTimeOrdered.size shouldEqual 2 + // first allocate 4 odp blocks + blockManager.requestBlock(true).isDefined shouldEqual true + blockManager.requestBlock(true).isDefined shouldEqual true + blockManager.requestBlock(true).isDefined shouldEqual true + blockManager.requestBlock(true).isDefined shouldEqual true + blockManager.usedOdpBlocks.size() shouldEqual 4 - // reclaim from time ordered blocks should fail now + // reclaim should fail now because none of the blocks are reclaimable try { - blockManager.requestBlock(Some(10000L)) + blockManager.requestBlock(true) fail } catch { case e: MemoryRequestException => // expected @@ -211,34 +206,29 @@ class PageAlignedBlockManagerSpec extends AnyFlatSpec with Matchers with BeforeA // This block manager has 5 blocks capacity val blockManager = new PageAlignedBlockManager(5 * pageSize, stats, testReclaimer, 1) - blockManager.usedBlocks.size() shouldEqual 0 - blockManager.numTimeOrderedBlocks shouldEqual 0 - blockManager.usedBlocksTimeOrdered.size shouldEqual 0 + blockManager.usedIngestionBlocks.size() shouldEqual 0 + blockManager.usedOdpBlocks.size() shouldEqual 0 - val factory = new BlockMemFactory(blockManager, Some(10000L), 24, Map("foo" -> "bar"), false) + val factory = new BlockMemFactory(blockManager, 24, Map("foo" -> "bar"), true) // There should be one time ordered block allocated, owned by factory - blockManager.usedBlocks.size shouldEqual 0 - blockManager.numTimeOrderedBlocks shouldEqual 1 - blockManager.hasTimeBucket(10000L) shouldEqual true + blockManager.usedIngestionBlocks.size shouldEqual 0 + blockManager.usedOdpBlocks.size() shouldEqual 1 factory.currentBlock.owner shouldEqual Some(factory) // Now allocate 4 more regular blocks, that will use up all blocks - blockManager.requestBlock(None).isDefined shouldEqual true - blockManager.requestBlock(None).isDefined shouldEqual true - blockManager.requestBlock(None).isDefined shouldEqual true - blockManager.requestBlock(None).isDefined shouldEqual true - blockManager.usedBlocks.size shouldEqual 4 - blockManager.numTimeOrderedBlocks shouldEqual 1 + blockManager.requestBlock(false).isDefined shouldEqual true + blockManager.requestBlock(false).isDefined shouldEqual true + blockManager.requestBlock(false).isDefined shouldEqual true + blockManager.requestBlock(false).isDefined shouldEqual true + blockManager.usedIngestionBlocks.size shouldEqual 4 + blockManager.usedOdpBlocks.size() shouldEqual 1 // Mark as reclaimable the blockMemFactory's block. Then request more blocks, that one will be reclaimed. // Check ownership is now cleared. factory.currentBlock.markReclaimable blockManager.ensureFreeBlocks(1) - blockManager.requestBlock(Some(9000L)).isDefined shouldEqual true - blockManager.hasTimeBucket(10000L) shouldEqual false - blockManager.hasTimeBucket(9000L) shouldEqual true factory.currentBlock.owner shouldEqual None // new requestor did not have owner } @@ -252,23 +242,23 @@ class PageAlignedBlockManagerSpec extends AnyFlatSpec with Matchers with BeforeA blockManager.ensureFreePercent(50) blockManager.numFreeBlocks shouldEqual 5 - blockManager.requestBlock(None).map(_.markReclaimable).isDefined shouldEqual true + blockManager.requestBlock(false).map(_.markReclaimable).isDefined shouldEqual true blockManager.numFreeBlocks shouldEqual 4 blockManager.ensureFreePercent(50) blockManager.numFreeBlocks shouldEqual 4 - blockManager.requestBlock(None).map(_.markReclaimable).isDefined shouldEqual true + blockManager.requestBlock(false).map(_.markReclaimable).isDefined shouldEqual true blockManager.numFreeBlocks shouldEqual 3 blockManager.ensureFreePercent(50) blockManager.numFreeBlocks shouldEqual 3 - blockManager.requestBlock(None).map(_.markReclaimable).isDefined shouldEqual true + blockManager.requestBlock(false).map(_.markReclaimable).isDefined shouldEqual true blockManager.numFreeBlocks shouldEqual 2 blockManager.ensureFreePercent(50) // Should actually have done something this time. blockManager.numFreeBlocks shouldEqual 3 - blockManager.requestBlock(None).map(_.markReclaimable).isDefined shouldEqual true + blockManager.requestBlock(false).map(_.markReclaimable).isDefined shouldEqual true blockManager.numFreeBlocks shouldEqual 2 blockManager.ensureFreePercent(90) // Should reclaim multiple blocks. diff --git a/memory/src/test/scala/filodb.memory/format/vectors/IntBinaryVectorTest.scala b/memory/src/test/scala/filodb.memory/format/vectors/IntBinaryVectorTest.scala index 795f4f1ab6..c24c7d8ce9 100644 --- a/memory/src/test/scala/filodb.memory/format/vectors/IntBinaryVectorTest.scala +++ b/memory/src/test/scala/filodb.memory/format/vectors/IntBinaryVectorTest.scala @@ -119,7 +119,7 @@ class IntBinaryVectorTest extends NativeVectorTest { new MemoryStats(Map("test"-> "test")), null, 16) { freeBlocks.asScala.foreach(_.set(0x55)) // initialize blocks to nonzero value } - val blockFactory = new BlockMemFactory(blockStore, None, 24, Map("foo" -> "bar"), true) + val blockFactory = new BlockMemFactory(blockStore, 24, Map("foo" -> "bar"), true) // original values will get mixed with nonzero contents if append does not overwrite original memory val builder = IntBinaryVector.appendingVectorNoNA(blockFactory, 10, nbits = 4, signed = false) 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 ce74dbb4d0..320f5cceba 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala @@ -28,12 +28,12 @@ trait RawDataWindowingSpec extends AnyFunSpec with Matchers with BeforeAndAfter private val blockStore = new PageAlignedBlockManager(100 * 1024 * 1024, new MemoryStats(Map("test"-> "test")), null, 16) val storeConf = TestData.storeConf.copy(maxChunksSize = 200) - protected val ingestBlockHolder = new BlockMemFactory(blockStore, None, timeseriesSchema.data.blockMetaSize, + protected val ingestBlockHolder = new BlockMemFactory(blockStore, timeseriesSchema.data.blockMetaSize, MMD.dummyContext, true) protected val tsBufferPool = new WriteBufferPool(TestData.nativeMem, timeseriesDatasetWithMetric.schema.data, storeConf) - protected val ingestBlockHolder2 = new BlockMemFactory(blockStore, None, downsampleSchema.data.blockMetaSize, + protected val ingestBlockHolder2 = new BlockMemFactory(blockStore, downsampleSchema.data.blockMetaSize, MMD.dummyContext, true) protected val tsBufferPool2 = new WriteBufferPool(TestData.nativeMem, downsampleSchema.data, storeConf) From d5a4b580ebff3884e3d156184d5cab0629f2722d Mon Sep 17 00:00:00 2001 From: TanviBhavsar Date: Wed, 28 Oct 2020 21:24:23 -0700 Subject: [PATCH 07/33] misc(query): use local dispatcher for binary join when all metrics belong to same cluster (#922) --- .../queryplanner/SinglePartitionPlanner.scala | 75 ++++++++++++------- .../SinglePartitionPlannerSpec.scala | 35 ++++----- 2 files changed, 63 insertions(+), 47 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SinglePartitionPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SinglePartitionPlanner.scala index 7c27d89a1f..3be9cfbf28 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SinglePartitionPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SinglePartitionPlanner.scala @@ -39,38 +39,61 @@ class SinglePartitionPlanner(planners: Map[String, QueryPlanner], if(planner.isEmpty) planners.values.head else planner.head } - private def materializeSimpleQuery(logicalPlan: LogicalPlan, qContext: QueryContext): ExecPlan = { - getPlanner(logicalPlan).materialize(logicalPlan, qContext) - } + /** + * Returns lhs and rhs planners of BinaryJoin + */ + private def getBinaryJoinPlanners(binaryJoin: BinaryJoin) : Seq[QueryPlanner] = { + val lhsPlanners = binaryJoin.lhs match { + case b: BinaryJoin => getBinaryJoinPlanners(b) + case _ => Seq(getPlanner(binaryJoin.lhs)) - private def materializeBinaryJoin(logicalPlan: BinaryJoin, qContext: QueryContext): ExecPlan = { + } - val lhsQueryContext = qContext.copy(origQueryParams = qContext.origQueryParams.asInstanceOf[PromQlQueryParams]. - copy(promQl = LogicalPlanParser.convertToQuery(logicalPlan.lhs))) - val rhsQueryContext = qContext.copy(origQueryParams = qContext.origQueryParams.asInstanceOf[PromQlQueryParams]. - copy(promQl = LogicalPlanParser.convertToQuery(logicalPlan.rhs))) + val rhsPlanners = binaryJoin.rhs match { + case b: BinaryJoin => getBinaryJoinPlanners(b) + case _ => Seq(getPlanner(binaryJoin.rhs)) - val lhsExec = logicalPlan.lhs match { - case b: BinaryJoin => materializeBinaryJoin(b, lhsQueryContext) - case _ => getPlanner(logicalPlan.lhs).materialize(logicalPlan.lhs, lhsQueryContext) } + lhsPlanners ++ rhsPlanners + } - val rhsExec = logicalPlan.rhs match { - case b: BinaryJoin => materializeBinaryJoin(b, rhsQueryContext) - case _ => getPlanner(logicalPlan.rhs).materialize(logicalPlan.rhs, rhsQueryContext) - } + private def materializeSimpleQuery(logicalPlan: LogicalPlan, qContext: QueryContext): ExecPlan = { + getPlanner(logicalPlan).materialize(logicalPlan, qContext) + } - val onKeysReal = ExtraOnByKeysUtil.getRealOnLabels(logicalPlan, queryConfig.addExtraOnByKeysTimeRanges) - - if (logicalPlan.operator.isInstanceOf[SetOperator]) - SetOperatorExec(qContext, InProcessPlanDispatcher, Seq(lhsExec), Seq(rhsExec), logicalPlan.operator, - LogicalPlanUtils.renameLabels(onKeysReal, datasetMetricColumn), - LogicalPlanUtils.renameLabels(logicalPlan.ignoring, datasetMetricColumn), datasetMetricColumn) - else - BinaryJoinExec(qContext, InProcessPlanDispatcher, Seq(lhsExec), Seq(rhsExec), logicalPlan.operator, - logicalPlan.cardinality, LogicalPlanUtils.renameLabels(onKeysReal, datasetMetricColumn), - LogicalPlanUtils.renameLabels(logicalPlan.ignoring, datasetMetricColumn), - LogicalPlanUtils.renameLabels(logicalPlan.include, datasetMetricColumn), datasetMetricColumn) + private def materializeBinaryJoin(logicalPlan: BinaryJoin, qContext: QueryContext): ExecPlan = { + val allPlanners = getBinaryJoinPlanners(logicalPlan) + + if (allPlanners.forall(_.equals(allPlanners.head))) allPlanners.head.materialize(logicalPlan, qContext) + else { + + val lhsQueryContext = qContext.copy(origQueryParams = qContext.origQueryParams.asInstanceOf[PromQlQueryParams]. + copy(promQl = LogicalPlanParser.convertToQuery(logicalPlan.lhs))) + val rhsQueryContext = qContext.copy(origQueryParams = qContext.origQueryParams.asInstanceOf[PromQlQueryParams]. + copy(promQl = LogicalPlanParser.convertToQuery(logicalPlan.rhs))) + + val lhsExec = logicalPlan.lhs match { + case b: BinaryJoin => materializeBinaryJoin(b, lhsQueryContext) + case _ => getPlanner(logicalPlan.lhs).materialize(logicalPlan.lhs, lhsQueryContext) + } + + val rhsExec = logicalPlan.rhs match { + case b: BinaryJoin => materializeBinaryJoin(b, rhsQueryContext) + case _ => getPlanner(logicalPlan.rhs).materialize(logicalPlan.rhs, rhsQueryContext) + } + + val onKeysReal = ExtraOnByKeysUtil.getRealOnLabels(logicalPlan, queryConfig.addExtraOnByKeysTimeRanges) + + if (logicalPlan.operator.isInstanceOf[SetOperator]) + SetOperatorExec(qContext, InProcessPlanDispatcher, Seq(lhsExec), Seq(rhsExec), logicalPlan.operator, + LogicalPlanUtils.renameLabels(onKeysReal, datasetMetricColumn), + LogicalPlanUtils.renameLabels(logicalPlan.ignoring, datasetMetricColumn), datasetMetricColumn) + else + BinaryJoinExec(qContext, InProcessPlanDispatcher, Seq(lhsExec), Seq(rhsExec), logicalPlan.operator, + logicalPlan.cardinality, LogicalPlanUtils.renameLabels(onKeysReal, datasetMetricColumn), + LogicalPlanUtils.renameLabels(logicalPlan.ignoring, datasetMetricColumn), + LogicalPlanUtils.renameLabels(logicalPlan.include, datasetMetricColumn), datasetMetricColumn) + } } private def materializeLabelValues(logicalPlan: LogicalPlan, qContext: QueryContext) = { diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/SinglePartitionPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/SinglePartitionPlannerSpec.scala index 671984cb82..d6bf4e3732 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/SinglePartitionPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/SinglePartitionPlannerSpec.scala @@ -97,32 +97,32 @@ class SinglePartitionPlannerSpec extends AnyFunSpec with Matchers { val lp = Parser.queryToLogicalPlan("test1{job = \"app\"} + test2{job = \"app\"}", 1000, 1000) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) - execPlan.printTree() + execPlan.dispatcher.isInstanceOf[ActorPlanDispatcher] shouldEqual true // Since all metrics belong to same cluster execPlan.isInstanceOf[BinaryJoinExec] shouldEqual (true) execPlan.children.foreach { l1 => - l1.isInstanceOf[LocalPartitionDistConcatExec] shouldEqual true - l1.children.foreach { l2 => - l2.isInstanceOf[MultiSchemaPartitionsExec] shouldEqual true - l2.rangeVectorTransformers.size shouldEqual 1 - l2.rangeVectorTransformers(0).isInstanceOf[PeriodicSamplesMapper] shouldEqual true + l1.isInstanceOf[MultiSchemaPartitionsExec] shouldEqual true + l1.rangeVectorTransformers.size shouldEqual 1 + l1.rangeVectorTransformers(0).isInstanceOf[PeriodicSamplesMapper] shouldEqual true } } - } + it("should generate exec plan for nested Binary Join query") { val lp = Parser.queryToLogicalPlan("test1{job = \"app\"} + test2{job = \"app\"} + test3{job = \"app\"}", 1000, 1000) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + + execPlan.dispatcher.isInstanceOf[ActorPlanDispatcher] shouldEqual true // Since all metrics belong to same cluster execPlan.isInstanceOf[BinaryJoinExec] shouldEqual (true) execPlan.asInstanceOf[BinaryJoinExec].lhs.head.isInstanceOf[BinaryJoinExec] shouldEqual true - execPlan.asInstanceOf[BinaryJoinExec].rhs.head.isInstanceOf[LocalPartitionDistConcatExec] shouldEqual true + execPlan.asInstanceOf[BinaryJoinExec].rhs.head.isInstanceOf[MultiSchemaPartitionsExec] shouldEqual true } it("should generate BinaryJoin Exec plan with remote and local cluster metrics") { val lp = Parser.queryToLogicalPlan("test{job = \"app\"} + rr1{job = \"app\"}", 1000, 1000) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) - execPlan.printTree() + execPlan.dispatcher shouldEqual (InProcessPlanDispatcher) //rr1 and test belong to different clusters execPlan.isInstanceOf[BinaryJoinExec] shouldEqual (true) execPlan.asInstanceOf[BinaryJoinExec].rhs.head.asInstanceOf[MockExecPlan].name shouldEqual ("rules1") execPlan.asInstanceOf[BinaryJoinExec].lhs.head.isInstanceOf[LocalPartitionDistConcatExec] shouldEqual true @@ -132,6 +132,7 @@ class SinglePartitionPlannerSpec extends AnyFunSpec with Matchers { val lp = Parser.queryToLogicalPlan("rr1{job = \"app\"} + rr2{job = \"app\"}", 1000, 1000) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + execPlan.dispatcher shouldEqual (InProcessPlanDispatcher) //rr1 and rr2 belong to different clusters execPlan.isInstanceOf[BinaryJoinExec] shouldEqual (true) execPlan.asInstanceOf[BinaryJoinExec].lhs.head.asInstanceOf[MockExecPlan].name shouldEqual ("rules1") execPlan.asInstanceOf[BinaryJoinExec].rhs.head.asInstanceOf[MockExecPlan].name shouldEqual ("rules2") @@ -160,21 +161,13 @@ class SinglePartitionPlannerSpec extends AnyFunSpec with Matchers { execPlan.isInstanceOf[TimeScalarGeneratorExec] shouldEqual true } - it("should generate BinaryJoin Exec with remote exec's having lhs or rhs query") { + it("should generate BinaryJoin Exec with remote exec's having lhs and rhs query") { val lp = Parser.queryRangeToLogicalPlan("""test1{job = "app"} + test2{job = "app"}""", TimeStepParams(300, 20, 500)) - val promQlQueryParams = PromQlQueryParams("test1{job = \"app\"} + test2{job = \"app\"}", 300, 20, 500) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) - println(execPlan.printTree()) - execPlan.isInstanceOf[BinaryJoinExec] shouldEqual (true) - execPlan.children(0).isInstanceOf[PromQlRemoteExec] shouldEqual(true) - execPlan.children(1).isInstanceOf[PromQlRemoteExec] shouldEqual(true) - - // LHS should have only LHS query and RHS should have oly RHS query - execPlan.children(0).asInstanceOf[PromQlRemoteExec].queryContext.origQueryParams.asInstanceOf[PromQlQueryParams]. - promQl shouldEqual("""test1{job="app"}""") - execPlan.children(1).asInstanceOf[PromQlRemoteExec].queryContext.origQueryParams.asInstanceOf[PromQlQueryParams]. - promQl shouldEqual("""test2{job="app"}""") + execPlan.isInstanceOf[PromQlRemoteExec] shouldEqual (true) + execPlan.asInstanceOf[PromQlRemoteExec].queryContext.origQueryParams.asInstanceOf[PromQlQueryParams]. + promQl shouldEqual("""test1{job = "app"} + test2{job = "app"}""") } } From de8398a46424500d33558a8985b7e75e1cb613ab Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Mon, 2 Nov 2020 12:08:32 -0800 Subject: [PATCH 08/33] debug(core): Restore chunk-map-lock debugging since we have a case of unreleased lock (#940) Restore the chunk map lock state debugging code removed by PR #822 Tests indicate some queries have unreleased locks. Also adding TimeSeriesParition.toString so debug line has full signature of time series. --- .../memstore/TimeSeriesPartition.scala | 4 +++ .../scala/filodb.core/query/RangeVector.scala | 7 +++--- .../filodb.core/query/RangeVectorSpec.scala | 2 +- .../scala/filodb.memory/data/ChunkMap.scala | 25 +++++++++++++++++-- .../scala/filodb/query/exec/ExecPlan.scala | 2 +- .../query/exec/MetadataRemoteExec.scala | 2 +- .../filodb/query/exec/PromQlRemoteExec.scala | 6 ++--- .../query/exec/RangeVectorTransformer.scala | 2 +- .../aggregator/CountValuesRowAggregator.scala | 6 +---- .../aggregator/TopBottomKRowAggregator.scala | 6 +---- .../query/exec/AggrOverRangeVectorsSpec.scala | 2 +- .../exec/InProcessPlanDispatcherSpec.scala | 2 +- .../exec/rangefn/AbsentFunctionSpec.scala | 2 +- .../rangefn/AggrOverTimeFunctionsSpec.scala | 2 +- 14 files changed, 44 insertions(+), 26 deletions(-) diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala index 1cf9988924..2aed0a93dc 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala @@ -444,6 +444,10 @@ extends ChunkMap(memFactory, initMapSize) with ReadablePartition { memFactory.freeMemory(partKeyOffset) if (currentInfo != nullInfo) bufferPool.release(currentInfo.infoAddr, currentChunks) } + + override def toString: String = { + s"TimeSeriesPartition(shard=$shard,partId=$partID){$stringPartition}" + } } /** diff --git a/core/src/main/scala/filodb.core/query/RangeVector.scala b/core/src/main/scala/filodb.core/query/RangeVector.scala index 9bc146ae5b..e2ccadd790 100644 --- a/core/src/main/scala/filodb.core/query/RangeVector.scala +++ b/core/src/main/scala/filodb.core/query/RangeVector.scala @@ -316,12 +316,13 @@ object SerializedRangeVector extends StrictLogging { // scalastyle:off null def apply(rv: RangeVector, builder: RecordBuilder, - schema: RecordSchema): SerializedRangeVector = { + schema: RecordSchema, + execPlan: String): SerializedRangeVector = { var numRows = 0 val oldContainerOpt = builder.currentContainer val startRecordNo = oldContainerOpt.map(_.numRecords).getOrElse(0) try { - ChunkMap.validateNoSharedLocks() + ChunkMap.validateNoSharedLocks(execPlan) val rows = rv.rows while (rows.hasNext) { numRows += 1 @@ -349,7 +350,7 @@ object SerializedRangeVector extends StrictLogging { */ def apply(rv: RangeVector, cols: Seq[ColumnInfo]): SerializedRangeVector = { val schema = toSchema(cols) - apply(rv, newBuilder(), schema) + apply(rv, newBuilder(), schema, "Test-Only-Plan") } // TODO: make this configurable.... diff --git a/core/src/test/scala/filodb.core/query/RangeVectorSpec.scala b/core/src/test/scala/filodb.core/query/RangeVectorSpec.scala index 8c15be7390..02b355c118 100644 --- a/core/src/test/scala/filodb.core/query/RangeVectorSpec.scala +++ b/core/src/test/scala/filodb.core/query/RangeVectorSpec.scala @@ -50,7 +50,7 @@ class RangeVectorSpec extends AnyFunSpec with Matchers { val builder = SerializedRangeVector.newBuilder() // Sharing one builder across multiple input RangeVectors - val srvs = rvs.map(rv => SerializedRangeVector(rv, builder, schema)) + val srvs = rvs.map(rv => SerializedRangeVector(rv, builder, schema, "RangeVectorSpec")) // Now verify each of them val observedTs = srvs(0).rows.toSeq.map(_.getLong(0)) diff --git a/memory/src/main/scala/filodb.memory/data/ChunkMap.scala b/memory/src/main/scala/filodb.memory/data/ChunkMap.scala index 94c96031a8..8d8283d4e7 100644 --- a/memory/src/main/scala/filodb.memory/data/ChunkMap.scala +++ b/memory/src/main/scala/filodb.memory/data/ChunkMap.scala @@ -1,5 +1,7 @@ package filodb.memory.data +import java.util.concurrent.ConcurrentHashMap + import scala.collection.mutable.{HashMap, Map} import scala.concurrent.duration._ @@ -58,6 +60,13 @@ object ChunkMap extends StrictLogging { override def initialValue() = new HashMap[ChunkMap, Int] } + /** + * FIXME: Remove this after debugging is done. + * This keeps track of which thread is running which execPlan. + * Entry is added on lock acquisition, removed when lock is released. + */ + private val execPlanTracker = new ConcurrentHashMap[Thread, String] + // Returns true if the current thread has acquired the shared lock at least once. private def hasSharedLock(inst: ChunkMap): Boolean = sharedLockCounts.get.contains(inst) @@ -83,6 +92,7 @@ object ChunkMap extends StrictLogging { */ //scalastyle:off null def releaseAllSharedLocks(): Int = { + execPlanTracker.remove(Thread.currentThread()) var total = 0 val countMap = sharedLockCounts.get if (countMap != null) { @@ -109,13 +119,19 @@ 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(unitTest: Boolean = false): 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)}") + } + val numLocksReleased = ChunkMap.releaseAllSharedLocks() if (numLocksReleased > 0) { val ex = new RuntimeException(s"Number of locks was non-zero: $numLocksReleased. " + s"This is indicative of a possible lock acquisition/release bug.") Shutdown.haltAndCatchFire(ex) } + execPlanTracker.put(t, execPlan) } } @@ -241,6 +257,7 @@ class ChunkMap(val memFactory: NativeMemoryManager, var capacity: Int) { var warned = false // scalastyle:off null + var locks1: ConcurrentHashMap[Thread, String] = null while (true) { if (tryAcquireExclusive(timeoutNanos)) { if (arrayPtr == 0) { @@ -261,10 +278,14 @@ class ChunkMap(val memFactory: NativeMemoryManager, var capacity: Int) { } exclusiveLockWait.increment() _logger.warn(s"Waiting for exclusive lock: $this") + locks1 = new ConcurrentHashMap[Thread, String](execPlanTracker) warned = true } else if (warned && timeoutNanos >= MaxExclusiveRetryTimeoutNanos) { + val locks2 = new ConcurrentHashMap[Thread, String](execPlanTracker) + locks2.entrySet().retainAll(locks1.entrySet()) val lockState = UnsafeUtils.getIntVolatile(this, lockStateOffset) - Shutdown.haltAndCatchFire(new RuntimeException(s"Unable to acquire exclusive lock: $lockState")) + Shutdown.haltAndCatchFire(new RuntimeException(s"Following execPlan locks have not been released for a" + + s"while: $locks2 $locks1 $execPlanTracker $lockState")) } } } diff --git a/query/src/main/scala/filodb/query/exec/ExecPlan.scala b/query/src/main/scala/filodb/query/exec/ExecPlan.scala index 0fb9b32400..f87c7e73e0 100644 --- a/query/src/main/scala/filodb/query/exec/ExecPlan.scala +++ b/query/src/main/scala/filodb/query/exec/ExecPlan.scala @@ -167,7 +167,7 @@ trait ExecPlan extends QueryCommand { srv case rv: RangeVector => // materialize, and limit rows per RV - val srv = SerializedRangeVector(rv, builder, recSchema) + val srv = SerializedRangeVector(rv, builder, recSchema, printTree(false)) numResultSamples += srv.numRowsInt // fail the query instead of limiting range vectors and returning incomplete/inaccurate results if (enforceLimit && numResultSamples > queryContext.plannerParams.sampleLimit) diff --git a/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala b/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala index 132cbdcdf2..b88ff61fa4 100644 --- a/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala +++ b/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala @@ -49,7 +49,7 @@ case class MetadataRemoteExec(queryEndpoint: String, val rangeVector = IteratorBackedRangeVector(new CustomRangeVectorKey(Map.empty), new UTF8MapIteratorRowReader(iteratorMap.toIterator)) - val srvSeq = Seq(SerializedRangeVector(rangeVector, builder, recordSchema)) + 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/PromQlRemoteExec.scala b/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala index 4df8902783..66757481c2 100644 --- a/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala +++ b/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala @@ -115,7 +115,7 @@ case class PromQlRemoteExec(queryEndpoint: String, override def numRows: Option[Int] = Option(samples.size) } - SerializedRangeVector(rv, builder, recordSchema.get("default").get) + SerializedRangeVector(rv, builder, recordSchema.get("default").get, printTree(false)) // TODO: Handle stitching with verbose flag } QueryResult(id, resultSchema.get("default").get, rangeVectors) @@ -149,7 +149,7 @@ case class PromQlRemoteExec(queryEndpoint: String, override def numRows: Option[Int] = Option(samples.size) } - SerializedRangeVector(rv, builder, recordSchema.get("histogram").get) + SerializedRangeVector(rv, builder, recordSchema.get("histogram").get, printTree(false)) // TODO: Handle stitching with verbose flag } QueryResult(id, resultSchema.get("histogram").get, rangeVectors) @@ -173,7 +173,7 @@ case class PromQlRemoteExec(queryEndpoint: String, } override def numRows: Option[Int] = Option(d.aggregateResponse.get.aggregateSampl.size) } - SerializedRangeVector(rv, builder, recordSchema.get(Avg.entryName).get) + SerializedRangeVector(rv, builder, recordSchema.get(Avg.entryName).get, printTree(false)) } // TODO: Handle stitching with verbose flag diff --git a/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala b/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala index 06506327c7..0f03c411fd 100644 --- a/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala +++ b/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala @@ -279,7 +279,7 @@ final case class SortFunctionMapper(function: SortFunctionId) extends RangeVecto // Create SerializedRangeVector so that sorting does not consume rows iterator val resultRv = source.toListL.map { rvs => - rvs.map(SerializedRangeVector(_, builder, recSchema)). + rvs.map(SerializedRangeVector(_, builder, recSchema, s"SortRangeVectorTransformer: $args")). sortBy { rv => if (rv.rows.hasNext) rv.rows.next().getDouble(1) else Double.NaN }(ordering) 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 cb74a54df8..6cdd4704fd 100644 --- a/query/src/main/scala/filodb/query/exec/aggregator/CountValuesRowAggregator.scala +++ b/query/src/main/scala/filodb/query/exec/aggregator/CountValuesRowAggregator.scala @@ -2,8 +2,6 @@ package filodb.query.exec.aggregator import scala.collection.mutable -import kamon.Kamon - import filodb.core.binaryrecord2.RecordBuilder import filodb.core.memstore.FiloSchedulers import filodb.core.memstore.FiloSchedulers.QuerySchedName @@ -88,11 +86,10 @@ class CountValuesRowAggregator(label: String, limit: Int = 1000) extends RowAggr ColumnInfo("value", ColumnType.DoubleColumn)) val recSchema = SerializedRangeVector.toSchema(colSchema) val resRvs = mutable.Map[RangeVectorKey, RecordBuilder]() - val span = Kamon.spanBuilder(s"execplan-scan-latency-CountValues").start() try { FiloSchedulers.assertThreadName(QuerySchedName) // aggRangeVector.rows.take below triggers the ChunkInfoIterator which requires lock/release - ChunkMap.validateNoSharedLocks() + ChunkMap.validateNoSharedLocks(s"CountValues-$label") aggRangeVector.rows.take(limit).foreach { row => val rowMap = CountValuesSerDeser.deserialize(row.getBlobBase(1), row.getBlobNumBytes(1), row.getBlobOffset(1)) @@ -111,7 +108,6 @@ class CountValuesRowAggregator(label: String, limit: Int = 1000) extends RowAggr aggRangeVector.rows.close() ChunkMap.releaseAllSharedLocks() } - span.finish() resRvs.map { case (key, builder) => val numRows = builder.allContainers.map(_.countRecords()).sum new SerializedRangeVector(key, numRows, builder.allContainers, recSchema, 0) 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 43d37c5b0b..b1a32abe99 100644 --- a/query/src/main/scala/filodb/query/exec/aggregator/TopBottomKRowAggregator.scala +++ b/query/src/main/scala/filodb/query/exec/aggregator/TopBottomKRowAggregator.scala @@ -2,8 +2,6 @@ package filodb.query.exec.aggregator import scala.collection.mutable -import kamon.Kamon - import filodb.core.binaryrecord2.RecordBuilder import filodb.core.memstore.FiloSchedulers import filodb.core.memstore.FiloSchedulers.QuerySchedName @@ -88,10 +86,9 @@ class TopBottomKRowAggregator(k: Int, bottomK: Boolean) extends RowAggregator { ColumnInfo("value", ColumnType.DoubleColumn)) val recSchema = SerializedRangeVector.toSchema(colSchema) val resRvs = mutable.Map[RangeVectorKey, RecordBuilder]() - val span = Kamon.spanBuilder(s"execplan-scan-latency-TopBottomK").start() try { FiloSchedulers.assertThreadName(QuerySchedName) - ChunkMap.validateNoSharedLocks() + ChunkMap.validateNoSharedLocks(s"TopkQuery-$k-$bottomK") // We limit the results wherever it is materialized first. So it is done here. aggRangeVector.rows.take(limit).foreach { row => var i = 1 @@ -111,7 +108,6 @@ class TopBottomKRowAggregator(k: Int, bottomK: Boolean) extends RowAggregator { aggRangeVector.rows().close() ChunkMap.releaseAllSharedLocks() } - span.finish() resRvs.map { case (key, builder) => val numRows = builder.allContainers.map(_.countRecords).sum new SerializedRangeVector(key, numRows, builder.allContainers, recSchema, 0) diff --git a/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala b/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala index d33e86d497..99c2c109dd 100644 --- a/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala +++ b/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala @@ -287,7 +287,7 @@ class AggrOverRangeVectorsSpec extends RawDataWindowingSpec with ScalaFutures { val recSchema = SerializedRangeVector.toSchema(Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), ColumnInfo("tdig", ColumnType.StringColumn))) val builder = SerializedRangeVector.newBuilder() - val srv = SerializedRangeVector(result7(0), builder, recSchema) + val srv = SerializedRangeVector(result7(0), builder, recSchema, "AggrOverRangeVectorsSpec") val resultObs7b = RangeVectorAggregator.present(agg7, Observable.now(srv), 1000) val finalResult = resultObs7b.toListL.runAsync.futureValue diff --git a/query/src/test/scala/filodb/query/exec/InProcessPlanDispatcherSpec.scala b/query/src/test/scala/filodb/query/exec/InProcessPlanDispatcherSpec.scala index 0e27d05d09..6cd596d924 100644 --- a/query/src/test/scala/filodb/query/exec/InProcessPlanDispatcherSpec.scala +++ b/query/src/test/scala/filodb/query/exec/InProcessPlanDispatcherSpec.scala @@ -50,7 +50,7 @@ class InProcessPlanDispatcherSpec extends AnyFunSpec } after { - ChunkMap.validateNoSharedLocks(true) + ChunkMap.validateNoSharedLocks("InProcessPlanDispatcherSpec", true) } override def afterAll(): Unit = { 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 70262e2327..0ba50d31b1 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/AbsentFunctionSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/AbsentFunctionSpec.scala @@ -18,7 +18,7 @@ import org.scalatest.matchers.should.Matchers class AbsentFunctionSpec extends AnyFunSpec with Matchers with ScalaFutures with BeforeAndAfter { after { - ChunkMap.validateNoSharedLocks(true) + ChunkMap.validateNoSharedLocks("AbsentFunctionSpec", true) } val config: Config = ConfigFactory.load("application_test.conf").getConfig("filodb") 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 320f5cceba..db7154ddf6 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala @@ -38,7 +38,7 @@ trait RawDataWindowingSpec extends AnyFunSpec with Matchers with BeforeAndAfter protected val tsBufferPool2 = new WriteBufferPool(TestData.nativeMem, downsampleSchema.data, storeConf) after { - ChunkMap.validateNoSharedLocks(true) + ChunkMap.validateNoSharedLocks(getClass().toString(), true) } override def afterAll(): Unit = { From e128fdbac291aa3b0acb597860e3343c0798de18 Mon Sep 17 00:00:00 2001 From: Jackson Jeyapaul Date: Mon, 2 Nov 2020 13:13:02 -0800 Subject: [PATCH 09/33] misc(query): add user-agent to remote query headers (#941) --- cli/src/main/scala/filodb.cli/CliMain.scala | 7 ++++--- .../scala/filodb.core/query/QueryContext.scala | 3 ++- .../filodb/query/exec/MetadataRemoteExec.scala | 3 ++- .../scala/filodb/query/exec/PromQlRemoteExec.scala | 3 ++- .../main/scala/filodb/query/exec/RemoteExec.scala | 14 ++++++++++---- 5 files changed, 20 insertions(+), 10 deletions(-) diff --git a/cli/src/main/scala/filodb.cli/CliMain.scala b/cli/src/main/scala/filodb.cli/CliMain.scala index 130be21a8a..2e302ff167 100644 --- a/cli/src/main/scala/filodb.cli/CliMain.scala +++ b/cli/src/main/scala/filodb.cli/CliMain.scala @@ -350,9 +350,10 @@ object CliMain extends FilodbClusterNode { val ref = DatasetRef(dataset) val spreadProvider: Option[SpreadProvider] = options.spread.map(s => StaticSpreadProvider(SpreadChange(0, s))) - val qOpts = QueryContext(origQueryParams = tsdbQueryParams, plannerParams = PlannerParams(spreadOverride = - spreadProvider, sampleLimit = options.sampleLimit, queryTimeoutMillis = options.timeout.toMillis.toInt, shardOverrides = - options.shardOverrides)) + val qOpts = QueryContext(origQueryParams = tsdbQueryParams, + plannerParams = PlannerParams(applicationId = "filodb-cli", spreadOverride = spreadProvider, + sampleLimit = options.sampleLimit, queryTimeoutMillis = options.timeout.toMillis.toInt, + shardOverrides = options.shardOverrides)) println(s"Sending query command to server for $ref with options $qOpts...") println(s"Query Plan:\n$plan") options.everyN match { diff --git a/core/src/main/scala/filodb.core/query/QueryContext.scala b/core/src/main/scala/filodb.core/query/QueryContext.scala index 98b3b4e0fe..138308c15e 100644 --- a/core/src/main/scala/filodb.core/query/QueryContext.scala +++ b/core/src/main/scala/filodb.core/query/QueryContext.scala @@ -16,7 +16,8 @@ case class PromQlQueryParams(promQl: String, startSecs: Long, stepSecs: Long, en case object UnavailablePromQlQueryParams extends TsdbQueryParams -case class PlannerParams(spread: Option[Int] = None, +case class PlannerParams(applicationId: String = "filodb", + spread: Option[Int] = None, spreadOverride: Option[SpreadProvider] = None, shardOverrides: Option[Seq[Int]] = None, queryTimeoutMillis: Int = 30000, diff --git a/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala b/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala index b88ff61fa4..f6e4d69488 100644 --- a/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala +++ b/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala @@ -28,7 +28,8 @@ case class MetadataRemoteExec(queryEndpoint: String, override def sendHttpRequest(execPlan2Span: Span, httpTimeoutMs: Long) (implicit sched: Scheduler): Future[QueryResponse] = { - remoteExecHttpClient.httpMetadataGet(queryEndpoint, httpTimeoutMs, queryContext.submitTime, getUrlParams()) + remoteExecHttpClient.httpMetadataGet(queryContext.plannerParams.applicationId, queryEndpoint, + httpTimeoutMs, queryContext.submitTime, getUrlParams()) .map { response => response.unsafeBody match { case Left(error) => QueryError(queryContext.queryId, error.error) diff --git a/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala b/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala index 66757481c2..d2b0c2705b 100644 --- a/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala +++ b/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala @@ -43,7 +43,8 @@ case class PromQlRemoteExec(queryEndpoint: String, override def sendHttpRequest(execPlan2Span: Span, httpTimeoutMs: Long) (implicit sched: Scheduler): Future[QueryResponse] = { - remoteExecHttpClient.httpGet(queryEndpoint, requestTimeoutMs, queryContext.submitTime, getUrlParams()) + remoteExecHttpClient.httpGet(queryContext.plannerParams.applicationId, queryEndpoint, + requestTimeoutMs, queryContext.submitTime, getUrlParams()) .map { response => response.unsafeBody match { case Left(error) => QueryError(queryContext.queryId, error.error) diff --git a/query/src/main/scala/filodb/query/exec/RemoteExec.scala b/query/src/main/scala/filodb/query/exec/RemoteExec.scala index e2e9e0f81e..229e605e04 100644 --- a/query/src/main/scala/filodb/query/exec/RemoteExec.scala +++ b/query/src/main/scala/filodb/query/exec/RemoteExec.scala @@ -93,11 +93,13 @@ trait RemoteExec extends LeafExecPlan with StrictLogging { */ trait RemoteExecHttpClient extends StrictLogging { - def httpGet(httpEndpoint: String, httpTimeoutMs: Long, submitTime: Long, urlParams: Map[String, Any]) + def httpGet(applicationId: String, httpEndpoint: String, + httpTimeoutMs: Long, submitTime: Long, urlParams: Map[String, Any]) (implicit scheduler: Scheduler): Future[Response[scala.Either[DeserializationError[io.circe.Error], SuccessResponse]]] - def httpMetadataGet(httpEndpoint: String, httpTimeoutMs: Long, submitTime: Long, urlParams: Map[String, Any]) + def httpMetadataGet(applicationId: String, httpEndpoint: String, + httpTimeoutMs: Long, submitTime: Long, urlParams: Map[String, Any]) (implicit scheduler: Scheduler): Future[Response[scala.Either[DeserializationError[io.circe.Error], MetadataSuccessResponse]]] @@ -115,7 +117,8 @@ class RemoteHttpClient private(asyncHttpClientConfig: AsyncHttpClientConfig) ext ShutdownHookThread(shutdown()) - def httpGet(httpEndpoint: String, httpTimeoutMs: Long, submitTime: Long, urlParams: Map[String, Any]) + def httpGet(applicationId: String, 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 @@ -123,13 +126,15 @@ class RemoteHttpClient private(asyncHttpClientConfig: AsyncHttpClientConfig) ext val url = uri"$httpEndpoint?$urlParams" logger.debug("promQlExec url={}", url) sttp + .header(HeaderNames.UserAgent, applicationId) .get(url) .readTimeout(readTimeout) .response(asJson[SuccessResponse]) .send() } - def httpMetadataGet(httpEndpoint: String, httpTimeoutMs: Long, submitTime: Long, urlParams: Map[String, Any]) + def httpMetadataGet(applicationId: String, 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 @@ -137,6 +142,7 @@ class RemoteHttpClient private(asyncHttpClientConfig: AsyncHttpClientConfig) ext val url = uri"$httpEndpoint?$urlParams" logger.debug("promMetadataExec url={}", url) sttp + .header(HeaderNames.UserAgent, applicationId) .get(url) .readTimeout(readTimeout) .response(asJson[MetadataSuccessResponse]) From f87a318082fecab23f277cea32f82f9036360b0b Mon Sep 17 00:00:00 2001 From: "Brian S. O'Neill" Date: Wed, 4 Nov 2020 12:03:51 -0800 Subject: [PATCH 10/33] fix(query,memory): Lenient lock validation check to account for possible re-entrancy. --- .../scala/filodb.memory/data/ChunkMap.scala | 17 ++++++++++++++--- 1 file changed, 14 insertions(+), 3 deletions(-) diff --git a/memory/src/main/scala/filodb.memory/data/ChunkMap.scala b/memory/src/main/scala/filodb.memory/data/ChunkMap.scala index 8d8283d4e7..17c82b139f 100644 --- a/memory/src/main/scala/filodb.memory/data/ChunkMap.scala +++ b/memory/src/main/scala/filodb.memory/data/ChunkMap.scala @@ -125,12 +125,23 @@ object ChunkMap extends StrictLogging { logger.error(s"Current thread ${t.getName} did not release lock for execPlan: ${execPlanTracker.get(t)}") } - val numLocksReleased = ChunkMap.releaseAllSharedLocks() - if (numLocksReleased > 0) { - val ex = new RuntimeException(s"Number of locks was non-zero: $numLocksReleased. " + + // Count up the number of held locks. + var total = 0 + val countMap = sharedLockCounts.get + if (countMap != null) { + for ((inst, amt) <- countMap) { + if (amt > 0) { + total += amt + } + } + } + + if (total > 10) { // lenient check for now + val ex = new RuntimeException(s"Number of locks lingering: $total. " + s"This is indicative of a possible lock acquisition/release bug.") Shutdown.haltAndCatchFire(ex) } + execPlanTracker.put(t, execPlan) } From f42d1699e49a098380d7afed040a58707eded646 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Thu, 5 Nov 2020 10:20:10 -0800 Subject: [PATCH 11/33] bug(memory): BlockMemFactory.endMetaSpan not synchronized (#939) * BlockMemFactory.endMetaSpan not synchronized causing block position corruption * Remove old spans * Flush metrics on forced shutdown --- .../memstore/DemandPagedChunkStore.scala | 19 ++++-- .../memstore/TimeSeriesPartition.scala | 64 ++++++++++--------- .../main/scala/filodb.memory/MemFactory.scala | 7 +- .../scala/filodb.memory/data/Shutdown.scala | 4 ++ 4 files changed, 56 insertions(+), 38 deletions(-) diff --git a/core/src/main/scala/filodb.core/memstore/DemandPagedChunkStore.scala b/core/src/main/scala/filodb.core/memstore/DemandPagedChunkStore.scala index 6516bd33b4..726bd2838c 100644 --- a/core/src/main/scala/filodb.core/memstore/DemandPagedChunkStore.scala +++ b/core/src/main/scala/filodb.core/memstore/DemandPagedChunkStore.scala @@ -37,6 +37,11 @@ extends RawToPartitionMaker with StrictLogging { private val baseContext = Map("dataset" -> tsShard.ref.toString, "shard" -> tsShard.shardNum.toString) + + /* + * Only one BlockMemFactory for ODP per shard needed (pooling not needed) since all ODP + * allocations happen on a single thread + */ val memFactory = new BlockMemFactory(blockManager, tsShard.maxMetaSize, baseContext ++ Map("odp" -> "true"), markFullBlocksAsReclaimable = true) @@ -65,13 +70,15 @@ extends RawToPartitionMaker with StrictLogging { if (!tsPart.chunkmapContains(chunkID)) { val chunkPtrs = new ArrayBuffer[BinaryVectorPtr](rawVectors.length) - memFactory.startMetaSpan() var metaAddr: Long = 0 - try { - copyToOffHeap(rawVectors, memFactory, chunkPtrs) - } finally { - metaAddr = memFactory.endMetaSpan(writeMeta(_, tsPart.partID, infoBytes, chunkPtrs), - tsPart.schema.data.blockMetaSize.toShort) + memFactory.synchronized { + memFactory.startMetaSpan() + try { + copyToOffHeap(rawVectors, memFactory, chunkPtrs) + } finally { + metaAddr = memFactory.endMetaSpan(writeMeta(_, tsPart.partID, infoBytes, chunkPtrs), + tsPart.schema.data.blockMetaSize.toShort) + } } require(metaAddr != 0) val infoAddr = metaAddr + 4 // Important: don't point at partID diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala index 2aed0a93dc..2001011761 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala @@ -220,39 +220,41 @@ extends ChunkMap(memFactory, initMapSize) with ReadablePartition { * Optimized chunks as well as chunk metadata are written into offheap block memory so they no longer consume */ private def encodeOneChunkset(info: ChunkSetInfo, appenders: AppenderArray, blockHolder: BlockMemFactory) = { - blockHolder.startMetaSpan() - val frozenVectors = try { - // optimize and compact chunks - appenders.zipWithIndex.map { case (appender, i) => - // This assumption cannot break. We should ensure one vector can be written - // to one block always atleast as per the current design. - // If this gets triggered, decrease the max writebuffer size so smaller chunks are encoded - require(blockHolder.blockAllocationSize() > appender.frozenSize) - val optimized = appender.optimize(blockHolder) - shardStats.encodedBytes.increment(BinaryVector.totalBytes(nativePtrReader, optimized)) - if (schema.data.columns(i).columnType == Column.ColumnType.HistogramColumn) - shardStats.encodedHistBytes.increment(BinaryVector.totalBytes(nativePtrReader, optimized)) - optimized + blockHolder.synchronized { + blockHolder.startMetaSpan() + val frozenVectors = try { + // optimize and compact chunks + appenders.zipWithIndex.map { case (appender, i) => + // This assumption cannot break. We should ensure one vector can be written + // to one block always atleast as per the current design. + // If this gets triggered, decrease the max writebuffer size so smaller chunks are encoded + require(blockHolder.blockAllocationSize() > appender.frozenSize) + val optimized = appender.optimize(blockHolder) + shardStats.encodedBytes.increment(BinaryVector.totalBytes(nativePtrReader, optimized)) + if (schema.data.columns(i).columnType == Column.ColumnType.HistogramColumn) + shardStats.encodedHistBytes.increment(BinaryVector.totalBytes(nativePtrReader, optimized)) + optimized + } + } catch { case e: Exception => + // Shutdown process right away! Reaching this state means that we could not reclaim + // a whole bunch of blocks possibly because they were not marked as reclaimable, + // because of some bug. Cleanup or rollback at this point is not viable. + Shutdown.haltAndCatchFire(new RuntimeException("Error occurred when encoding vectors", e)) + throw e } - } catch { case e: Exception => - // Shutdown process right away! Reaching this state means that we could not reclaim - // a whole bunch of blocks possibly because they were not marked as reclaimable, - // because of some bug. Cleanup or rollback at this point is not viable. - Shutdown.haltAndCatchFire(new RuntimeException("Error occurred when encoding vectors", e)) - throw e + shardStats.numSamplesEncoded.increment(info.numRows) + // Now, write metadata into offheap block metadata space and update infosChunks + val metaAddr = blockHolder.endMetaSpan(TimeSeriesShard.writeMeta(_, partID, info, frozenVectors), + schema.data.blockMetaSize.toShort) + + val newInfo = ChunkSetInfo(metaAddr + 4) + _log.trace(s"Adding new chunk ${newInfo.debugString} to part $stringPartition") + infoPut(newInfo) + + // release older write buffers back to pool. Nothing at this point should reference the older appenders. + bufferPool.release(info.infoAddr, appenders) + frozenVectors } - shardStats.numSamplesEncoded.increment(info.numRows) - // Now, write metadata into offheap block metadata space and update infosChunks - val metaAddr = blockHolder.endMetaSpan(TimeSeriesShard.writeMeta(_, partID, info, frozenVectors), - schema.data.blockMetaSize.toShort) - - val newInfo = ChunkSetInfo(metaAddr + 4) - _log.trace(s"Adding new chunk ${newInfo.debugString} to part $stringPartition") - infoPut(newInfo) - - // release older write buffers back to pool. Nothing at this point should reference the older appenders. - bufferPool.release(info.infoAddr, appenders) - frozenVectors } /** diff --git a/memory/src/main/scala/filodb.memory/MemFactory.scala b/memory/src/main/scala/filodb.memory/MemFactory.scala index 78159b619f..74e2d1f4f8 100644 --- a/memory/src/main/scala/filodb.memory/MemFactory.scala +++ b/memory/src/main/scala/filodb.memory/MemFactory.scala @@ -261,6 +261,9 @@ class BlockMemFactory(blockStore: BlockManager, /** * Starts tracking a span of multiple Blocks over which the same metadata should be applied. * An example would be chunk metadata for chunks written to potentially more than 1 block. + * + * IMPORTANT: Acquire blockMemFactory.synchronized before calling startMetaSpan and release after endMetaSpan + * */ def startMetaSpan(): Unit = { metadataSpan.clear() @@ -270,6 +273,8 @@ class BlockMemFactory(blockStore: BlockManager, /** * Stops tracking the blocks that the same metadata should be applied to, and allocates and writes metadata * for those spanned blocks. + * IMPORTANT: Acquire blockMemFactory.synchronized before calling startMetaSpan and release after endMetaSpan + * * @param metadataWriter the function to write metadata to each block. Param is the long metadata address. * @param metaSize the number of bytes the piece of metadata takes * @return the Long native address of the last metadata block written @@ -295,7 +300,7 @@ class BlockMemFactory(blockStore: BlockManager, if (markFullBlocksAsReclaimable) { // We know that all the blocks in the span except the last one is full, so mark them reclaimable blk.markReclaimable() - } else synchronized { + } else { fullBlocksToBeMarkedAsReclaimable += blk } } diff --git a/memory/src/main/scala/filodb.memory/data/Shutdown.scala b/memory/src/main/scala/filodb.memory/data/Shutdown.scala index 3fdf2d6aff..2427bd3bed 100644 --- a/memory/src/main/scala/filodb.memory/data/Shutdown.scala +++ b/memory/src/main/scala/filodb.memory/data/Shutdown.scala @@ -1,5 +1,7 @@ package filodb.memory.data +import scala.concurrent.Await + import com.typesafe.scalalogging.StrictLogging import kamon.Kamon @@ -10,6 +12,8 @@ object Shutdown extends StrictLogging { forcedShutdowns.increment() if (unitTest) throw e logger.error(s"Shutting down process since it may be in an unstable/corrupt state", e) + import scala.concurrent.duration._ + Await.result(Kamon.stopModules(), 5.minutes) Runtime.getRuntime.halt(189) } From da68477e2571ca767f0114e8b74e2511f1d435f5 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Fri, 6 Nov 2020 13:30:27 -0800 Subject: [PATCH 12/33] debug(core): TracingTimeSeriesPartition trace chunkmap locks (#950) --- .../memstore/TimeSeriesPartition.scala | 22 ++++++++++++++++++- .../scala/filodb.memory/data/ChunkMap.scala | 8 +++---- 2 files changed, 25 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala index 2001011761..8e8466bd26 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala @@ -496,8 +496,28 @@ TimeSeriesPartition(partID, schema, partitionKey, shard, bufferPool, shardStats, _log.info(s"dataset=$ref schema=${schema.name} shard=$shard partId=$partID $stringPartition - " + s"newly created ChunkInfo ${currentInfo.debugString}") } -} + override def chunkmapAcquireShared(): Unit = { + super.chunkmapAcquireShared() + _log.info(s"SHARED LOCK ACQUIRED for shard=$shard partId=$partID $stringPartition", new RuntimeException) + } + + override def chunkmapReleaseShared(): Unit = { + super.chunkmapReleaseShared() + _log.info(s"SHARED LOCK RELEASED for shard=$shard partId=$partID $stringPartition", new RuntimeException) + } + + override def chunkmapAcquireExclusive(): Unit = { + super.chunkmapAcquireExclusive() + _log.info(s"EXCLUSIVE LOCK ACQUIRED for shard=$shard partId=$partID $stringPartition", new RuntimeException) + } + + override def chunkmapReleaseExclusive(): Unit = { + super.chunkmapReleaseExclusive() + _log.info(s"EXCLUSIVE LOCK RELEASED for shard=$shard partId=$partID $stringPartition", new RuntimeException) + } + +} final case class PartKeyRowReader(records: Iterator[PartKeyWithTimes]) extends Iterator[RowReader] { var currVal: PartKeyWithTimes = _ diff --git a/memory/src/main/scala/filodb.memory/data/ChunkMap.scala b/memory/src/main/scala/filodb.memory/data/ChunkMap.scala index 17c82b139f..992b38b779 100644 --- a/memory/src/main/scala/filodb.memory/data/ChunkMap.scala +++ b/memory/src/main/scala/filodb.memory/data/ChunkMap.scala @@ -258,7 +258,7 @@ class ChunkMap(val memFactory: NativeMemoryManager, var capacity: Int) { /** * Acquire exclusive access to this map, spinning if necessary. Exclusive lock isn't re-entrant. */ - final def chunkmapAcquireExclusive(): Unit = { + def chunkmapAcquireExclusive(): Unit = { // Spin-lock implementation. Because the owner of the shared lock might be blocked by this // thread as it waits for an exclusive lock, deadlock is possible. To mitigate this problem, // timeout and retry, allowing shared lock waiters to make progress. The timeout doubles @@ -352,7 +352,7 @@ class ChunkMap(val memFactory: NativeMemoryManager, var capacity: Int) { /** * Release an acquired exclusive lock. */ - final def chunkmapReleaseExclusive(): Unit = { + def chunkmapReleaseExclusive(): Unit = { UnsafeUtils.setIntVolatile(this, lockStateOffset, 0) } @@ -371,7 +371,7 @@ class ChunkMap(val memFactory: NativeMemoryManager, var capacity: Int) { /** * Acquire shared access to this map, spinning if necessary. */ - final def chunkmapAcquireShared(): Unit = { + def chunkmapAcquireShared(): Unit = { // Spin-lock implementation. var lockState = 0 @@ -391,7 +391,7 @@ class ChunkMap(val memFactory: NativeMemoryManager, var capacity: Int) { /** * Release an acquired shared lock. */ - final def chunkmapReleaseShared(): Unit = { + def chunkmapReleaseShared(): Unit = { var lockState = 0 do { lockState = UnsafeUtils.getIntVolatile(this, lockStateOffset) From 1d1b3714d150a0a0b76576c45397024346ed5311 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Fri, 6 Nov 2020 13:44:45 -0800 Subject: [PATCH 13/33] bug(memory): Log lingering locks for debugging purposes (#948) (#951) --- memory/src/main/scala/filodb.memory/data/ChunkMap.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/memory/src/main/scala/filodb.memory/data/ChunkMap.scala b/memory/src/main/scala/filodb.memory/data/ChunkMap.scala index 992b38b779..d44295aa3d 100644 --- a/memory/src/main/scala/filodb.memory/data/ChunkMap.scala +++ b/memory/src/main/scala/filodb.memory/data/ChunkMap.scala @@ -132,6 +132,8 @@ object ChunkMap extends StrictLogging { for ((inst, amt) <- countMap) { if (amt > 0) { total += amt + sharedLockLingering.increment(amt) + _logger.warn(s"THIS IS A BUG. ChunkMap shared locks was not released for: $inst, amount: $amt") } } } From 882f7ff7d3bcaf7617d8233049c17530ef24814f Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Fri, 6 Nov 2020 16:40:18 -0800 Subject: [PATCH 14/33] feat(core): Rate Limiting Number of Time Series Ingested into TSDB (#578) * Introduce Cardinality Tracker can enforce quotas. Quotas are read from static configuration file for starters * Use RocksDB Cardinality Store implementation to store cardinality information on disk at the shard level. Can add caching configuration to make it faster. * RocksDB Cardinality Store can be replaced with anything else. We are not married to it in any way. * When Quotas are breached, actions can be taken. For this PR, no action is taken except that the data is dropped. * Cardinality tracked and quotas are enforced per shard. Currently, not possible at the global level. * Filo CLI can query current cardinality at any shard level. Aggregation across shards can be added in the future. --- .travis.yml | 2 +- cli/src/main/scala/filodb.cli/CliMain.scala | 24 +- conf/timeseries-dev-source.conf | 3 + conf/timeseries-filodb-server.conf | 16 ++ .../scala/filodb.coordinator/QueryActor.scala | 10 + .../client/QueryCommands.scala | 8 +- .../filodb.coordinator/client/QueryOps.scala | 13 +- core/src/main/resources/filodb-defaults.conf | 17 +- .../binaryrecord2/RecordSchema.scala | 38 ++- .../DownsampledTimeSeriesStore.scala | 7 + .../memstore/OnDemandPagingShard.scala | 5 +- .../memstore/PartKeyLuceneIndex.scala | 2 +- .../memstore/TimeSeriesMemStore.scala | 18 +- .../memstore/TimeSeriesShard.scala | 48 +++- .../memstore/ratelimit/CardinalityStore.scala | 71 +++++ .../ratelimit/CardinalityTracker.scala | 185 +++++++++++++ .../ratelimit/QuotaExceededProtocol.scala | 33 +++ .../memstore/ratelimit/QuotaSource.scala | 60 +++++ .../ratelimit/RocksDbCardinalityStore.scala | 251 ++++++++++++++++++ .../scala/filodb.core/store/ChunkSource.scala | 4 + .../filodb.core/store/IngestionConfig.scala | 10 +- core/src/test/resources/application_test.conf | 5 + .../src/test/scala/filodb.core/TestData.scala | 5 +- .../ratelimit/CardinalityTrackerSpec.scala | 249 +++++++++++++++++ ...RocksDbCardinalityStoreMemoryCapSpec.scala | 55 ++++ project/Dependencies.scala | 5 +- project/FiloBuild.scala | 3 +- .../query/exec/InProcessPlanDispatcher.scala | 12 +- 28 files changed, 1129 insertions(+), 30 deletions(-) create mode 100644 core/src/main/scala/filodb.core/memstore/ratelimit/CardinalityStore.scala create mode 100644 core/src/main/scala/filodb.core/memstore/ratelimit/CardinalityTracker.scala create mode 100644 core/src/main/scala/filodb.core/memstore/ratelimit/QuotaExceededProtocol.scala create mode 100644 core/src/main/scala/filodb.core/memstore/ratelimit/QuotaSource.scala create mode 100644 core/src/main/scala/filodb.core/memstore/ratelimit/RocksDbCardinalityStore.scala create mode 100644 core/src/test/scala/filodb.core/memstore/ratelimit/CardinalityTrackerSpec.scala create mode 100644 core/src/test/scala/filodb.core/memstore/ratelimit/RocksDbCardinalityStoreMemoryCapSpec.scala diff --git a/.travis.yml b/.travis.yml index e2e672db87..7dd2ea1535 100644 --- a/.travis.yml +++ b/.travis.yml @@ -2,7 +2,7 @@ language: scala dist: trusty env: global: - _JAVA_OPTIONS="-Dakka.test.timefactor=3 -XX:MaxMetaspaceSize=256m" + _JAVA_OPTIONS="-Dakka.test.timefactor=3 -XX:MaxMetaspaceSize=512m" scala: - 2.11.12 jdk: diff --git a/cli/src/main/scala/filodb.cli/CliMain.scala b/cli/src/main/scala/filodb.cli/CliMain.scala index 2e302ff167..77e8e70ad7 100644 --- a/cli/src/main/scala/filodb.cli/CliMain.scala +++ b/cli/src/main/scala/filodb.cli/CliMain.scala @@ -30,7 +30,6 @@ import filodb.query._ // scalastyle:off class Arguments(args: Seq[String]) extends ScallopConf(args) { - val dataset = opt[String]() val database = opt[String]() val command = opt[String]() @@ -64,6 +63,9 @@ class Arguments(args: Seq[String]) extends ScallopConf(args) { val everynseconds = opt[String]() val shards = opt[List[String]]() val spread = opt[Int]() + val k = opt[Int]() + val shardkeyprefix = opt[List[String]](default = Some(List())) + verify() override def onError(e: Throwable): Unit = e match { @@ -101,6 +103,7 @@ object CliMain extends FilodbClusterNode { println(" --host [--port ...] --command list") println(" --host [--port ...] --command status --dataset ") println(" --host [--port ...] --command labelvalues --labelName --labelfilter --dataset ") + println(" --host [--port ...] --command topkcard --dataset prometheus --k 2 --shardkeyprefix demo App-0") println(""" --command promFilterToPartKeyBR --promql "myMetricName{_ws_='myWs',_ns_='myNs'}" --schema prom-counter""") println(""" --command partKeyBrAsString --hexpk 0x2C0000000F1712000000200000004B8B36940C006D794D65747269634E616D650E00C104006D794E73C004006D795773""") println(""" --command decodeChunkInfo --hexchunkinfo 0x12e8253a267ea2db060000005046fc896e0100005046fc896e010000""") @@ -132,8 +135,8 @@ object CliMain extends FilodbClusterNode { } def main(rawArgs: Array[String]): Unit = { - val args = new Arguments(rawArgs) try { + val args = new Arguments(rawArgs) val timeout = args.timeoutseconds().seconds args.command.toOption match { case Some("init") => @@ -165,6 +168,23 @@ object CliMain extends FilodbClusterNode { val values = remote.getIndexValues(ref, args.indexname(), args.shards().head.toInt, args.limit()) values.foreach { case (term, freq) => println(f"$term%40s\t$freq") } + case Some("topkcard") => + require(args.host.isDefined && args.dataset.isDefined && args.k.isDefined, + "--host, --dataset, --k must be defined") + val (remote, ref) = getClientAndRef(args) + val res = remote.getTopkCardinality(ref, args.shards.getOrElse(Nil).map(_.toInt), + args.shardkeyprefix(), args.k()) + println(s"ShardKeyPrefix: ${args.shardkeyprefix}") + res.groupBy(_.shard).foreach { crs => + println(s"Shard: ${crs._1}") + printf("%40s %12s %10s %10s\n", "Child", "TimeSeries", "Children", "Children") + printf("%40s %12s %10s %10s\n", "Name", "Count", "Count", "Quota") + println("===================================================================================") + crs._2.foreach { cr => + printf("%40s %12d %10d %10d\n", cr.childName, cr.timeSeriesCount, cr.childrenCount, cr.childrenQuota) + } + } + case Some("status") => val (remote, ref) = getClientAndRef(args) dumpShardStatus(remote, ref) diff --git a/conf/timeseries-dev-source.conf b/conf/timeseries-dev-source.conf index 246e2b9245..5240c576ba 100644 --- a/conf/timeseries-dev-source.conf +++ b/conf/timeseries-dev-source.conf @@ -97,6 +97,9 @@ # Limits maximum amount of data a single leaf query can scan max-data-per-shard-query = 50 MB + + # Set to true to enable metering of time series. Used for rate-limiting + metering-enabled = true } downsample { # Resolutions for downsampled data ** in ascending order ** diff --git a/conf/timeseries-filodb-server.conf b/conf/timeseries-filodb-server.conf index 2800835194..5915f323ec 100644 --- a/conf/timeseries-filodb-server.conf +++ b/conf/timeseries-filodb-server.conf @@ -11,6 +11,22 @@ filodb { "conf/timeseries-dev-source.conf" ] + quotas { + prometheus { + defaults = [100, 500, 10000, 100000] + custom = [ + { + shardKeyPrefix = ["demo", "App-0", "heap_usage"] + quota = 100 + }, + { + shardKeyPrefix = ["demo"] + quota = 10 + } + ] + } + } + spread-default = 1 # Override default spread for application using override block which will have non metric shard keys and spread. diff --git a/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala b/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala index 1b9035599d..e8d5953140 100644 --- a/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala +++ b/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala @@ -202,6 +202,15 @@ final class QueryActor(memStore: MemStore, } } + private def execTopkCardinalityQuery(q: GetTopkCardinality, sender: ActorRef): Unit = { + try { + val ret = memStore.topKCardinality(q.dataset, q.shards, q.shardKeyPrefix, q.k) + sender ! ret + } catch { case e: Exception => + sender ! QueryError(s"Error Occurred", e) + } + } + def checkTimeout(queryContext: QueryContext, replyTo: ActorRef): Boolean = { // timeout can occur here if there is a build up in actor mailbox queue and delayed delivery val queryTimeElapsed = System.currentTimeMillis() - queryContext.submitTime @@ -218,6 +227,7 @@ final class QueryActor(memStore: MemStore, case q: ExplainPlan2Query => val replyTo = sender() processExplainPlanQuery(q, replyTo) case q: ExecPlan => execPhysicalPlan2(q, sender()) + case q: GetTopkCardinality => execTopkCardinalityQuery(q, sender()) case GetIndexNames(ref, limit, _) => sender() ! memStore.indexNames(ref, limit).map(_._1).toBuffer diff --git a/coordinator/src/main/scala/filodb.coordinator/client/QueryCommands.scala b/coordinator/src/main/scala/filodb.coordinator/client/QueryCommands.scala index d03654df5f..f5f8872996 100644 --- a/coordinator/src/main/scala/filodb.coordinator/client/QueryCommands.scala +++ b/coordinator/src/main/scala/filodb.coordinator/client/QueryCommands.scala @@ -32,9 +32,11 @@ object QueryCommands { limit: Int = 100, submitTime: Long = System.currentTimeMillis()) extends QueryCommand - - - + final case class GetTopkCardinality(dataset: DatasetRef, + shards: Seq[Int], + shardKeyPrefix: Seq[String], + k: Int, + submitTime: Long = System.currentTimeMillis()) extends QueryCommand final case class StaticSpreadProvider(spreadChange: SpreadChange = SpreadChange()) extends SpreadProvider { def spreadFunc(filter: Seq[ColumnFilter]): Seq[SpreadChange] = { diff --git a/coordinator/src/main/scala/filodb.coordinator/client/QueryOps.scala b/coordinator/src/main/scala/filodb.coordinator/client/QueryOps.scala index 055dc9254c..ae7a1d1125 100644 --- a/coordinator/src/main/scala/filodb.coordinator/client/QueryOps.scala +++ b/coordinator/src/main/scala/filodb.coordinator/client/QueryOps.scala @@ -5,8 +5,9 @@ import scala.concurrent.duration._ import com.typesafe.scalalogging.StrictLogging import filodb.core._ +import filodb.core.memstore.ratelimit.CardinalityRecord import filodb.core.query.QueryContext -import filodb.query.{LogicalPlan => LogicalPlan2, QueryResponse => QueryResponse2} +import filodb.query.{LogicalPlan => LogicalPlan2, QueryError, QueryResponse => QueryResponse2} trait QueryOps extends ClientBase with StrictLogging { import QueryCommands._ @@ -44,6 +45,16 @@ trait QueryOps extends ClientBase with StrictLogging { case s: Seq[(String, Int)] @unchecked => s } + def getTopkCardinality(dataset: DatasetRef, + shards: Seq[Int], + shardKeyPrefix: Seq[String], + k: Int, + timeout: FiniteDuration = 15.seconds): Seq[CardinalityRecord] = + askCoordinator(GetTopkCardinality(dataset, shards, shardKeyPrefix, k), timeout) { + case s: Seq[CardinalityRecord] @unchecked => s + case e: QueryError => throw e.t + } + /** * Asks the FiloDB node to perform a query using a LogicalPlan. * @param dataset the Dataset (and Database) to query diff --git a/core/src/main/resources/filodb-defaults.conf b/core/src/main/resources/filodb-defaults.conf index a5b0ec168e..fec424ca82 100644 --- a/core/src/main/resources/filodb-defaults.conf +++ b/core/src/main/resources/filodb-defaults.conf @@ -44,7 +44,8 @@ filodb { ignoreShardKeyColumnSuffixes = { "_metric_" = ["_bucket", "_count", "_sum"] } ignoreTagsOnPartitionKeyHash = ["le"] metricColumn = "_metric_" - shardKeyColumns = [ "_metric_", "_ws_", "_ns_" ] + # shard key columns should be in hierarchical order + shardKeyColumns = ["_ws_", "_ns_", "_metric_"] } } @@ -109,6 +110,20 @@ filodb { } } + quotas { + # if one is not defined for data source, this number is used for all limits + default = 1000000 + # prometheus { + # defaults = [100, 500, 10000, 100000] + # custom = [ + # { + # shardKeyPrefix = ["myWs", "myNs", "myMetricName"] + # quota = 10000 + # } + # ] + # } + } + tasks { # Frequency with which new shard maps are published shardmap-publish-frequency = 5s diff --git a/core/src/main/scala/filodb.core/binaryrecord2/RecordSchema.scala b/core/src/main/scala/filodb.core/binaryrecord2/RecordSchema.scala index d71b391a1f..e302ceb4d3 100644 --- a/core/src/main/scala/filodb.core/binaryrecord2/RecordSchema.scala +++ b/core/src/main/scala/filodb.core/binaryrecord2/RecordSchema.scala @@ -260,7 +260,7 @@ final class RecordSchema(val columns: Seq[ColumnInfo], def toStringPairs(base: Any, offset: Long): Seq[(String, String)] = { import Column.ColumnType._ val result = new collection.mutable.ArrayBuffer[(String, String)]() - columnTypes.zipWithIndex.map { + columnTypes.zipWithIndex.foreach { case (IntColumn, i) => result += ((colNames(i), getInt(base, offset, i).toString)) case (LongColumn, i) => result += ((colNames(i), getLong(base, offset, i).toString)) case (DoubleColumn, i) => result += ((colNames(i), getDouble(base, offset, i).toString)) @@ -281,6 +281,30 @@ final class RecordSchema(val columns: Seq[ColumnInfo], result } + def colValues(base: Any, offset: Long, cols: Seq[String]): Seq[String] = { + import Column.ColumnType._ + val res = collection.mutable.ArrayBuffer.fill[String](cols.size)(UnsafeUtils.ZeroPointer.asInstanceOf[String]) + columnTypes.zipWithIndex.foreach { + case (IntColumn, i) if cols.contains(colNames(i)) => + res(cols.indexOf(colNames(i))) = getInt(base, offset, i).toString + case (LongColumn, i) if cols.contains(colNames(i)) => + res(cols.indexOf(colNames(i))) = getLong(base, offset, i).toString + case (DoubleColumn, i) if cols.contains(colNames(i)) => + res(cols.indexOf(colNames(i))) = getDouble(base, offset, i).toString + case (StringColumn, i) if cols.contains(colNames(i)) => + res(cols.indexOf(colNames(i))) = asJavaString(base, offset, i).toString + case (TimestampColumn, i) if cols.contains(colNames(i)) => + res(cols.indexOf(colNames(i))) = getLong(base, offset, i).toString + case (MapColumn, i) => val consumer = new SelectColsMapItemConsumer(cols, res) + consumeMapItems(base, offset, i, consumer) + case (BinaryRecordColumn, i) => ??? + case (HistogramColumn, i) => ??? + case _ => // column not selected + } + res + + } + /** * Iterates through each key/value pair of a MapColumn field without any object allocations. * How is this done? By calling the consumer for each pair and directly passing the base and offset. @@ -415,6 +439,18 @@ class DebugStringMapItemConsumer(baseOffset: Long) extends MapItemConsumer { } } + +/** + * A MapItemConsumer which selects col values from map + */ +class SelectColsMapItemConsumer(cols: Seq[String], buf: ArrayBuffer[String]) extends MapItemConsumer { + def consume(keyBase: Any, keyOffset: Long, valueBase: Any, valueOffset: Long, index: Int): Unit = { + val key = UTF8StringShort.toString(keyBase, keyOffset) + if (cols.contains(key)) buf(cols.indexOf(key)) = UTF8StringMedium.toString(valueBase, valueOffset) + } +} + + object RecordSchema { import Column.ColumnType._ diff --git a/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesStore.scala b/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesStore.scala index 0673265623..f526cefb34 100644 --- a/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesStore.scala +++ b/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesStore.scala @@ -13,6 +13,7 @@ import org.jctools.maps.NonBlockingHashMapLong import filodb.core.{DatasetRef, Response, Types} import filodb.core.memstore._ +import filodb.core.memstore.ratelimit.CardinalityRecord import filodb.core.metadata.Schemas import filodb.core.query.{ColumnFilter, QuerySession} import filodb.core.store._ @@ -169,4 +170,10 @@ extends MemStore with StrictLogging { override def readRawPartitions(ref: DatasetRef, maxChunkTime: Long, partMethod: PartitionScanMethod, chunkMethod: ChunkScanMethod): Observable[RawPartData] = ??? + + // TODO we need breakdown for downsample store too, but in a less memory intensive way + override def topKCardinality(ref: DatasetRef, + shards: Seq[Int], + shardKeyPrefix: scala.Seq[String], + k: Int): scala.Seq[CardinalityRecord] = ??? } diff --git a/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala b/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala index e036791fd7..c76bd49e0c 100644 --- a/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala +++ b/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala @@ -13,6 +13,7 @@ import monix.reactive.{Observable, OverflowStrategy} import filodb.core.{DatasetRef, Types} import filodb.core.binaryrecord2.RecordSchema +import filodb.core.memstore.ratelimit.QuotaSource import filodb.core.metadata.Schemas import filodb.core.query.QuerySession import filodb.core.store._ @@ -25,13 +26,15 @@ import filodb.memory.NativeMemoryManager class OnDemandPagingShard(ref: DatasetRef, schemas: Schemas, storeConfig: StoreConfig, + quotaSource: QuotaSource, shardNum: Int, bufferMemoryManager: NativeMemoryManager, rawStore: ColumnStore, metastore: MetaStore, evictionPolicy: PartitionEvictionPolicy) (implicit ec: ExecutionContext) extends -TimeSeriesShard(ref, schemas, storeConfig, shardNum, bufferMemoryManager, rawStore, metastore, evictionPolicy)(ec) { +TimeSeriesShard(ref, schemas, storeConfig, quotaSource, shardNum, bufferMemoryManager, rawStore, + metastore, evictionPolicy)(ec) { import TimeSeriesShard._ import FiloSchedulers._ diff --git a/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala b/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala index e064a1ae9e..77e340dec3 100644 --- a/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala +++ b/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala @@ -210,7 +210,7 @@ class PartKeyLuceneIndex(ref: DatasetRef, /** * Fetch values/terms for a specific column/key/field, in order from most frequent on down. - * Note that it iterates through all docs up to a certain limit only, so if there are too many terms + * Note that it iterates through all docs up to a certain limit only, so if there are too many terms * it will not report an accurate top k in exchange for not running too long. * @param fieldName the name of the column/field/key to get terms for * @param topK the number of top k results to fetch diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala index 4423f4ca92..7aaf49944e 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala @@ -13,6 +13,7 @@ import org.jctools.maps.NonBlockingHashMapLong import filodb.core.{DatasetRef, Response, Types} import filodb.core.downsample.DownsampleConfig +import filodb.core.memstore.ratelimit.{CardinalityRecord, ConfigQuotaSource} import filodb.core.metadata.Schemas import filodb.core.query.{ColumnFilter, QuerySession} import filodb.core.store._ @@ -30,6 +31,7 @@ extends MemStore with StrictLogging { type Shards = NonBlockingHashMapLong[TimeSeriesShard] private val datasets = new HashMap[DatasetRef, Shards] private val datasetMemFactories = new HashMap[DatasetRef, NativeMemoryManager] + private val quotaSources = new HashMap[DatasetRef, ConfigQuotaSource] val stats = new ChunkSourceStats @@ -45,6 +47,8 @@ extends MemStore with StrictLogging { def setup(ref: DatasetRef, schemas: Schemas, shard: Int, storeConf: StoreConfig, downsample: DownsampleConfig = DownsampleConfig.disabled): Unit = synchronized { val shards = datasets.getOrElseUpdate(ref, new NonBlockingHashMapLong[TimeSeriesShard](32, false)) + val quotaSource = quotaSources.getOrElseUpdate(ref, + new ConfigQuotaSource(config, schemas.part.options.shardKeyColumns.length)) if (shards.containsKey(shard)) { throw ShardAlreadySetup(ref, shard) } else { @@ -55,12 +59,21 @@ extends MemStore with StrictLogging { new NativeMemoryManager(bufferMemorySize, tags) }) - val tsdb = new OnDemandPagingShard(ref, schemas, storeConf, shard, memFactory, store, metastore, + val tsdb = new OnDemandPagingShard(ref, schemas, storeConf, quotaSource, shard, memFactory, store, metastore, partEvictionPolicy) shards.put(shard, tsdb) } } + def topKCardinality(ref: DatasetRef, shards: Seq[Int], + shardKeyPrefix: Seq[String], k: Int): Seq[CardinalityRecord] = { + datasets.get(ref).toSeq + .flatMap { ts => + ts.values().asScala + .filter(s => shards.isEmpty || shards.contains(s.shardNum)) + .flatMap(_.topKCardinality(k, shardKeyPrefix)) + } + } /** * WARNING: use only for testing. Not performant */ @@ -242,6 +255,7 @@ extends MemStore with StrictLogging { def reset(): Unit = { datasets.clear() + quotaSources.clear() store.reset() } @@ -256,8 +270,8 @@ extends MemStore with StrictLogging { // Release memory etc. def shutdown(): Unit = { + quotaSources.clear() datasets.values.foreach(_.values.asScala.foreach(_.shutdown())) - datasets.values.foreach(_.values().asScala.foreach(_.closePartKeyIndex())) reset() } diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala index 72c9392299..aae20747d0 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala @@ -25,6 +25,7 @@ import spire.syntax.cfor._ import filodb.core.{ErrorResponse, _} import filodb.core.binaryrecord2._ +import filodb.core.memstore.ratelimit.{CardinalityRecord, CardinalityTracker, QuotaSource, RocksDbCardinalityStore} import filodb.core.metadata.{Schema, Schemas} import filodb.core.query.{ColumnFilter, QuerySession} import filodb.core.store._ @@ -220,6 +221,7 @@ object SchemaMismatch { class TimeSeriesShard(val ref: DatasetRef, val schemas: Schemas, val storeConfig: StoreConfig, + quotaSource: QuotaSource, val shardNum: Int, val bufferMemoryManager: NativeMemoryManager, colStore: ColumnStore, @@ -251,6 +253,19 @@ class TimeSeriesShard(val ref: DatasetRef, private[memstore] final val partKeyIndex = new PartKeyLuceneIndex(ref, schemas.part, shardNum, storeConfig.demandPagedRetentionPeriod) + private val cardTracker: CardinalityTracker = if (storeConfig.meteringEnabled) { + // FIXME switch this to some local-disk based store when we graduate out of POC mode + val cardStore = new RocksDbCardinalityStore(ref, shardNum) + + val defaultQuota = quotaSource.getDefaults(ref) + val tracker = new CardinalityTracker(ref, shardNum, schemas.part.options.shardKeyColumns.length, + defaultQuota, cardStore) + quotaSource.getQuotas(ref).foreach { q => + tracker.setQuota(q.shardKeyPrefix, q.quota) + } + tracker + } else UnsafeUtils.ZeroPointer.asInstanceOf[CardinalityTracker] + /** * Keeps track of count of rows ingested into memstore, not necessarily flushed. * This is generally used to report status and metrics. @@ -532,6 +547,11 @@ class TimeSeriesShard(val ref: DatasetRef, _offset } + def topKCardinality(k: Int, shardKeyPrefix: Seq[String]): Seq[CardinalityRecord] = { + if (storeConfig.meteringEnabled) cardTracker.topk(k, shardKeyPrefix) + else throw new IllegalArgumentException("Metering is not enabled") + } + def startFlushingIndex(): Unit = partKeyIndex.startFlushThread(storeConfig.partIndexFlushMinDelaySeconds, storeConfig.partIndexFlushMaxDelaySeconds) @@ -555,11 +575,11 @@ class TimeSeriesShard(val ref: DatasetRef, // scalastyle:off method.length private[memstore] def bootstrapPartKey(pk: PartKeyRecord): Int = { assertThreadName(IngestSchedName) + val schemaId = RecordSchema.schemaID(pk.partKey, UnsafeUtils.arayOffset) + val schema = schemas(schemaId) val partId = if (pk.endTime == Long.MaxValue) { // this is an actively ingesting partition val group = partKeyGroup(schemas.part.binSchema, pk.partKey, UnsafeUtils.arayOffset, numGroups) - val schemaId = RecordSchema.schemaID(pk.partKey, UnsafeUtils.arayOffset) - val schema = schemas(schemaId) if (schema != Schemas.UnknownSchema) { val part = createNewPartition(pk.partKey, UnsafeUtils.arayOffset, group, CREATE_NEW_PARTID, schema, 4) // In theory, we should not get an OutOfMemPartition here since @@ -600,6 +620,10 @@ class TimeSeriesShard(val ref: DatasetRef, else activelyIngesting -= partId } shardStats.indexRecoveryNumRecordsProcessed.increment() + if (storeConfig.meteringEnabled) { + val shardKey = schema.partKeySchema.colValues(pk.partKey, UnsafeUtils.arayOffset, schema.options.shardKeyColumns) + cardTracker.incrementCount(shardKey) + } partId } @@ -715,8 +739,6 @@ class TimeSeriesShard(val ref: DatasetRef, */ def refreshPartKeyIndexBlocking(): Unit = partKeyIndex.refreshReadersBlocking() - def closePartKeyIndex(): Unit = partKeyIndex.closeIndex() - def numRowsIngested: Long = ingested def numActivePartitions: Int = partSet.size @@ -767,6 +789,11 @@ class TimeSeriesShard(val ref: DatasetRef, if (!p.ingesting) { logger.debug(s"Purging partition with partId=${p.partID} ${p.stringPartition} from " + s"memory in dataset=$ref shard=$shardNum") + if (storeConfig.meteringEnabled) { + val schema = p.schema + val shardKey = schema.partKeySchema.colValues(p.partKeyBase, p.partKeyOffset, schema.options.shardKeyColumns) + cardTracker.decrementCount(shardKey) + } removePartition(p) removedParts += p.partID numDeleted += 1 @@ -1098,11 +1125,11 @@ class TimeSeriesShard(val ref: DatasetRef, private def addPartitionForIngestion(recordBase: Any, recordOff: Long, schema: Schema, group: Int) = { assertThreadName(IngestSchedName) // TODO: remove when no longer needed - or figure out how to log only for tracing partitions - logger.debug(s"Adding ingestion record details: ${schema.ingestionSchema.debugString(recordBase, recordOff)}") + logger.trace(s"Adding ingestion record details: ${schema.ingestionSchema.debugString(recordBase, recordOff)}") val partKeyOffset = schema.comparator.buildPartKeyFromIngest(recordBase, recordOff, partKeyBuilder) val previousPartId = lookupPreviouslyAssignedPartId(partKeyArray, partKeyOffset) // TODO: remove when no longer needed - logger.debug(s"Adding part key details: ${schema.partKeySchema.debugString(partKeyArray, partKeyOffset)}") + logger.trace(s"Adding part key details: ${schema.partKeySchema.debugString(partKeyArray, partKeyOffset)}") val newPart = createNewPartition(partKeyArray, partKeyOffset, group, previousPartId, schema) if (newPart != OutOfMemPartition) { val partId = newPart.partID @@ -1111,6 +1138,11 @@ class TimeSeriesShard(val ref: DatasetRef, // add new lucene entry if this partKey was never seen before // causes endTime to be set to Long.MaxValue partKeyIndex.addPartKey(newPart.partKeyBytes, partId, startTime)() + if (storeConfig.meteringEnabled) { + val shardKey = schema.partKeySchema.colValues(newPart.partKeyBase, newPart.partKeyOffset, + schema.options.shardKeyColumns) + cardTracker.incrementCount(shardKey) + } } else { // newly created partition is re-ingesting now, so update endTime updatePartEndTimeInIndex(newPart, Long.MaxValue) @@ -1509,6 +1541,9 @@ class TimeSeriesShard(val ref: DatasetRef, } def shutdown(): Unit = { + if (storeConfig.meteringEnabled) { + cardTracker.close() + } evictedPartKeys.synchronized { if (!evictedPartKeysDisposed) { evictedPartKeysDisposed = true @@ -1516,6 +1551,7 @@ class TimeSeriesShard(val ref: DatasetRef, } } reset() // Not really needed, but clear everything just to be consistent + partKeyIndex.closeIndex() logger.info(s"Shutting down dataset=$ref shard=$shardNum") /* Don't explcitly free the memory just yet. These classes instead rely on a finalize method to ensure that no threads are accessing the memory before it's freed. diff --git a/core/src/main/scala/filodb.core/memstore/ratelimit/CardinalityStore.scala b/core/src/main/scala/filodb.core/memstore/ratelimit/CardinalityStore.scala new file mode 100644 index 0000000000..bc38b9e627 --- /dev/null +++ b/core/src/main/scala/filodb.core/memstore/ratelimit/CardinalityStore.scala @@ -0,0 +1,71 @@ +package filodb.core.memstore.ratelimit + +case class Cardinality(name: String, timeSeriesCount: Int, childrenCount: Int, childrenQuota: Int) + +/** + * + * Abstracts storage of cardinality for each shard prefix. + * + * Data model needs to represent a trie like data structure. + * For the Ws/Ns/Name shard key example, here is the trie structure: + * + *
+ * Root
+ * * myWs1
+ * ** myNs11
+ * *** myMetric111
+ * *** myMetric112
+ * ** myNs12
+ * *** myMetric121
+ * *** myMetric122
+ * * myWs2
+ * ** myNs21
+ * *** myMetric211
+ * *** myMetric212
+ * ** myNs22
+ * *** myMetric221
+ * *** myMetric222
+ * 
+ * + * The root to leaf path forms a full shard key. At each level in that path, we store + * the cardinality under that shard key prefix. + * + * The store also needs to be able to fetch immediate children of any node quickly. + * There can potentially be numerous nodes in the trie, so exhaustive tree-wide searches would + * be inefficient. So it is important that the store chosen is an implementation of some kind of tree + * like data structure and provides prefix search capability. + * + * Amount of memory used should be configurable. So it does not affect rest of the system. + * + * Implementations need to be local, fast and should not involve network I/O. + */ +trait CardinalityStore { + + /** + * This method will be called for each shard key prefix when a new time series is added + * to the index. + */ + def store(shardKeyPrefix: Seq[String], card: Cardinality): Unit + + /** + * Read existing cardinality value, if one does not exist return the zero value + * indicated + */ + def getOrZero(shardKeyPrefix: Seq[String], zero: Cardinality): Cardinality + + /** + * Remove entry from store. Need to call for each shard key prefix to fully remove shard key. + * Called when a time series is purged from the index. + */ + def remove(shardKeyPrefix: Seq[String]): Unit + + /** + * Fetch immediate children of the node for the given shard key prefix + */ + def scanChildren(shardKeyPrefix: Seq[String]): Seq[Cardinality] + + /** + * Close store. Data will be thrown away + */ + def close(): Unit +} diff --git a/core/src/main/scala/filodb.core/memstore/ratelimit/CardinalityTracker.scala b/core/src/main/scala/filodb.core/memstore/ratelimit/CardinalityTracker.scala new file mode 100644 index 0000000000..7524e64f25 --- /dev/null +++ b/core/src/main/scala/filodb.core/memstore/ratelimit/CardinalityTracker.scala @@ -0,0 +1,185 @@ +package filodb.core.memstore.ratelimit + +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer + +import com.typesafe.scalalogging.StrictLogging + +import filodb.core.DatasetRef + +case class QuotaReachedException(cannotSetShardKey: Seq[String], prefix: Seq[String], quota: Int) + extends RuntimeException + +case class CardinalityRecord(shard: Int, childName: String, timeSeriesCount: Int, + childrenCount: Int, childrenQuota: Int) + +/** + * Tracks cardinality (number of time series) under each shard key prefix. The shard key + * essentially comprises of the part key key/value pairs that determine which shard a time + * series goes into. + * + * For example if shard Key is Seq("myWs", "myNs", "myMetric") then cardinality of each prefix + * Seq(), Seq("myWs"), Seq("myWs, "myNs"), Seq("myWs", "myNs", "myMetric") would be tracked. + * + * Thus, the cardinality store can be represented as a Trie of shard key elements. For the above example, + * the trie would have 4 levels. Cardinality is tracked at each node of the trie. Both count of number + * of immediate children as well as number of time series under that level are tracked. + * + * This Cardinality Tracker can also enforce quotas. Quotas are set by invoking the setQuota method. + * While this tracker tracks both immediate children as well as total number of time series under each node, + * quota enforcement is for immediate children only. + * + * @param ref the dataset for which we track the cardinality + * @param shard the shard number + * @param shardKeyLen number of elements in the shard key + * @param defaultChildrenQuota the default quota at each level if no explicit quota is set + * @param store fast memory or disk based store where cardinality and quota can be read and written + * @param quotaExceededProtocol action to be taken when quota is breached + */ +class CardinalityTracker(ref: DatasetRef, + shard: Int, + shardKeyLen: Int, + defaultChildrenQuota: Seq[Int], + val store: CardinalityStore, + quotaExceededProtocol: QuotaExceededProtocol = NoActionQuotaProtocol) extends StrictLogging { + + require(defaultChildrenQuota.length == shardKeyLen + 1) + require(defaultChildrenQuota.forall(q => q > 0 && q < 2000000)) + logger.info(s"Initializing Cardinality Tracker for shard $shard with $defaultChildrenQuota") + + /** + * Call when a new time series with the given shard key has been added to the system. + * This will update the cardinality at each level within the trie. If quota is breached, + * QuotaReachedException will be thrown and quotaExceededProtocol will be invoked + * + * @param shardKey elements in the shard key of time series. For example: (ws, ns, name). Full shard key needed. + * @return current cardinality for each shard key prefix. There + * will be shardKeyLen + 1 items in the return value + */ + def incrementCount(shardKey: Seq[String]): Seq[Cardinality] = { + require(shardKey.length == shardKeyLen, "full shard key is needed") + + val toStore = ArrayBuffer[(Seq[String], Cardinality)]() + // first make sure there is no breach for any prefix + (0 to shardKey.length).foreach { i => + val prefix = shardKey.take(i) + val name = if (prefix.isEmpty) "" else prefix.last + val old = store.getOrZero(prefix, Cardinality(name, 0, 0, defaultChildrenQuota(i))) + val neu = old.copy(timeSeriesCount = old.timeSeriesCount + 1, + childrenCount = if (i == shardKeyLen) old.childrenCount + 1 else old.childrenCount) + if (i == shardKeyLen && neu.timeSeriesCount > neu.childrenQuota) { + quotaExceededProtocol.quotaExceeded(ref, shard, prefix, neu.childrenQuota) + throw QuotaReachedException(shardKey, prefix, neu.childrenQuota) + } + if (i > 0 && neu.timeSeriesCount == 1) { // parent's new child + val parent = toStore(i-1) + val neuParent = parent._2.copy(childrenCount = parent._2.childrenCount + 1) + toStore(i-1) = (parent._1, neuParent) + if (neuParent.childrenCount > neuParent.childrenQuota) { + quotaExceededProtocol.quotaExceeded(ref, shard, parent._1, neuParent.childrenQuota) + throw QuotaReachedException(shardKey, parent._1, neuParent.childrenQuota) + } + } + toStore += (prefix -> neu) + } + + toStore.map { case (prefix, neu) => + store.store(prefix, neu) + neu + } + } + + /** + * Fetch cardinality for given shard key or shard key prefix + * + * @param shardKeyPrefix zero or more elements that form a valid shard key prefix + */ + def getCardinality(shardKeyPrefix: Seq[String]): Cardinality = { + require(shardKeyPrefix.length <= shardKeyLen, s"Too many shard keys in $shardKeyPrefix - max $shardKeyLen") + val name = if (shardKeyPrefix.isEmpty) "" else shardKeyPrefix.last + store.getOrZero(shardKeyPrefix, Cardinality(name, 0, 0, defaultChildrenQuota(shardKeyPrefix.length))) + } + + /** + * Set quota for given shard key or shard key prefix + * + * @param shardKeyPrefix zero or more elements that form a valid shard key prefix + * @param childrenQuota maximum number of time series for this prefix + * @return current Cardinality for the prefix + */ + def setQuota(shardKeyPrefix: Seq[String], childrenQuota: Int): Cardinality = { + require(shardKeyPrefix.length <= shardKeyLen, s"Too many shard keys in $shardKeyPrefix - max $shardKeyLen") + require(childrenQuota > 0 && childrenQuota < 2000000, "Children quota invalid. Provide [1, 2000000)") + + logger.debug(s"Setting children quota for $shardKeyPrefix as $childrenQuota") + val name = if (shardKeyPrefix.isEmpty) "" else shardKeyPrefix.last + val old = store.getOrZero(shardKeyPrefix, Cardinality(name, 0, 0, defaultChildrenQuota(shardKeyPrefix.length))) + val neu = old.copy(childrenQuota = childrenQuota) + store.store(shardKeyPrefix, neu) + neu + } + + /** + * Call when an existing time series with the given shard key has been removed from the system. + * This will reduce the cardinality at each level within the trie. + * + * If cardinality reduces to 0, and the quota is the default quota, record will be removed from the store + * + * @param shardKey elements in the shard key of time series. + * For example: (ws, ns, name). Full shard key is needed. + * @return current cardinality for each shard key prefix. There + * will be shardKeyLen + 1 items in the return value + */ + def decrementCount(shardKey: Seq[String]): Seq[Cardinality] = { + require(shardKey.length == shardKeyLen, "full shard key is needed") + val toStore = (0 to shardKey.length).map { i => + val prefix = shardKey.take(i) + val old = store.getOrZero(prefix, Cardinality("", 0, 0, defaultChildrenQuota(i))) + if (old.timeSeriesCount == 0) + throw new IllegalArgumentException(s"$prefix count is already zero - cannot reduce further") + val neu = old.copy(timeSeriesCount = old.timeSeriesCount - 1) + (prefix, neu) + } + toStore.map { case (prefix, neu) => + val name = if (prefix.isEmpty) "" else prefix.last + if (neu == Cardinality(name, 0, 0, defaultChildrenQuota(prefix.length))) { + // node can be removed + store.remove(prefix) + } else { + store.store(prefix, neu) + } + neu + } + } + + /** + * Use this method to query the top-k cardinality consumers immediately + * under a provided shard key prefix + * + * @param k + * @param shardKeyPrefix zero or more elements that form a valid shard key prefix + * @return Top-K records, can the less than K if fewer children + */ + def topk(k: Int, shardKeyPrefix: Seq[String]): Seq[CardinalityRecord] = { + require(shardKeyPrefix.length <= shardKeyLen, s"Too many shard keys in $shardKeyPrefix - max $shardKeyLen") + implicit val ord = new Ordering[CardinalityRecord]() { + override def compare(x: CardinalityRecord, y: CardinalityRecord): Int = { + x.timeSeriesCount - y.timeSeriesCount + } + }.reverse + val heap = mutable.PriorityQueue[CardinalityRecord]() + store.scanChildren(shardKeyPrefix).foreach { card => + heap.enqueue( + CardinalityRecord(shard, card.name, + card.timeSeriesCount, + if (shardKeyPrefix.length == shardKeyLen - 1) card.timeSeriesCount else card.childrenCount, + card.childrenQuota)) + if (heap.size > k) heap.dequeue() + } + heap.toSeq + } + + def close(): Unit = { + store.close() + } +} diff --git a/core/src/main/scala/filodb.core/memstore/ratelimit/QuotaExceededProtocol.scala b/core/src/main/scala/filodb.core/memstore/ratelimit/QuotaExceededProtocol.scala new file mode 100644 index 0000000000..7bd01e72da --- /dev/null +++ b/core/src/main/scala/filodb.core/memstore/ratelimit/QuotaExceededProtocol.scala @@ -0,0 +1,33 @@ +package filodb.core.memstore.ratelimit + +import filodb.core.DatasetRef + +/** + * Allows for action to be taken when quota is breached + */ +trait QuotaExceededProtocol { + /** + * Invoked when quota is breached. + * + * Example actions that could be taken: + * 1. Send message to gateway to block ingestion of this shardKeyPrefix + * 2. Automatically increase quota if reasonable + * 3. Send a notification/alert to customers or operations + * + * Note that this can be invoked multiple times until either ingestion of invalid data stops + * or if quota is fixed. So implementations should ensure that actions are idempotent. + * + * @param ref dataset + * @param shardNum the shardNumber that breached the quota + * @param shardKeyPrefix the shardKeyPrefix for which quota was breached + * @param quota the actual quota number that was breached + */ + def quotaExceeded(ref: DatasetRef, shardNum: Int, shardKeyPrefix: Seq[String], quota: Int): Unit +} + +/** + * Default implementation which takes no action. + */ +object NoActionQuotaProtocol extends QuotaExceededProtocol { + def quotaExceeded(ref: DatasetRef, shardNum: Int, shardKeyPrefix: Seq[String], quota: Int): Unit = {} +} \ No newline at end of file diff --git a/core/src/main/scala/filodb.core/memstore/ratelimit/QuotaSource.scala b/core/src/main/scala/filodb.core/memstore/ratelimit/QuotaSource.scala new file mode 100644 index 0000000000..c524cd9319 --- /dev/null +++ b/core/src/main/scala/filodb.core/memstore/ratelimit/QuotaSource.scala @@ -0,0 +1,60 @@ +package filodb.core.memstore.ratelimit + +import com.typesafe.config.Config +import net.ceedubs.ficus.Ficus._ +import net.ceedubs.ficus.readers.ValueReader + +import filodb.core.DatasetRef + +case class QuotaRecord(shardKeyPrefix: Seq[String], quota: Int) + +/** + * Source of quotas for shard key prefixes. + */ +trait QuotaSource { + + /** + * Fetch all configured quotas. Invoked when a new Time Series Shard is bootstrapped. + * + * The quota represents number of immediate children allowed for the given + * shard key prefix within each shard. + */ + def getQuotas(dataset: DatasetRef): Iterator[QuotaRecord] + + /** + * Quota to use in case explicit quota record is not present. + * Return value is one item for each level of the tree. + * Hence number of items in the returned sequence should be + * shardKeyLen + 1 + */ + def getDefaults(dataset: DatasetRef): Seq[Int] +} + +/** + * QuotaSource implementation where static quota definitions are loaded from server configuration. + */ +class ConfigQuotaSource(filodbConfig: Config, shardKeyLen: Int) extends QuotaSource { + implicit val quotaReader: ValueReader[QuotaRecord] = ValueReader.relative { quotaConfig => + QuotaRecord(quotaConfig.as[Seq[String]]("shardKeyPrefix"), + quotaConfig.as[Int]("quota")) + } + + def getQuotas(dataset: DatasetRef): Iterator[QuotaRecord] = { + if (filodbConfig.hasPath(s"quotas.$dataset.custom")) { + filodbConfig.as[Seq[QuotaRecord]](s"quotas.$dataset.custom").iterator + } else { + Iterator.empty + } + } + + def getDefaults(dataset: DatasetRef): Seq[Int] = { + if (filodbConfig.hasPath(s"quotas.$dataset.custom")) { + val defaults = filodbConfig.as[Seq[Int]](s"quotas.$dataset.defaults") + require(defaults.length == shardKeyLen + 1, s"Quota defaults $defaults was not of length ${shardKeyLen + 1}") + defaults + } else { + val default = filodbConfig.as[Int]("quotas.default") + Seq.fill(shardKeyLen + 1)(default) + } + } +} \ No newline at end of file diff --git a/core/src/main/scala/filodb.core/memstore/ratelimit/RocksDbCardinalityStore.scala b/core/src/main/scala/filodb.core/memstore/ratelimit/RocksDbCardinalityStore.scala new file mode 100644 index 0000000000..d4e91502de --- /dev/null +++ b/core/src/main/scala/filodb.core/memstore/ratelimit/RocksDbCardinalityStore.scala @@ -0,0 +1,251 @@ +package filodb.core.memstore.ratelimit + +import java.io.File + +import scala.collection.mutable.ArrayBuffer +import scala.concurrent.duration._ +import scala.reflect.io.Directory + +import com.esotericsoftware.kryo.{Kryo, Serializer} +import com.esotericsoftware.kryo.io.{Input, Output} +import com.typesafe.scalalogging.StrictLogging +import kamon.Kamon +import kamon.metric.MeasurementUnit +import kamon.tag.TagSet +import monix.reactive.Observable +import org.rocksdb._ +import spire.syntax.cfor._ + +import filodb.core.{DatasetRef, GlobalScheduler} +import filodb.memory.format.UnsafeUtils + +class CardinalitySerializer extends Serializer[Cardinality] { + def write(kryo: Kryo, output: Output, card: Cardinality): Unit = { + output.writeString(card.name) + output.writeInt(card.timeSeriesCount, true) + output.writeInt(card.childrenCount, true) + output.writeInt(card.childrenQuota, true) + } + + def read(kryo: Kryo, input: Input, t: Class[Cardinality]): Cardinality = { + Cardinality(input.readString(), input.readInt(true), input.readInt(true), input.readInt(true)) + } +} + +object RocksDbCardinalityStore { + private lazy val loadRocksDbLibrary = RocksDB.loadLibrary() + private val LastKeySeparator: Char = 0x1E + private val NotLastKeySeparator: Char = 0x1D + private val NotFound = UnsafeUtils.ZeroPointer.asInstanceOf[Array[Byte]] + + // ======= DB Tuning =========== + // not making them config intentionally since RocksDB tuning needs more care + private[ratelimit] val TOTAL_OFF_HEAP_SIZE = 32L << 20 // 32 MB + private[ratelimit] val LRU_CACHE_SIZE = 16L << 20 // 16 MB + private val BLOCK_SIZE = 4096L // 4 KB + private val NUM_WRITE_BUFFERS = 4 + private val WRITE_BUF_SIZE = 4L << 20 // 4 MB + +} + +class RocksDbCardinalityStore(ref: DatasetRef, shard: Int) extends CardinalityStore with StrictLogging { + + import RocksDbCardinalityStore._ + loadRocksDbLibrary + + // ======= DB Config =========== + private val cache = new LRUCache(LRU_CACHE_SIZE) + // caps total memory used by rocksdb memTables, blockCache + private val writeBufferManager = new WriteBufferManager(TOTAL_OFF_HEAP_SIZE, cache) + private val options = { + val opts = new Options().setCreateIfMissing(true) + + val tableConfig = new BlockBasedTableConfig() + tableConfig.setBlockCache(cache) + tableConfig.setCacheIndexAndFilterBlocks(true) + tableConfig.setCacheIndexAndFilterBlocksWithHighPriority(true) + tableConfig.setPinTopLevelIndexAndFilter(true) + tableConfig.setBlockSize(BLOCK_SIZE) + opts.setTableFormatConfig(tableConfig) + + opts.setWriteBufferManager(writeBufferManager) + opts.setMaxWriteBufferNumber(NUM_WRITE_BUFFERS) // number of memtables + opts.setWriteBufferSize(WRITE_BUF_SIZE) // size of each memtable + + opts + } + + private val baseDir = new File(System.getProperty("java.io.tmpdir")) + private val baseName = s"cardStore-$ref-$shard-${System.currentTimeMillis()}" + private val dbDirInTmp = new File(baseDir, baseName) + private val db = RocksDB.open(options, dbDirInTmp.getAbsolutePath) + logger.info(s"Opening new Cardinality DB at ${dbDirInTmp.getAbsolutePath}") + + private val kryo = new ThreadLocal[Kryo]() { + override def initialValue(): Kryo = { + val k = new Kryo() + k.addDefaultSerializer(classOf[Cardinality], classOf[CardinalitySerializer]) + k + } + } + + // ======= Metrics =========== + private val tags = Map("shard" -> shard.toString, "dataset" -> ref.toString) + private val diskSpaceUsedMetric = Kamon.gauge("card-store-disk-space-used", MeasurementUnit.information.bytes) + .withTags(TagSet.from(tags)) + private val memoryUsedMetric = Kamon.gauge("card-store-offheap-mem-used") + .withTags(TagSet.from(tags)) + private val compactionBytesPendingMetric = Kamon.gauge("card-store-compaction-pending", + MeasurementUnit.information.bytes).withTags(TagSet.from(tags)) + private val numRunningCompactionsMetric = Kamon.gauge("card-store-num-running-compactions") + .withTags(TagSet.from(tags)) + private val numKeysMetric = Kamon.gauge("card-store-est-num-keys") + .withTags(TagSet.from(tags)) + + private val metricsReporter = Observable.interval(1.minute) + .onErrorRestart(Int.MaxValue) + .foreach(_ => updateMetrics())(GlobalScheduler.globalImplicitScheduler) + + var lastMetricsReportTime = 0L + private def updateMetrics(): Unit = { + val now = System.currentTimeMillis() + // dump DB stats every 5 minutes + if (now - lastMetricsReportTime > 1000 * 60 * 5 ) { + logger.info(s"Card Store Stats dataset=$ref shard=$shard $statsAsString") + lastMetricsReportTime = now + } + diskSpaceUsedMetric.update(diskSpaceUsed) + numKeysMetric.update(estimatedNumKeys) + memoryUsedMetric.update(memTablesSize + blockCacheSize + tableReadersSize) + compactionBytesPendingMetric.update(compactionBytesPending) + numRunningCompactionsMetric.update(numRunningCompactions) + } + + // List of all RocksDB properties at https://github.com/facebook/rocksdb/blob/6.12.fb/include/rocksdb/db.h#L720 + def statsAsString: String = db.getProperty("rocksdb.stats") + def estimatedNumKeys: Long = db.getLongProperty("rocksdb.estimate-num-keys") + // Returns the total size, in bytes, of all the SST files. + // WAL files are not included in the calculation. + def diskSpaceUsed: Long = db.getLongProperty("rocksdb.total-sst-files-size") + def memTablesSize: Long = db.getLongProperty("rocksdb.size-all-mem-tables") + def blockCacheSize: Long = db.getLongProperty("rocksdb.block-cache-usage") + def tableReadersSize: Long = db.getLongProperty("rocksdb.estimate-table-readers-mem") + def compactionBytesPending: Long = db.getLongProperty("rocksdb.estimate-pending-compaction-bytes") + def numRunningCompactions: Long = db.getLongProperty("rocksdb.num-running-compactions") + // consider compaction-pending yes/no + + /** + * In order to enable quick prefix search, we formulate string based keys to the RocksDB + * key-value store. + * + * For example, here is the list of rocksDb keys for a few shard keys. {LastKeySeparator} and + * {NotLastKeySeparator} are special characters chosen as separator char between shard key elements. + * {LastKeySeparator} is used just prior to last shard key element. {NotLastKeySeparator} is used otherwise. + * This model helps with fast prefix searches to do top-k scans. + * + * BTW, Remove quote chars from actual string key. + * They are there just to emphasize the shard key element in the string. "" represents the root. + * + *
+   * ""
+   * ""{LastKeySeparator}"myWs1"
+   * ""{LastKeySeparator}"myWs2"
+   * ""{NotLastKeySeparator}"myWs1"{LastKeySeparator}"myNs11"
+   * ""{NotLastKeySeparator}"myWs1"{LastKeySeparator}"myNs12"
+   * ""{NotLastKeySeparator}"myWs2"{LastKeySeparator}"myNs21"
+   * ""{NotLastKeySeparator}"myWs2"{LastKeySeparator}"myNs22"
+   * ""{NotLastKeySeparator}"myWs1"{NotLastKeySeparator}"myNs11"{LastKeySeparator}"heap_usage"
+   * ""{NotLastKeySeparator}"myWs1"{NotLastKeySeparator}"myNs11"{LastKeySeparator}"cpu_usage"
+   * ""{NotLastKeySeparator}"myWs1"{NotLastKeySeparator}"myNs11"{LastKeySeparator}"network_usage"
+   * 
+ * + * In the above tree, we simply do a prefix search on
 ""{NotLastKeySeparator}"myWs1"{LastKeySeparator} 
+ * to get namespaces under workspace myWs1. + * + * @param shardKeyPrefix Zero or more elements that make up shard key prefix + * @param prefixSearch If true, returns key that can be used to perform prefix search to + * fetch immediate children in trie. Use false to fetch one specific node. + * @return string key to use to perform reads and writes of entries into RocksDB + */ + private def toStringKey(shardKeyPrefix: Seq[String], prefixSearch: Boolean): String = { + import RocksDbCardinalityStore._ + if (shardKeyPrefix.isEmpty) { + if (prefixSearch) LastKeySeparator.toString else "" + } else { + val b = new StringBuilder + cforRange { 0 until shardKeyPrefix.length - 1 } { i => + b.append(NotLastKeySeparator) + b.append(shardKeyPrefix(i)) + } + if (prefixSearch) { + b.append(NotLastKeySeparator) + b.append(shardKeyPrefix.last) + b.append(LastKeySeparator) + } else { + b.append(LastKeySeparator) + b.append(shardKeyPrefix.last) + } + b.toString() + } + } + + private def cardinalityToBytes(card: Cardinality): Array[Byte] = { + val out = new Output(500) + kryo.get().writeObject(out, card) + out.close() + out.toBytes + } + + private def bytesToCardinality(bytes: Array[Byte]): Cardinality = { + val inp = new Input(bytes) + val c = kryo.get().readObject(inp, classOf[Cardinality]) + inp.close() + c + } + + override def store(shardKeyPrefix: Seq[String], card: Cardinality): Unit = { + val key = toStringKey(shardKeyPrefix, false).getBytes() + logger.debug(s"Storing ${new String(key)} with $card") + db.put(key, cardinalityToBytes(card)) + } + + def getOrZero(shardKeyPrefix: Seq[String], zero: Cardinality): Cardinality = { + val value = db.get(toStringKey(shardKeyPrefix, false).getBytes()) + if (value == NotFound) zero else bytesToCardinality(value) + } + + override def remove(shardKeyPrefix: Seq[String]): Unit = { + db.delete(toStringKey(shardKeyPrefix, false).getBytes()) + } + + override def scanChildren(shardKeyPrefix: Seq[String]): Seq[Cardinality] = { + val it = db.newIterator() + val searchPrefix = toStringKey(shardKeyPrefix, true) + logger.debug(s"Scanning ${new String(searchPrefix)}") + it.seek(searchPrefix.getBytes()) + val buf = ArrayBuffer[Cardinality]() + import scala.util.control.Breaks._ + + breakable { + while (it.isValid()) { + val key = new String(it.key()) + if (key.startsWith(searchPrefix)) { + buf += bytesToCardinality(it.value()) + } else break // dont continue beyond valid results + it.next() + } + } + buf + } + + def close(): Unit = { + db.cancelAllBackgroundWork(true) + db.close() + writeBufferManager.close() + cache.close() + options.close() + val directory = new Directory(dbDirInTmp) + directory.deleteRecursively() + metricsReporter.cancel() + } +} diff --git a/core/src/main/scala/filodb.core/store/ChunkSource.scala b/core/src/main/scala/filodb.core/store/ChunkSource.scala index 502f10499a..d173b161f1 100644 --- a/core/src/main/scala/filodb.core/store/ChunkSource.scala +++ b/core/src/main/scala/filodb.core/store/ChunkSource.scala @@ -9,6 +9,7 @@ import monix.reactive.Observable import filodb.core._ import filodb.core.memstore.{PartLookupResult, SchemaMismatch, TimeSeriesShard} +import filodb.core.memstore.ratelimit.CardinalityRecord import filodb.core.metadata.{Schema, Schemas} import filodb.core.query._ @@ -167,6 +168,9 @@ trait ChunkSource extends RawChunkSource with StrictLogging { RawDataRangeVector(key, partition, lookupRes.chunkMethod, ids) } } + + def topKCardinality(ref: DatasetRef, shard: Seq[Int], shardKeyPrefix: Seq[String], k: Int): Seq[CardinalityRecord] + } /** diff --git a/core/src/main/scala/filodb.core/store/IngestionConfig.scala b/core/src/main/scala/filodb.core/store/IngestionConfig.scala index 6deedbd3d2..5d556b0681 100644 --- a/core/src/main/scala/filodb.core/store/IngestionConfig.scala +++ b/core/src/main/scala/filodb.core/store/IngestionConfig.scala @@ -37,7 +37,8 @@ final case class StoreConfig(flushInterval: FiniteDuration, ensureHeadroomPercent: Double, // filters on ingested records to log in detail traceFilters: Map[String, String], - maxDataPerShardQuery: Long) { + maxDataPerShardQuery: Long, + meteringEnabled: Boolean) { import collection.JavaConverters._ def toConfig: Config = ConfigFactory.parseMap(Map("flush-interval" -> (flushInterval.toSeconds + "s"), @@ -61,7 +62,8 @@ final case class StoreConfig(flushInterval: FiniteDuration, "demand-paging-enabled" -> demandPagingEnabled, "max-data-per-shard-query" -> maxDataPerShardQuery, "evicted-pk-bloom-filter-capacity" -> evictedPkBfCapacity, - "ensure-headroom-percent" -> ensureHeadroomPercent).asJava) + "ensure-headroom-percent" -> ensureHeadroomPercent, + "metering-enabled" -> meteringEnabled).asJava) } final case class AssignShardConfig(address: String, shardList: Seq[Int]) @@ -97,6 +99,7 @@ object StoreConfig { |evicted-pk-bloom-filter-capacity = 5000000 |ensure-headroom-percent = 5.0 |trace-filters = {} + |metering-enabled = false |""".stripMargin) /** Pass in the config inside the store {} */ def apply(storeConfig: Config): StoreConfig = { @@ -128,7 +131,8 @@ object StoreConfig { config.getInt("evicted-pk-bloom-filter-capacity"), config.getDouble("ensure-headroom-percent"), config.as[Map[String, String]]("trace-filters"), - config.getMemorySize("max-data-per-shard-query").toBytes) + config.getMemorySize("max-data-per-shard-query").toBytes, + config.getBoolean("metering-enabled")) } } diff --git a/core/src/test/resources/application_test.conf b/core/src/test/resources/application_test.conf index 7699534503..62d17ce680 100644 --- a/core/src/test/resources/application_test.conf +++ b/core/src/test/resources/application_test.conf @@ -30,6 +30,11 @@ filodb { shardmap-publish-frequency = 1s } + quotas { + default = 1000000 + } + + columnstore { # Number of cache entries for the table cache tablecache-size = 50 diff --git a/core/src/test/scala/filodb.core/TestData.scala b/core/src/test/scala/filodb.core/TestData.scala index ccd932c1ae..4ee9238d89 100644 --- a/core/src/test/scala/filodb.core/TestData.scala +++ b/core/src/test/scala/filodb.core/TestData.scala @@ -320,7 +320,8 @@ object MachineMetricsData { } } - val dataset2 = Dataset("metrics2", Seq("series:string", "tags:map"), columns) + val dataset2 = Dataset("metrics2", Seq("series:string", "tags:map"), columns, + options = DatasetOptions(shardKeyColumns = Seq("_ws_", "_ns_", "_metric_"), "_metric_")) val schema2 = dataset2.schema def withMap(data: Stream[Seq[Any]], n: Int = 5, extraTags: UTF8Map = Map.empty): Stream[Seq[Any]] = @@ -339,7 +340,7 @@ object MachineMetricsData { val histDataset = Dataset("histogram", Seq("metric:string", "tags:map"), Seq("timestamp:ts", "count:long", "sum:long", "h:hist:counter=false"), - DatasetOptions.DefaultOptions.copy(metricColumn = "metric")) + options = DatasetOptions(shardKeyColumns = Seq("_ws_", "_ns_", "metric"), "metric")) var histBucketScheme: bv.HistogramBuckets = _ def linearHistSeries(startTs: Long = 100000L, numSeries: Int = 10, timeStep: Int = 1000, numBuckets: Int = 8, diff --git a/core/src/test/scala/filodb.core/memstore/ratelimit/CardinalityTrackerSpec.scala b/core/src/test/scala/filodb.core/memstore/ratelimit/CardinalityTrackerSpec.scala new file mode 100644 index 0000000000..9596f4ccdb --- /dev/null +++ b/core/src/test/scala/filodb.core/memstore/ratelimit/CardinalityTrackerSpec.scala @@ -0,0 +1,249 @@ +package filodb.core.memstore.ratelimit + +import org.scalatest.funspec.AnyFunSpec +import org.scalatest.matchers.should.Matchers + +import filodb.core.{DatasetRef, MachineMetricsData} + +class CardinalityTrackerSpec extends AnyFunSpec with Matchers { + + val ref = MachineMetricsData.dataset2.ref + + private def newCardStore = { + new RocksDbCardinalityStore(DatasetRef("test"), 0) + } + + it("should enforce quota when set explicitly for all levels") { + val t = new CardinalityTracker(ref, 0, 3, Seq(4, 4, 4, 4), newCardStore) + t.setQuota(Seq("a", "aa", "aaa"), 1) shouldEqual Cardinality("aaa", 0, 0, 1) + t.setQuota(Seq("a", "aa"), 2) shouldEqual Cardinality("aa", 0, 0, 2) + t.setQuota(Seq("a"), 1) shouldEqual Cardinality("a",0, 0, 1) + t.incrementCount(Seq("a", "aa", "aaa")) shouldEqual + Seq(Cardinality("", 1, 1, 4), + Cardinality("a", 1, 1, 1), + Cardinality("aa", 1, 1, 2), + Cardinality("aaa", 1, 1, 1)) + t.incrementCount(Seq("a", "aa", "aab")) shouldEqual + Seq(Cardinality("", 2, 1, 4), + Cardinality("a", 2, 1, 1), + Cardinality("aa", 2, 2, 2), + Cardinality("aab", 1, 1, 4)) + + val ex = intercept[QuotaReachedException] { + t.incrementCount(Seq("a", "aa", "aac")) + } + ex.prefix shouldEqual (Seq("a", "aa")) + + // increment should not have been applied for any prefix + t.getCardinality(Seq("a")) shouldEqual Cardinality("a", 2, 1, 1) + t.getCardinality(Seq("a", "aa")) shouldEqual Cardinality("aa", 2, 2, 2) + t.getCardinality(Seq("a", "aa", "aac")) shouldEqual Cardinality("aac", 0, 0, 4) + t.close() + } + + it("should invoke quota exceeded protocol when breach occurs") { + + class MyQEP extends QuotaExceededProtocol { + var breachedPrefix: Seq[String] = Nil + var breachedQuota = -1 + def quotaExceeded(ref: DatasetRef, shard: Int, shardKeyPrefix: Seq[String], quota: Int): Unit = { + breachedPrefix = shardKeyPrefix + breachedQuota = quota + } + } + + val qp = new MyQEP + val t = new CardinalityTracker(ref, 0, 3, Seq(1, 1, 1, 1), newCardStore, qp) + t.incrementCount(Seq("a", "aa", "aaa")) + val ex = intercept[QuotaReachedException] { + t.incrementCount(Seq("a", "aa", "aaa")) + } + qp.breachedQuota shouldEqual 1 + qp.breachedPrefix shouldEqual Seq("a", "aa", "aaa") + t.close() + } + + it("should enforce quota when not set for any level") { + val t = new CardinalityTracker(ref, 0, 3, Seq(4, 4, 4, 4), newCardStore) + t.incrementCount(Seq("a", "ab", "aba")) shouldEqual + Seq(Cardinality("", 1, 1, 4), + Cardinality("a", 1, 1, 4), + Cardinality("ab", 1, 1, 4), + Cardinality("aba", 1, 1, 4)) + t.close() + } + + it("should be able to enforce for top 2 levels always, and enforce for 3rd level only in some cases") { + val t = new CardinalityTracker(ref, 0, 3, Seq(20, 20, 20, 20), newCardStore) + t.setQuota(Seq("a"), 10) shouldEqual Cardinality("a", 0, 0, 10) + t.setQuota(Seq("a", "aa"), 10) shouldEqual Cardinality("aa", 0, 0, 10) + // enforce for 3rd level only for aaa + t.setQuota(Seq("a", "aa", "aaa"), 2) shouldEqual Cardinality("aaa", 0, 0, 2) + t.incrementCount(Seq("a", "aa", "aaa")) shouldEqual + Seq(Cardinality("", 1, 1, 20), + Cardinality("a", 1, 1, 10), + Cardinality("aa", 1, 1, 10), + Cardinality("aaa", 1, 1, 2)) + t.incrementCount(Seq("a", "aa", "aaa")) shouldEqual + Seq(Cardinality("", 2, 1, 20), + Cardinality("a", 2, 1, 10), + Cardinality("aa", 2, 1, 10), + Cardinality("aaa", 2, 2, 2)) + t.incrementCount(Seq("a", "aa", "aab")) shouldEqual + Seq(Cardinality("", 3, 1, 20), + Cardinality("a", 3, 1, 10), + Cardinality("aa", 3, 2, 10), + Cardinality("aab", 1, 1, 20)) + t.incrementCount(Seq("a", "aa", "aab")) shouldEqual + Seq(Cardinality("", 4, 1, 20), + Cardinality("a", 4, 1, 10), + Cardinality("aa", 4, 2, 10), + Cardinality("aab", 2, 2, 20)) + t.incrementCount(Seq("a", "aa", "aab")) shouldEqual + Seq(Cardinality("", 5, 1, 20), + Cardinality("a", 5, 1, 10), + Cardinality("aa", 5, 2, 10), + Cardinality("aab", 3, 3, 20)) + t.incrementCount(Seq("a", "aa", "aab")) shouldEqual + Seq(Cardinality("", 6, 1, 20), + Cardinality("a", 6, 1, 10), + Cardinality("aa", 6, 2, 10), + Cardinality("aab", 4, 4, 20)) + + val ex = intercept[QuotaReachedException] { + t.incrementCount(Seq("a", "aa", "aaa")) + } + ex.prefix shouldEqual Seq("a", "aa", "aaa") + t.close() + + } + + it("should be able to increase and decrease quota after it has been set before") { + val t = new CardinalityTracker(ref, 0, 3, Seq(20, 20, 20, 20), newCardStore) + t.setQuota(Seq("a"), 10) shouldEqual Cardinality("a", 0, 0, 10) + t.setQuota(Seq("a", "aa"), 10) shouldEqual Cardinality("aa", 0, 0, 10) + // enforce for 3rd level only for aaa + t.setQuota(Seq("a", "aa", "aaa"), 2) shouldEqual Cardinality("aaa", 0, 0, 2) + t.incrementCount(Seq("a", "aa", "aaa")) shouldEqual + Seq(Cardinality("", 1, 1, 20), + Cardinality("a", 1, 1, 10), + Cardinality("aa", 1, 1, 10), + Cardinality("aaa", 1, 1, 2)) + t.incrementCount(Seq("a", "aa", "aaa")) shouldEqual + Seq(Cardinality("", 2, 1, 20), + Cardinality("a", 2, 1, 10), + Cardinality("aa", 2, 1, 10), + Cardinality("aaa", 2, 2, 2)) + + val ex = intercept[QuotaReachedException] { + t.incrementCount(Seq("a", "aa", "aaa")) + } + ex.prefix shouldEqual (Seq("a", "aa", "aaa")) + + // increase quota + t.setQuota(Seq("a", "aa", "aaa"), 5) shouldEqual Cardinality("aaa", 2, 2, 5) + t.incrementCount(Seq("a", "aa", "aaa")) shouldEqual + Seq(Cardinality("", 3, 1, 20), + Cardinality("a", 3, 1, 10), + Cardinality("aa", 3, 1, 10), + Cardinality("aaa", 3, 3, 5)) + + // decrease quota + t.setQuota(Seq("a", "aa", "aaa"), 4) shouldEqual Cardinality("aaa", 3, 3, 4) + t.incrementCount(Seq("a", "aa", "aaa")) shouldEqual + Seq(Cardinality("", 4, 1, 20), + Cardinality("a", 4, 1, 10), + Cardinality("aa", 4, 1, 10), + Cardinality("aaa", 4, 4, 4)) + val ex2 = intercept[QuotaReachedException] { + t.incrementCount(Seq("a", "aa", "aaa")) + } + ex2.prefix shouldEqual Seq("a", "aa", "aaa") + t.close() + } + + it("should be able to decrease quota if count is higher than new quota") { + val t = new CardinalityTracker(ref, 0, 3, Seq(20, 20, 20, 20), newCardStore) + t.setQuota(Seq("a"), 10) shouldEqual Cardinality("a", 0, 0, 10) + t.setQuota(Seq("a", "aa"), 10) shouldEqual Cardinality("aa", 0, 0, 10) + t.incrementCount(Seq("a", "aa", "aab")) shouldEqual + Seq(Cardinality("", 1, 1, 20), + Cardinality("a", 1, 1, 10), + Cardinality("aa", 1, 1, 10), + Cardinality("aab", 1, 1, 20)) + t.incrementCount(Seq("a", "aa", "aab")) shouldEqual + Seq(Cardinality("", 2, 1, 20), + Cardinality("a", 2, 1, 10), + Cardinality("aa", 2, 1, 10), + Cardinality("aab", 2, 2, 20)) + t.incrementCount(Seq("a", "aa", "aab")) shouldEqual + Seq(Cardinality("", 3, 1, 20), + Cardinality("a", 3, 1, 10), + Cardinality("aa", 3, 1, 10), + Cardinality("aab", 3, 3, 20)) + t.incrementCount(Seq("a", "aa", "aab")) shouldEqual + Seq(Cardinality("", 4, 1, 20), + Cardinality("a", 4, 1, 10), + Cardinality("aa", 4, 1, 10), + Cardinality("aab", 4, 4, 20)) + + t.getCardinality(Seq("a", "aa", "aab")) shouldEqual Cardinality("aab", 4, 4, 20) + + t.setQuota(Seq("a", "aa", "aab"), 3) + t.getCardinality(Seq("a", "aa", "aab")) shouldEqual Cardinality("aab", 4, 4, 3) + val ex2 = intercept[QuotaReachedException] { + t.incrementCount(Seq("a", "aa", "aab")) + } + ex2.prefix shouldEqual Seq("a", "aa", "aab") + t.close() + } + + it ("should be able to do topk") { + val t = new CardinalityTracker(ref, 0, 3, Seq(100, 100, 100, 100), newCardStore) + (1 to 10).foreach(_ => t.incrementCount(Seq("a", "ac", "aca"))) + (1 to 20).foreach(_ => t.incrementCount(Seq("a", "ac", "acb"))) + (1 to 11).foreach(_ => t.incrementCount(Seq("a", "ac", "acc"))) + (1 to 6).foreach(_ => t.incrementCount(Seq("a", "ac", "acd"))) + (1 to 1).foreach(_ => t.incrementCount(Seq("a", "ac", "ace"))) + (1 to 9).foreach(_ => t.incrementCount(Seq("a", "ac", "acf"))) + (1 to 15).foreach(_ => t.incrementCount(Seq("a", "ac", "acg"))) + + (1 to 15).foreach(_ => t.incrementCount(Seq("b", "bc", "bcg"))) + (1 to 9).foreach(_ => t.incrementCount(Seq("b", "bc", "bch"))) + (1 to 9).foreach(_ => t.incrementCount(Seq("b", "bd", "bdh"))) + + (1 to 3).foreach(_ => t.incrementCount(Seq("c", "cc", "ccg"))) + (1 to 2).foreach(_ => t.incrementCount(Seq("c", "cc", "cch"))) + + t.incrementCount(Seq("a", "aa", "aaa")) + t.incrementCount(Seq("a", "aa", "aab")) + t.incrementCount(Seq("a", "aa", "aac")) + t.incrementCount(Seq("a", "aa", "aad")) + t.incrementCount(Seq("b", "ba", "baa")) + t.incrementCount(Seq("b", "bb", "bba")) + t.incrementCount(Seq("a", "ab", "aba")) + t.incrementCount(Seq("a", "ab", "abb")) + t.incrementCount(Seq("a", "ab", "abc")) + t.incrementCount(Seq("a", "ab", "abd")) + t.incrementCount(Seq("a", "ab", "abe")) + + t.topk(3, Seq("a", "ac")) shouldEqual Seq( + CardinalityRecord(0, "acc", 11, 11, 100), + CardinalityRecord(0, "acg", 15, 15, 100), + CardinalityRecord(0, "acb", 20, 20, 100) + ) + + t.topk(3, Seq("a")) shouldEqual Seq( + CardinalityRecord(0, "aa", 4, 4, 100), + CardinalityRecord(0, "ab", 5, 5, 100), + CardinalityRecord(0, "ac", 72, 7, 100) + ) + + t.topk(3, Nil) shouldEqual Seq( + CardinalityRecord(0, "c", 5, 1, 100), + CardinalityRecord(0, "a", 81, 3, 100), + CardinalityRecord(0, "b", 35, 4, 100) + ) + t.close() + } +} \ No newline at end of file diff --git a/core/src/test/scala/filodb.core/memstore/ratelimit/RocksDbCardinalityStoreMemoryCapSpec.scala b/core/src/test/scala/filodb.core/memstore/ratelimit/RocksDbCardinalityStoreMemoryCapSpec.scala new file mode 100644 index 0000000000..5d7bbf230c --- /dev/null +++ b/core/src/test/scala/filodb.core/memstore/ratelimit/RocksDbCardinalityStoreMemoryCapSpec.scala @@ -0,0 +1,55 @@ +package filodb.core.memstore.ratelimit + +import org.scalatest.funspec.AnyFunSpec +import org.scalatest.matchers.should.Matchers + +import filodb.core.MachineMetricsData +import filodb.core.memstore.ratelimit.RocksDbCardinalityStore._ + +class RocksDbCardinalityStoreMemoryCapSpec extends AnyFunSpec with Matchers { + + val ref = MachineMetricsData.dataset2.ref + + val db = new RocksDbCardinalityStore(ref, 0) + val tracker = new CardinalityTracker(ref, 0, 3, Seq(100, 100, 1000, 1000), db) + + it("should be able to write keys quickly and cap memory usage") { + + def dumpStats() = { + println(db.statsAsString) + println(s"memTablesSize=${db.memTablesSize}") + println(s"blockCacheSize=${db.blockCacheSize}") + println(s"diskSpaceUsed=${db.diskSpaceUsed}") + println(s"estimatedNumKeys=${db.estimatedNumKeys}") + println() + } + + def assertStats() = { + db.blockCacheSize should be < LRU_CACHE_SIZE + (db.memTablesSize + db.blockCacheSize) should be < TOTAL_OFF_HEAP_SIZE + db.diskSpaceUsed should be < (100L << 20) + } + + val start = System.nanoTime() + for { ws <- 0 until 5 + ns <- 0 until 20 + name <- 0 until 50 + ts <- 0 until 100 } { + val mName = s"name_really_really_really_really_very_really_long_metric_name_$name" + tracker.incrementCount(Seq( s"ws_prefix_$ws", s"ns_prefix_$ns", mName)) + if (name == 0 && ts ==0 ) assertStats() + } + val end = System.nanoTime() + + assertStats() + dumpStats() + val numTimeSeries = 5 * 20 * 100 * 50 + val totalTimeSecs = (end-start) / 1000000000L + val timePerIncrementMicroSecs = (end-start) / numTimeSeries / 1000 + println(s"Was able to increment $numTimeSeries time series, $timePerIncrementMicroSecs" + + s"us each increment total of ${totalTimeSecs}s") + timePerIncrementMicroSecs should be < 200L + + } + +} diff --git a/project/Dependencies.scala b/project/Dependencies.scala index 53c70769ed..809f30edf9 100644 --- a/project/Dependencies.scala +++ b/project/Dependencies.scala @@ -71,7 +71,10 @@ object Dependencies { "com.github.rholder.fauxflake" % "fauxflake-core" % "1.1.0", "org.scalactic" %% "scalactic" % "3.2.0" withJavadoc(), "org.apache.lucene" % "lucene-core" % "7.3.0" withJavadoc(), - "com.github.alexandrnikitin" %% "bloom-filter" % "0.11.0" + "com.github.alexandrnikitin" %% "bloom-filter" % "0.11.0", + "org.rocksdb" % "rocksdbjni" % "6.11.4", + "com.esotericsoftware" % "kryo" % "4.0.0" excludeAll(excludeMinlog), + "com.dorkbox" % "MinLog-SLF4J" % "1.12" ) lazy val sparkJobsDeps = commonDeps ++ Seq( diff --git a/project/FiloBuild.scala b/project/FiloBuild.scala index 27989aab7b..de15abc600 100644 --- a/project/FiloBuild.scala +++ b/project/FiloBuild.scala @@ -41,7 +41,8 @@ object Submodules { libraryDependencies ++= coordDeps, libraryDependencies += "com.typesafe.akka" %% "akka-contrib" % akkaVersion exclude( - "com.typesafe.akka", s"akka-persistence-experimental_${scalaBinaryVersion.value}") + "com.typesafe.akka", s"akka-persistence-experimental_${scalaBinaryVersion.value}"), + allExcludeDependencies ++= Seq(excludeMinlog) ) lazy val prometheus = (project in file("prometheus")) diff --git a/query/src/main/scala/filodb/query/exec/InProcessPlanDispatcher.scala b/query/src/main/scala/filodb/query/exec/InProcessPlanDispatcher.scala index d760603dd0..db7883adb7 100644 --- a/query/src/main/scala/filodb/query/exec/InProcessPlanDispatcher.scala +++ b/query/src/main/scala/filodb/query/exec/InProcessPlanDispatcher.scala @@ -7,6 +7,7 @@ import monix.reactive.Observable import filodb.core.{DatasetRef, Types} import filodb.core.memstore.PartLookupResult +import filodb.core.memstore.ratelimit.CardinalityRecord import filodb.core.metadata.Schemas import filodb.core.query.{EmptyQueryConfig, QueryConfig, QuerySession} import filodb.core.store._ @@ -70,10 +71,13 @@ case class UnsupportedChunkSource() extends ChunkSource { chunkMethod: ChunkScanMethod): Observable[RawPartData] = throw new UnsupportedOperationException("This operation is not supported") - /** - * True if this store is in the mode of serving downsampled data. - * This is used to switch ingestion and query behaviors for downsample cluster. - */ override def isDownsampleStore: Boolean = false + + override def topKCardinality(ref: DatasetRef, + shards: Seq[Int], + shardKeyPrefix: scala.Seq[String], + k: Int): scala.Seq[CardinalityRecord] = + throw new UnsupportedOperationException("This operation is not supported") + } From da33958cec377f4a74689f89e078096e13d36da7 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Mon, 9 Nov 2020 15:31:56 -0800 Subject: [PATCH 15/33] maint(build): Remove unnecessary exclusion in sbt build (#953) --- .../memstore/ratelimit/RocksDbCardinalityStore.scala | 6 +++--- project/FiloBuild.scala | 3 +-- 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/filodb.core/memstore/ratelimit/RocksDbCardinalityStore.scala b/core/src/main/scala/filodb.core/memstore/ratelimit/RocksDbCardinalityStore.scala index d4e91502de..364ec030b8 100644 --- a/core/src/main/scala/filodb.core/memstore/ratelimit/RocksDbCardinalityStore.scala +++ b/core/src/main/scala/filodb.core/memstore/ratelimit/RocksDbCardinalityStore.scala @@ -79,7 +79,7 @@ class RocksDbCardinalityStore(ref: DatasetRef, shard: Int) extends CardinalitySt private val baseName = s"cardStore-$ref-$shard-${System.currentTimeMillis()}" private val dbDirInTmp = new File(baseDir, baseName) private val db = RocksDB.open(options, dbDirInTmp.getAbsolutePath) - logger.info(s"Opening new Cardinality DB at ${dbDirInTmp.getAbsolutePath}") + logger.info(s"Opening new Cardinality DB for shard=$shard dataset=$ref at ${dbDirInTmp.getAbsolutePath}") private val kryo = new ThreadLocal[Kryo]() { override def initialValue(): Kryo = { @@ -205,7 +205,7 @@ class RocksDbCardinalityStore(ref: DatasetRef, shard: Int) extends CardinalitySt override def store(shardKeyPrefix: Seq[String], card: Cardinality): Unit = { val key = toStringKey(shardKeyPrefix, false).getBytes() - logger.debug(s"Storing ${new String(key)} with $card") + logger.debug(s"Storing shard=$shard dataset=$ref ${new String(key)} with $card") db.put(key, cardinalityToBytes(card)) } @@ -221,7 +221,7 @@ class RocksDbCardinalityStore(ref: DatasetRef, shard: Int) extends CardinalitySt override def scanChildren(shardKeyPrefix: Seq[String]): Seq[Cardinality] = { val it = db.newIterator() val searchPrefix = toStringKey(shardKeyPrefix, true) - logger.debug(s"Scanning ${new String(searchPrefix)}") + logger.debug(s"Scanning shard=$shard dataset=$ref ${new String(searchPrefix)}") it.seek(searchPrefix.getBytes()) val buf = ArrayBuffer[Cardinality]() import scala.util.control.Breaks._ diff --git a/project/FiloBuild.scala b/project/FiloBuild.scala index de15abc600..27989aab7b 100644 --- a/project/FiloBuild.scala +++ b/project/FiloBuild.scala @@ -41,8 +41,7 @@ object Submodules { libraryDependencies ++= coordDeps, libraryDependencies += "com.typesafe.akka" %% "akka-contrib" % akkaVersion exclude( - "com.typesafe.akka", s"akka-persistence-experimental_${scalaBinaryVersion.value}"), - allExcludeDependencies ++= Seq(excludeMinlog) + "com.typesafe.akka", s"akka-persistence-experimental_${scalaBinaryVersion.value}") ) lazy val prometheus = (project in file("prometheus")) From 2652b62056612e52bb0d61c346431637909f1e0a Mon Sep 17 00:00:00 2001 From: TanviBhavsar Date: Tue, 10 Nov 2020 16:00:51 -0800 Subject: [PATCH 16/33] feat(query): stddev, stdvar for shard key regex (#938) --- .../scala/filodb.core/query/ResultTypes.scala | 1 + .../prometheus/query/PrometheusModel.scala | 13 +++++ .../main/scala/filodb/query/PlanEnums.scala | 4 ++ .../scala/filodb/query/PromCirceSupport.scala | 16 +++++- .../filodb/query/PromQueryResponse.scala | 1 + .../filodb/query/exec/PromQlRemoteExec.scala | 39 ++++++++++++-- .../filodb/query/PromCirceSupportSpec.scala | 54 ++++++++++++++++++- 7 files changed, 120 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/filodb.core/query/ResultTypes.scala b/core/src/main/scala/filodb.core/query/ResultTypes.scala index 46eec48b56..602235c1b0 100644 --- a/core/src/main/scala/filodb.core/query/ResultTypes.scala +++ b/core/src/main/scala/filodb.core/query/ResultTypes.scala @@ -52,6 +52,7 @@ final case class ResultSchema(columns: Seq[ColumnInfo], numRowKeyColumns: Int, columns(1).colType == HistogramColumn && columns(2).colType == DoubleColumn def isHistogram: Boolean = columns.length == 2 && columns(1).colType == HistogramColumn def isAvgAggregator: Boolean = columns.length == 3 && columns(2).name.equals("count") + def isStdValAggregator: Boolean = columns.length == 4 && columns(2).name.equals("mean") def hasSameColumnsAs(other: ResultSchema): Boolean = { // exclude fixedVectorLen & colIDs diff --git a/prometheus/src/main/scala/filodb/prometheus/query/PrometheusModel.scala b/prometheus/src/main/scala/filodb/prometheus/query/PrometheusModel.scala index 1bd34eb941..4032228a0a 100644 --- a/prometheus/src/main/scala/filodb/prometheus/query/PrometheusModel.scala +++ b/prometheus/src/main/scala/filodb/prometheus/query/PrometheusModel.scala @@ -167,6 +167,19 @@ object PrometheusModel { Result(tags, None, None, Some(AggregateResponse(Avg.entryName, samples))) } + def toStdValResult(srv: RangeVector, + verbose: Boolean, + typ: QueryResultType, + processMultiPartition: Boolean = true): Result = { + val tags = srv.key.labelValues.map { case (k, v) => (k.toString, v.toString)} ++ + (if (verbose) makeVerboseLabels(srv.key) + else Map.empty) + val samples = srv.rows.map { r => StdValSampl(r.getLong(0)/1000, r.getDouble(1), + r.getDouble(2), r.getLong(3)) + }.toSeq + + Result(tags, None, None, Some(AggregateResponse(QueryFunctionConstants.stdVal, samples))) + } def makeVerboseLabels(rvk: RangeVectorKey): Map[String, String] = { Map("_shards_" -> rvk.sourceShards.mkString(","), diff --git a/query/src/main/scala/filodb/query/PlanEnums.scala b/query/src/main/scala/filodb/query/PlanEnums.scala index 4d0fe5bc74..87aaede047 100644 --- a/query/src/main/scala/filodb/query/PlanEnums.scala +++ b/query/src/main/scala/filodb/query/PlanEnums.scala @@ -82,6 +82,10 @@ object FiloFunctionId extends Enum[FiloFunctionId] { sealed abstract class AggregationOperator(override val entryName: String) extends EnumEntry +object QueryFunctionConstants { + val stdVal = "stdval" +} + object AggregationOperator extends Enum[AggregationOperator] { val values = findValues diff --git a/query/src/main/scala/filodb/query/PromCirceSupport.scala b/query/src/main/scala/filodb/query/PromCirceSupport.scala index fe44d11448..35434350ad 100644 --- a/query/src/main/scala/filodb/query/PromCirceSupport.scala +++ b/query/src/main/scala/filodb/query/PromCirceSupport.scala @@ -12,7 +12,6 @@ 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 h @ AvgSampl(t, v, c) => Json.fromValues(Seq(t.asJson, v.toString.asJson, c.toString.asJson)) case m @ MetadataSampl(v) => Json.fromValues(Seq(v.asJson)) } @@ -27,6 +26,18 @@ object PromCirceSupport { } } + implicit val decodeStdValSampl: Decoder[StdValSampl] = new Decoder[StdValSampl] { + final def apply(c: HCursor): Decoder.Result[StdValSampl] = { + for {timestamp <- c.downArray.as[Long] + stddev <- c.downArray.right.as[String] + mean <- c.downArray.right.right.as[String] + count <- c.downArray.right.right.right.as[Long] + } yield { + StdValSampl(timestamp, stddev.toDouble, mean.toDouble, count) + } + } + } + implicit val decodeFoo: Decoder[DataSampl] = new Decoder[DataSampl] { final def apply(c: HCursor): Decoder.Result[DataSampl] = { val tsResult = c.downArray.as[Long] @@ -53,7 +64,8 @@ object PromCirceSupport { } val aggregateSamples = functionName match { - case Avg.entryName => c.downField ("aggregateValues").as[List[AvgSampl]] + case Avg.entryName => c.downField ("aggregateValues").as[List[AvgSampl]] + case QueryFunctionConstants.stdVal => c.downField("aggregateValues").as[List[StdValSampl]] } for { diff --git a/query/src/main/scala/filodb/query/PromQueryResponse.scala b/query/src/main/scala/filodb/query/PromQueryResponse.scala index 439cf30b71..e8dd842cef 100644 --- a/query/src/main/scala/filodb/query/PromQueryResponse.scala +++ b/query/src/main/scala/filodb/query/PromQueryResponse.scala @@ -37,3 +37,4 @@ final case class MetadataSampl(values: Map[String, String]) extends DataSampl final case class AvgSampl(timestamp: Long, value: Double, count: Long) extends AggregateSampl +final case class StdValSampl(timestamp: Long, stddev: Double, mean: Double, count: Long) extends AggregateSampl \ No newline at end of file diff --git a/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala b/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala index d2b0c2705b..10a3f41a8d 100644 --- a/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala +++ b/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala @@ -27,15 +27,18 @@ case class PromQlRemoteExec(queryEndpoint: String, val columns= Map("histogram" -> Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), ColumnInfo("h", ColumnType.HistogramColumn)), Avg.entryName -> (defaultColumns :+ ColumnInfo("count", ColumnType.LongColumn)) , - "default" -> defaultColumns) + "default" -> defaultColumns, QueryFunctionConstants.stdVal -> (defaultColumns ++ + Seq(ColumnInfo("mean", ColumnType.DoubleColumn), ColumnInfo("count", ColumnType.LongColumn)))) val recordSchema = Map("histogram" -> SerializedRangeVector.toSchema(columns.get("histogram").get), Avg.entryName -> SerializedRangeVector.toSchema(columns.get(Avg.entryName).get), - "default" -> SerializedRangeVector.toSchema(columns.get("default").get)) + "default" -> SerializedRangeVector.toSchema(columns.get("default").get), + QueryFunctionConstants.stdVal -> SerializedRangeVector.toSchema(columns.get(QueryFunctionConstants.stdVal).get)) val resultSchema = Map("histogram" -> ResultSchema(columns.get("histogram").get, 1), Avg.entryName -> ResultSchema(columns.get(Avg.entryName).get, 1), - "default" -> ResultSchema(columns.get("default").get, 1)) + "default" -> ResultSchema(columns.get("default").get, 1), + QueryFunctionConstants.stdVal -> ResultSchema(columns.get(QueryFunctionConstants.stdVal).get, 1)) private val builder = SerializedRangeVector.newBuilder() @@ -91,7 +94,8 @@ case class PromQlRemoteExec(queryEndpoint: String, if (aggregateResponse.aggregateSampl.isEmpty) QueryResult(id, ResultSchema.empty, Seq.empty) else { aggregateResponse.aggregateSampl.head match { - case AvgSampl(timestamp, value, count) => genAvgQueryResult(data, id) + case AvgSampl(timestamp, value, count) => genAvgQueryResult(data, id) + case StdValSampl(timestamp, stddev, mean, count) => genStdValQueryResult(data, id) } } } @@ -177,8 +181,33 @@ case class PromQlRemoteExec(queryEndpoint: String, SerializedRangeVector(rv, builder, recordSchema.get(Avg.entryName).get, printTree(false)) } - // TODO: Handle stitching with verbose flag + // TODO: Handle stitching with verbose flag QueryResult(id, resultSchema.get(Avg.entryName).get, rangeVectors) } + def genStdValQueryResult(data: Data, id: String): QueryResult = { + val rangeVectors = data.result.map { d => + val rv = new RangeVector { + val row = new StdValAggTransientRow() + + override def key: RangeVectorKey = CustomRangeVectorKey(d.metric.map(m => m._1.utf8 -> m._2.utf8)) + + override def rows(): RangeVectorCursor = { + import NoCloseCursor._ + d.aggregateResponse.get.aggregateSampl.iterator.collect { case a: StdValSampl => + row.setLong(0, a.timestamp * 1000) + row.setDouble(1, a.stddev) + row.setDouble(2, a.mean) + row.setLong(3, a.count) + row + } + } + override def numRows: Option[Int] = Option(d.aggregateResponse.get.aggregateSampl.size) + } + SerializedRangeVector(rv, builder, recordSchema.get(QueryFunctionConstants.stdVal).get) + } + + // TODO: Handle stitching with verbose flag + QueryResult(id, resultSchema.get("stdval").get, rangeVectors) + } } diff --git a/query/src/test/scala/filodb/query/PromCirceSupportSpec.scala b/query/src/test/scala/filodb/query/PromCirceSupportSpec.scala index d2c76c4f21..4fd00031d5 100644 --- a/query/src/test/scala/filodb/query/PromCirceSupportSpec.scala +++ b/query/src/test/scala/filodb/query/PromCirceSupportSpec.scala @@ -2,7 +2,6 @@ package filodb.query import io.circe.parser import io.circe.generic.auto._ -//import io.circe.syntax.EncoderOps import org.scalatest.concurrent.ScalaFutures import org.scalatest.funspec.AnyFunSpec import org.scalatest.matchers.should.Matchers @@ -89,6 +88,59 @@ class PromCirceSupportSpec extends AnyFunSpec with Matchers with ScalaFutures { } } + it("should parse sttdev aggregateResponse") { + val input = """[{ + | "status": "success", + | "data": { + | "resultType": "matrix", + | "result": [ + | { + | "metric": { + | + | }, + | "aggregateResponse": { + | "aggregateValues": [ + | [ + | 1603920650, + | "NaN", + | "NaN", + | 0 + | ], + | [ + | 1603920740, + | "0.0", + | "16.068496952984738", + | 1 + | ] + | ], + | "function": "stdval" + | } + | } + | ] + | }, + | "errorType": null, + | "error": null + |}]""".stripMargin + val expectedResult =List(StdValSampl(1603920650,Double.NaN, Double.NaN, 0), + StdValSampl(1603920740,0,16.068496952984738,1) + ) + + parser.decode[List[SuccessResponse]](input) match { + case Right(successResponse) => val aggregateResponse = successResponse.head.data.result.head.aggregateResponse.get + aggregateResponse.function shouldEqual("stdval") + aggregateResponse.aggregateSampl.map(_.asInstanceOf[StdValSampl]).zip(expectedResult).foreach { + case (res, ex) => if (res.mean.isNaN) { + ex.mean.isNaN shouldEqual(true) + ex.stddev.isNaN shouldEqual true + ex.count shouldEqual(res.count) + ex.timestamp shouldEqual(res.timestamp) + } else ex shouldEqual(res) + } + case Left(ex) => println(ex) + } + } + + def parseAndValidate(input: String, expectedResult: List[DataSampl]): Unit = { parser.decode[List[DataSampl]](input) match { case Right(samples) => From 5ce3d24089f4250a8b136f6c81bfa44ce5583687 Mon Sep 17 00:00:00 2001 From: TanviBhavsar Date: Tue, 10 Nov 2020 18:45:08 -0800 Subject: [PATCH 17/33] fix build (#958) --- query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala b/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala index 10a3f41a8d..03ed7077e7 100644 --- a/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala +++ b/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala @@ -204,7 +204,7 @@ case class PromQlRemoteExec(queryEndpoint: String, } override def numRows: Option[Int] = Option(d.aggregateResponse.get.aggregateSampl.size) } - SerializedRangeVector(rv, builder, recordSchema.get(QueryFunctionConstants.stdVal).get) + SerializedRangeVector(rv, builder, recordSchema.get(QueryFunctionConstants.stdVal).get, printTree(false)) } // TODO: Handle stitching with verbose flag From ca6d7a8ebf30b02424b69626024474fb8db3d009 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Wed, 11 Nov 2020 09:47:59 -0800 Subject: [PATCH 18/33] version increase to 0.9.11 --- version.sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/version.sbt b/version.sbt index b3567421e6..f8c5770b78 100644 --- a/version.sbt +++ b/version.sbt @@ -1 +1 @@ -version in ThisBuild := "0.9.10.integration-SNAPSHOT" +version in ThisBuild := "0.9.11.integration-SNAPSHOT" From cd8686698d38758e4132f6bcc870b573ad5f0155 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Fri, 13 Nov 2020 09:34:51 -0800 Subject: [PATCH 19/33] bug(memory): Update used block metric after reclaim (#961) --- .../src/main/scala/filodb.memory/BlockManager.scala | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/memory/src/main/scala/filodb.memory/BlockManager.scala b/memory/src/main/scala/filodb.memory/BlockManager.scala index 9d027b7a10..af399b8860 100644 --- a/memory/src/main/scala/filodb.memory/BlockManager.scala +++ b/memory/src/main/scala/filodb.memory/BlockManager.scala @@ -7,7 +7,7 @@ import com.kenai.jffi.{MemoryIO, PageManager} import com.typesafe.scalalogging.StrictLogging import java.util import kamon.Kamon -import kamon.metric.Counter +import kamon.metric.{Counter, Gauge} import kamon.tag.TagSet final case class MemoryRequestException(msg: String) extends Exception(msg) @@ -377,15 +377,18 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, protected[memory] def tryReclaim(num: Int): Int = { var reclaimed = 0 - reclaimFrom(usedOdpBlocks, stats.odpBlocksReclaimedMetric) - if (reclaimed < num) reclaimFrom(usedIngestionBlocks, stats.ingestionBlocksReclaimedMetric) + reclaimFrom(usedOdpBlocks, stats.odpBlocksReclaimedMetric, stats.usedOdpBlocksMetric) + if (reclaimed < num) + reclaimFrom(usedIngestionBlocks, stats.ingestionBlocksReclaimedMetric, stats.usedIngestionBlocksMetric) // if we do not get required blocks even after reclaim call if (reclaimed < num) { logger.warn(s"$num blocks to reclaim but only reclaimed $reclaimed. " + s"usedIngestionBlocks=${usedIngestionBlocks.size} usedOdpBlocks=${usedOdpBlocks.size()}") } - def reclaimFrom(list: util.ArrayDeque[Block], reclaimedCounter: Counter): Seq[Block] = { + def reclaimFrom(list: util.ArrayDeque[Block], + reclaimedCounter: Counter, + usedBlocksStats: Gauge): Seq[Block] = { val entries = list.iterator val removed = new collection.mutable.ArrayBuffer[Block] while (entries.hasNext && reclaimed < num) { @@ -402,6 +405,7 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, reclaimed = reclaimed + 1 } } + usedBlocksStats.update(list.size()) removed } reclaimed From c09f7402d4a66da96fd81d0068f898ebf2fd41c4 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Fri, 20 Nov 2020 09:20:07 -0800 Subject: [PATCH 20/33] maint(coord): Remove priority mailbox for query actor (#963) Simplify to remove heap in actor mailbox. No evidence that actor mailbox is a query bottleneck. --- .../scala/filodb.coordinator/QueryActor.scala | 21 +++---------------- core/src/main/resources/filodb-defaults.conf | 4 ---- 2 files changed, 3 insertions(+), 22 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala b/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala index e8d5953140..3b53aa60e1 100644 --- a/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala +++ b/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala @@ -5,9 +5,7 @@ import java.util.concurrent.{ForkJoinPool, ForkJoinWorkerThread} import scala.util.control.NonFatal -import akka.actor.{ActorRef, ActorSystem, Props} -import akka.dispatch.{Envelope, UnboundedStablePriorityMailbox} -import com.typesafe.config.Config +import akka.actor.{ActorRef, Props} import kamon.Kamon import kamon.instrumentation.executor.ExecutorInstrumentation import kamon.tag.TagSet @@ -25,20 +23,6 @@ import filodb.core.store.CorruptVectorException import filodb.query._ import filodb.query.exec.ExecPlan -object QueryCommandPriority extends java.util.Comparator[Envelope] { - override def compare(o1: Envelope, o2: Envelope): Int = { - (o1.message, o2.message) match { - case (q1: QueryCommand, q2: QueryCommand) => q1.submitTime.compareTo(q2.submitTime) - case (_, _: QueryCommand) => -1 // non-query commands are admin and have higher priority - case (_: QueryCommand, _) => 1 // non-query commands are admin and have higher priority - case _ => 0 - } - } -} - -class QueryActorMailbox(settings: ActorSystem.Settings, config: Config) - extends UnboundedStablePriorityMailbox(QueryCommandPriority) - object QueryActor { final case class ThrowException(dataset: DatasetRef) @@ -46,7 +30,7 @@ object QueryActor { schemas: Schemas, shardMapFunc: => ShardMapper, earliestRawTimestampFn: => Long): Props = Props(new QueryActor(memStore, dsRef, schemas, - shardMapFunc, earliestRawTimestampFn)).withMailbox("query-actor-mailbox") + shardMapFunc, earliestRawTimestampFn)) } /** @@ -136,6 +120,7 @@ final class QueryActor(memStore: MemStore, Kamon.currentSpan().tag("query", q.getClass.getSimpleName) Kamon.currentSpan().tag("query-id", q.queryContext.queryId) val querySession = QuerySession(q.queryContext, queryConfig) + Kamon.currentSpan().mark("query-actor-received-scheduling-into-executor") q.execute(memStore, querySession)(queryScheduler) .foreach { res => FiloSchedulers.assertThreadName(QuerySchedName) diff --git a/core/src/main/resources/filodb-defaults.conf b/core/src/main/resources/filodb-defaults.conf index fec424ca82..021a10ca43 100644 --- a/core/src/main/resources/filodb-defaults.conf +++ b/core/src/main/resources/filodb-defaults.conf @@ -420,10 +420,6 @@ filodb { } } -query-actor-mailbox { - mailbox-type = "filodb.coordinator.QueryActorMailbox" -} - # Configuration for the open-source ingestion gateway gateway { # TCP Port for Influx Line Protocol incoming data From 219ca22a97a3f556dec58a427a57f5cc05f8ea8b Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Fri, 20 Nov 2020 16:15:44 -0800 Subject: [PATCH 21/33] debug(memory): Additional debugging for root causing faster-rate bug (#964) --- .../NodeCoordinatorActor.scala | 2 ++ .../filodb.memory/format/MemoryLogger.scala | 8 ++++++++ .../format/vectors/DoubleVector.scala | 16 ++++++++++++---- .../query/exec/PeriodicSamplesMapper.scala | 8 +++++--- .../query/exec/rangefn/RangeFunction.scala | 10 ++++++++-- 5 files changed, 35 insertions(+), 9 deletions(-) create mode 100644 memory/src/main/scala/filodb.memory/format/MemoryLogger.scala diff --git a/coordinator/src/main/scala/filodb.coordinator/NodeCoordinatorActor.scala b/coordinator/src/main/scala/filodb.coordinator/NodeCoordinatorActor.scala index fd5e211a5d..2dc8a3380f 100644 --- a/coordinator/src/main/scala/filodb.coordinator/NodeCoordinatorActor.scala +++ b/coordinator/src/main/scala/filodb.coordinator/NodeCoordinatorActor.scala @@ -8,6 +8,7 @@ import scala.concurrent.duration._ import akka.actor.{ActorRef, OneForOneStrategy, PoisonPill, Props, Terminated} import akka.actor.SupervisorStrategy.{Restart, Stop} import akka.event.LoggingReceive +import kamon.Kamon import net.ceedubs.ficus.Ficus._ import filodb.coordinator.client.MiscCommands @@ -195,6 +196,7 @@ private[filodb] final class NodeCoordinatorActor(metaStore: MetaStore, def queryHandlers: Receive = LoggingReceive { case q: QueryCommand => val originator = sender() + Kamon.currentSpan().mark("NodeCoordinatorActor received query") withQueryActor(originator, q.dataset) { _.tell(q, originator) } case QueryActor.ThrowException(dataset) => val originator = sender() diff --git a/memory/src/main/scala/filodb.memory/format/MemoryLogger.scala b/memory/src/main/scala/filodb.memory/format/MemoryLogger.scala new file mode 100644 index 0000000000..39094ed67e --- /dev/null +++ b/memory/src/main/scala/filodb.memory/format/MemoryLogger.scala @@ -0,0 +1,8 @@ +package filodb.memory.format + +import com.typesafe.scalalogging.{Logger, StrictLogging} + +object MemoryLogger extends StrictLogging { + protected[memory] val mLogger: Logger = logger + +} diff --git a/memory/src/main/scala/filodb.memory/format/vectors/DoubleVector.scala b/memory/src/main/scala/filodb.memory/format/vectors/DoubleVector.scala index a1808c20ea..2e8f91e2c6 100644 --- a/memory/src/main/scala/filodb.memory/format/vectors/DoubleVector.scala +++ b/memory/src/main/scala/filodb.memory/format/vectors/DoubleVector.scala @@ -341,10 +341,18 @@ extends DoubleVectorDataReader { override def correctedValue(acc2: MemoryReader, vector: BinaryVectorPtr, n: Int, correctionMeta: CorrectionMeta): Double = { assert(vector == vect && acc == acc2) - correctionMeta match { - // corrected value + any carryover correction - case DoubleCorrection(_, corr) => corrected(n) + corr - case NoCorrection => corrected(n) + try { + correctionMeta match { + // corrected value + any carryover correction + case DoubleCorrection(_, corr) => corrected(n) + corr + case NoCorrection => corrected(n) + } + } catch { case e: ArrayIndexOutOfBoundsException => + MemoryLogger.mLogger.error(s"ArrayIndexOutOfBoundsException Vector is [${toHexString(acc2, vector)}] ") + MemoryLogger.mLogger.error(s"ArrayIndexOutOfBoundsException corrected is ${corrected.mkString(",")}") + MemoryLogger.mLogger.error(s"ArrayIndexOutOfBoundsException vectorLength is ${length(acc2, vector)}") + MemoryLogger.mLogger.error(s"ArrayIndexOutOfBoundsException _drops is ${_drops}") + throw e } } diff --git a/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala b/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala index 55e51f0f25..eb97361e94 100644 --- a/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala +++ b/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala @@ -237,10 +237,12 @@ extends WrappedCursor(rv.rows()) with StrictLogging { val valReader = rv.partition.schema.data.reader(rv.valueColID, nextInfo.getValueVectorAccessor, nextInfo.getValueVectorAddr) - qLogger.error(s"addChunks Exception: info.numRows=${nextInfo.numRows} " + - s"info.endTime=${nextInfo.endTime} curWindowEnd=${wit.curWindowEnd} tsReader=$tsReader " + + qLogger.error(s"addChunks Exception: ChunkInfo=[${nextInfo.debugString}] " + + s"curWinStart=${wit.curWindowStart} curWindowEnd=${wit.curWindowEnd} tsReader=$tsReader " + s"timestampVectorLength=${tsReader.length(nextInfo.getTsVectorAccessor, nextInfo.getTsVectorAddr)} " + - s"valueVectorLength=${valReader.length(nextInfo.getValueVectorAccessor, nextInfo.getValueVectorAddr)}", e) + s"valueVectorLength=${valReader.length(nextInfo.getValueVectorAccessor, nextInfo.getValueVectorAddr)} " + + s"partition ${rv.partition.stringPartition} " + + s"start=$start end=$end step=$step", e) throw e } } diff --git a/query/src/main/scala/filodb/query/exec/rangefn/RangeFunction.scala b/query/src/main/scala/filodb/query/exec/rangefn/RangeFunction.scala index 356c03195c..cd155f6230 100644 --- a/query/src/main/scala/filodb/query/exec/rangefn/RangeFunction.scala +++ b/query/src/main/scala/filodb/query/exec/rangefn/RangeFunction.scala @@ -6,6 +6,7 @@ import filodb.core.query._ import filodb.core.store.ChunkSetInfoReader import filodb.memory.format.{vectors => bv, _} import filodb.memory.format.BinaryVector.BinaryVectorPtr +import filodb.query.Query import filodb.query.exec._ /** @@ -143,8 +144,13 @@ trait CounterChunkedRangeFunction[R <: MutableRowReader] extends ChunkedRangeFun // At least one sample is present if (startRowNum <= endRowNum) { - addTimeChunks(valueVectorAcc, valueVector, ccReader, startRowNum, endRowNum, - tsReader(tsVectorAcc, tsVector, startRowNum), tsReader(tsVectorAcc, tsVector, endRowNum)) + try { + addTimeChunks(valueVectorAcc, valueVector, ccReader, startRowNum, endRowNum, + tsReader(tsVectorAcc, tsVector, startRowNum), tsReader(tsVectorAcc, tsVector, endRowNum)) + } catch { case e: ArrayIndexOutOfBoundsException => + Query.qLogger.error(s"ArrayIndexOutOfBoundsException startRowNum=$startRowNum endRowNum=$endRowNum") + throw e + } } // Add any corrections from this chunk, pass on lastValue also to next chunk computation From 35ba8833d093ca2add3c4a61d827aafff7de0365 Mon Sep 17 00:00:00 2001 From: sherali42 <43357447+sherali42@users.noreply.github.com> Date: Tue, 1 Dec 2020 10:02:38 -0800 Subject: [PATCH 22/33] feat(core): add addititional condition to create chunks at flush interval boundary, only when enabled (#962) --- .../columnstore/OdpSpec.scala | 2 +- .../memstore/TimeSeriesPartition.scala | 38 ++++-- .../memstore/TimeSeriesShard.scala | 9 +- .../filodb.core/store/IngestionConfig.scala | 12 ++ .../src/test/scala/filodb.core/TestData.scala | 6 +- .../downsample/ShardDownsamplerSpec.scala | 2 +- .../memstore/TimeSeriesPartitionSpec.scala | 127 +++++++++++++----- .../rangefn/AggrOverTimeFunctionsSpec.scala | 9 +- .../exec/rangefn/RateFunctionsSpec.scala | 3 +- .../downsampler/chunk/BatchDownsampler.scala | 3 +- .../downsampler/DownsamplerMainSpec.scala | 15 ++- 11 files changed, 168 insertions(+), 58 deletions(-) diff --git a/cassandra/src/test/scala/filodb.cassandra/columnstore/OdpSpec.scala b/cassandra/src/test/scala/filodb.cassandra/columnstore/OdpSpec.scala index 1fa0fb8647..cc682fe197 100644 --- a/cassandra/src/test/scala/filodb.cassandra/columnstore/OdpSpec.scala +++ b/cassandra/src/test/scala/filodb.cassandra/columnstore/OdpSpec.scala @@ -82,7 +82,7 @@ class OdpSpec extends AnyFunSpec with Matchers with BeforeAndAfterAll with Scala 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.ingest( System.currentTimeMillis(), rr, offheapMem.blockMemFactory, false, Option.empty) part.switchBuffers(offheapMem.blockMemFactory, true) } val chunks = part.makeFlushChunks(offheapMem.blockMemFactory) diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala index 8e8466bd26..fc1c429e09 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala @@ -121,10 +121,19 @@ extends ChunkMap(memFactory, initMapSize) with ReadablePartition { * If ingesting a new row causes WriteBuffers to overflow, then the current chunks are encoded, a new set * of appending chunks are obtained, and we re-ingest into the new chunks. * + * createChunkAtFlushBoundary - switch buffers and create chunk when current sample's timestamp crosses flush boundary + * e.g. for a flush-interval of 1hour, if new sample falls in different hour than last sample, then switch buffers + * and create chunk. This helps in aligning chunks across Active/Active HA clusters and facilitates chunk migration + * between the clusters during disaster recovery. + * Note: Enabling this might result into creation of smaller suboptimal chunks. + * * @param ingestionTime time (as milliseconds from 1970) at the data source * @param blockHolder the BlockMemFactory to use for encoding chunks in case of WriteBuffer overflow + * @param createChunkAtFlushBoundary create time bucketed (flush-interval) chunks + * @param flushIntervalMillis flush-interval in milliseconds */ def ingest(ingestionTime: Long, row: RowReader, blockHolder: BlockMemFactory, + createChunkAtFlushBoundary: Boolean, flushIntervalMillis: Option[Long], maxChunkTime: Long = Long.MaxValue): Unit = { // NOTE: lastTime is not persisted for recovery. Thus the first sample after recovery might still be out of order. val ts = schema.timestamp(row) @@ -136,18 +145,26 @@ extends ChunkMap(memFactory, initMapSize) with ReadablePartition { val newChunk = currentChunks == nullChunks if (newChunk) initNewChunk(ts, ingestionTime) - if (ts - currentInfo.startTime > maxChunkTime) { + if(!newChunk && createChunkAtFlushBoundary + && ts/flushIntervalMillis.get != timestampOfLatestSample/flushIntervalMillis.get) { + // we have reached maximum userTime in chunk. switch buffers, start a new chunk and ingest + switchBuffersAndIngest(ingestionTime, ts, row, blockHolder, + createChunkAtFlushBoundary, flushIntervalMillis, maxChunkTime) + } else if (ts - currentInfo.startTime > maxChunkTime) { // we have reached maximum userTime in chunk. switch buffers, start a new chunk and ingest - switchBuffersAndIngest(ingestionTime, ts, row, blockHolder, maxChunkTime) + switchBuffersAndIngest(ingestionTime, ts, row, blockHolder, + createChunkAtFlushBoundary, flushIntervalMillis, maxChunkTime) } else { cforRange { 0 until schema.numDataColumns } { col => currentChunks(col).addFromReaderNoNA(row, col) match { case r: VectorTooSmall => - switchBuffersAndIngest(ingestionTime, ts, row, blockHolder, maxChunkTime) + switchBuffersAndIngest(ingestionTime, ts, row, blockHolder, + createChunkAtFlushBoundary, flushIntervalMillis, maxChunkTime) return // Different histogram bucket schema: need a new vector here case BucketSchemaMismatch => - switchBuffersAndIngest(ingestionTime, ts, row, blockHolder, maxChunkTime) + switchBuffersAndIngest(ingestionTime, ts, row, blockHolder, + createChunkAtFlushBoundary, flushIntervalMillis, maxChunkTime) return case other: AddResponse => } @@ -168,12 +185,15 @@ extends ChunkMap(memFactory, initMapSize) with ReadablePartition { userTime: Long, row: RowReader, blockHolder: BlockMemFactory, - maxChunkTime: Long): Unit = { + createChunkAtFlushBoundary: Boolean, + flushIntervalMillis: Option[Long], + maxChunkTime: Long = Long.MaxValue): Unit = { // NOTE: a very bad infinite loop is possible if switching buffers fails (if the # rows is 0) but one of the // vectors fills up. This is possible if one vector fills up but the other one does not for some reason. // So we do not call ingest again unless switcing buffers succeeds. // re-ingest every element, allocating new WriteBuffers - if (switchBuffers(blockHolder, encode = true)) { ingest(ingestionTime, row, blockHolder, maxChunkTime) } + if (switchBuffers(blockHolder, encode = true)) { ingest(ingestionTime, row, blockHolder, + createChunkAtFlushBoundary, flushIntervalMillis, maxChunkTime) } else { _log.warn("EMPTY WRITEBUFFERS when switchBuffers called! Likely a severe bug!!! " + s"Part=$stringPartition userTime=$userTime numRows=${currentInfo.numRows}") } } @@ -475,12 +495,14 @@ TimeSeriesPartition(partID, schema, partitionKey, shard, bufferPool, shardStats, _log.info(s"Creating TracingTimeSeriesPartition dataset=$ref schema=${schema.name} partId=$partID $stringPartition") - override def ingest(ingestionTime: Long, row: RowReader, blockHolder: BlockMemFactory, maxChunkTime: Long): Unit = { + override def ingest(ingestionTime: Long, row: RowReader, blockHolder: BlockMemFactory, + createChunkAtFlushBoundary: Boolean, flushIntervalMillis: Option[Long], + maxChunkTime: Long = Long.MaxValue): Unit = { val ts = row.getLong(0) _log.info(s"Ingesting dataset=$ref schema=${schema.name} shard=$shard partId=$partID $stringPartition " + s"ingestionTime=$ingestionTime ts=$ts " + (1 until schema.numDataColumns).map(row.getAny).mkString("[", ",", "]")) - super.ingest(ingestionTime, row, blockHolder, maxChunkTime) + super.ingest(ingestionTime, row, blockHolder, createChunkAtFlushBoundary, flushIntervalMillis, maxChunkTime) } override def switchBuffers(blockHolder: BlockMemFactory, encode: Boolean = false): Boolean = { diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala index aae20747d0..30bb4fc38e 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala @@ -407,12 +407,12 @@ class TimeSeriesShard(val ref: DatasetRef, // Flush groups when ingestion time is observed to cross a time boundary (typically an hour), // plus a group-specific offset. This simplifies disaster recovery -- chunks can be copied // without concern that they may overlap in time. - private val flushBoundaryMillis = storeConfig.flushInterval.toMillis + private val flushBoundaryMillis = Option(storeConfig.flushInterval.toMillis) // Defines the group-specific flush offset, to distribute the flushes around such they don't // all flush at the same time. With an hourly boundary and 60 flush groups, flushes are // scheduled once a minute. - private val flushOffsetMillis = flushBoundaryMillis / numGroups + private val flushOffsetMillis = flushBoundaryMillis.get / numGroups private[memstore] val evictedPartKeys = BloomFilter[PartKey](storeConfig.evictedPkBfCapacity, falsePositiveRate = 0.01)(new CanGenerateHashFrom[PartKey] { @@ -837,7 +837,7 @@ class TimeSeriesShard(val ref: DatasetRef, As written the code the same thing but with fewer operations. It's also a bit shorter, but you also had to read this comment... */ - if (oldTimestamp / flushBoundaryMillis != newTimestamp / flushBoundaryMillis) { + if (oldTimestamp / flushBoundaryMillis.get != newTimestamp / flushBoundaryMillis.get) { // Flush out the group before ingesting records for a new hour (by group offset). tasks += createFlushTask(prepareFlushGroup(group)) } @@ -1184,7 +1184,8 @@ class TimeSeriesShard(val ref: DatasetRef, val tsp = part.asInstanceOf[TimeSeriesPartition] brRowReader.schema = schema.ingestionSchema brRowReader.recordOffset = recordOff - tsp.ingest(ingestionTime, brRowReader, overflowBlockFactory, maxChunkTime) + tsp.ingest(ingestionTime, brRowReader, overflowBlockFactory, + storeConfig.timeAlignedChunksEnabled, flushBoundaryMillis, maxChunkTime) // Below is coded to work concurrently with logic in updateIndexWithEndTime // where we try to de-activate an active time series if (!tsp.ingesting) { diff --git a/core/src/main/scala/filodb.core/store/IngestionConfig.scala b/core/src/main/scala/filodb.core/store/IngestionConfig.scala index 5d556b0681..f5c745f7f6 100644 --- a/core/src/main/scala/filodb.core/store/IngestionConfig.scala +++ b/core/src/main/scala/filodb.core/store/IngestionConfig.scala @@ -10,6 +10,7 @@ import filodb.core.{DatasetRef, IngestionKeys} import filodb.core.downsample.DownsampleConfig final case class StoreConfig(flushInterval: FiniteDuration, + timeAlignedChunksEnabled: Boolean, diskTTLSeconds: Int, demandPagedRetentionPeriod: FiniteDuration, maxChunksSize: Int, @@ -42,6 +43,7 @@ final case class StoreConfig(flushInterval: FiniteDuration, import collection.JavaConverters._ def toConfig: Config = ConfigFactory.parseMap(Map("flush-interval" -> (flushInterval.toSeconds + "s"), + "time-aligned-chunks-enabled" -> timeAlignedChunksEnabled, "disk-time-to-live" -> (diskTTLSeconds + "s"), "demand-paged-chunk-retention-period" -> (demandPagedRetentionPeriod.toSeconds + "s"), "max-chunks-size" -> maxChunksSize, @@ -100,16 +102,26 @@ object StoreConfig { |ensure-headroom-percent = 5.0 |trace-filters = {} |metering-enabled = false + |time-aligned-chunks-enabled = false |""".stripMargin) /** Pass in the config inside the store {} */ def apply(storeConfig: Config): StoreConfig = { val config = storeConfig.withFallback(defaults) val flushInterval = config.as[FiniteDuration]("flush-interval") + + // switch buffers and create chunk when current sample's timestamp crosses flush boundary. + // e.g. for a flush-interval of 1hour, if new sample falls in different hour than last sample, then switch buffers + // and create chunk. This helps in aligning chunks across Active/Active HA clusters and facilitates chunk migration + // between the clusters during disaster recovery. + // Note: Enabling this might result into creation of smaller suboptimal chunks. + val timeAlignedChunksEnabled = config.getBoolean("time-aligned-chunks-enabled") + // maxChunkTime should atleast be length of flush interval to accommodate all data within one chunk. // better to be slightly greater so if more samples arrive within that flush period, two chunks are not created. val fallbackMaxChunkTime = (flushInterval.toMillis * 1.1).toLong.millis val maxChunkTime = config.as[Option[FiniteDuration]]("max-chunk-time").getOrElse(fallbackMaxChunkTime) StoreConfig(flushInterval, + timeAlignedChunksEnabled, config.as[FiniteDuration]("disk-time-to-live").toSeconds.toInt, config.as[FiniteDuration]("demand-paged-chunk-retention-period"), config.getInt("max-chunks-size"), diff --git a/core/src/test/scala/filodb.core/TestData.scala b/core/src/test/scala/filodb.core/TestData.scala index 4ee9238d89..f3b2a759ad 100644 --- a/core/src/test/scala/filodb.core/TestData.scala +++ b/core/src/test/scala/filodb.core/TestData.scala @@ -412,7 +412,8 @@ object MachineMetricsData { val histData = linearHistSeries(startTS, 1, pubFreq.toInt, numBuckets, infBucket).take(numSamples) val container = records(ds, histData).records val part = TimeSeriesPartitionSpec.makePart(0, ds, partKey=histPartKey, bufferPool=pool) - container.iterate(ds.ingestionSchema).foreach { row => part.ingest(0, row, histIngestBH, 1.hour.toMillis) } + container.iterate(ds.ingestionSchema).foreach { row => part.ingest(0, row, histIngestBH, + false, Option.empty, 1.hour.toMillis) } // Now flush and ingest the rest to ensure two separate chunks part.switchBuffers(histIngestBH, encode = true) (histData, RawDataRangeVector(null, part, AllChunkScan, Array(0, 3))) // select timestamp and histogram columns only @@ -426,7 +427,8 @@ object MachineMetricsData { val histData = histMax(linearHistSeries(startTS, 1, pubFreq.toInt, numBuckets)).take(numSamples) val container = records(histMaxDS, histData).records val part = TimeSeriesPartitionSpec.makePart(0, histMaxDS, partKey=histPartKey, bufferPool=histMaxBP) - container.iterate(histMaxDS.ingestionSchema).foreach { row => part.ingest(0, row, histMaxBH, 1.hour.toMillis) } + container.iterate(histMaxDS.ingestionSchema).foreach { row => part.ingest(0, row, histMaxBH, false, + Option.empty, 1.hour.toMillis) } // Now flush and ingest the rest to ensure two separate chunks part.switchBuffers(histMaxBH, encode = true) // Select timestamp, hist, max diff --git a/core/src/test/scala/filodb.core/downsample/ShardDownsamplerSpec.scala b/core/src/test/scala/filodb.core/downsample/ShardDownsamplerSpec.scala index eded32d4ec..762ec46506 100644 --- a/core/src/test/scala/filodb.core/downsample/ShardDownsamplerSpec.scala +++ b/core/src/test/scala/filodb.core/downsample/ShardDownsamplerSpec.scala @@ -74,7 +74,7 @@ class ShardDownsamplerSpec extends AnyFunSpec with Matchers with BeforeAndAfterA def timeValueRV(tuples: Seq[(Long, Double)]): RawDataRangeVector = { val part = TimeSeriesPartitionSpec.makePart(0, promDataset, partKeyOffset, bufferPool = tsBufferPool) val readers = tuples.map { case (ts, d) => TupleRowReader((Some(ts), Some(d))) } - readers.foreach { row => part.ingest(0, row, ingestBlockHolder) } + readers.foreach { row => part.ingest(0, row, ingestBlockHolder, false, Option.empty) } // Now flush and ingest the rest to ensure two separate chunks part.switchBuffers(ingestBlockHolder, encode = true) // part.encodeAndReleaseBuffers(ingestBlockHolder) diff --git a/core/src/test/scala/filodb.core/memstore/TimeSeriesPartitionSpec.scala b/core/src/test/scala/filodb.core/memstore/TimeSeriesPartitionSpec.scala index 7c1584af53..160054cdc8 100644 --- a/core/src/test/scala/filodb.core/memstore/TimeSeriesPartitionSpec.scala +++ b/core/src/test/scala/filodb.core/memstore/TimeSeriesPartitionSpec.scala @@ -1,27 +1,30 @@ package filodb.core.memstore import scala.concurrent.Future +import scala.concurrent.duration._ import com.typesafe.config.ConfigFactory import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll} import org.scalatest.concurrent.ScalaFutures +import org.scalatest.funspec.AnyFunSpec +import org.scalatest.matchers.should.Matchers import org.scalatest.time.{Millis, Seconds, Span} import filodb.core._ import filodb.core.metadata.Dataset import filodb.core.store._ import filodb.memory._ -import filodb.memory.format.UnsafeUtils -import org.scalatest.funspec.AnyFunSpec -import org.scalatest.matchers.should.Matchers +import filodb.memory.format.{RowReader, TupleRowReader, UnsafeUtils} object TimeSeriesPartitionSpec { - import MachineMetricsData._ import BinaryRegion.NativePointer + import MachineMetricsData._ val memFactory = new NativeMemoryManager(50 * 1024 * 1024) val maxChunkSize = TestData.storeConf.maxChunksSize + private val flushIntervalMillis = Option(TestData.storeConf.flushInterval.toMillis) + private val timeAlignedChunksEnabled = TestData.storeConf.timeAlignedChunksEnabled protected val myBufferPool = new WriteBufferPool(memFactory, schema1.data, TestData.storeConf) def makePart(partNo: Int, dataset: Dataset, @@ -81,7 +84,8 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { it("should be able to read immediately after ingesting one row") { part = makePart(0, dataset1) val data = singleSeriesReaders().take(5) - part.ingest(0, data(0), ingestBlockHolder) // just one row + part.ingest(0, data(0), ingestBlockHolder, timeAlignedChunksEnabled, + flushIntervalMillis = flushIntervalMillis) // just one row part.numChunks shouldEqual 1 part.appendingChunkLen shouldEqual 1 part.unflushedChunksets shouldEqual 1 @@ -95,7 +99,8 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { val data = singleSeriesReaders().take(11) val minData = data.map(_.getDouble(1)) val initTS = data(0).getLong(0) - data.take(10).zipWithIndex.foreach { case (r, i) => part.ingest(0, r, ingestBlockHolder) } + data.take(10).zipWithIndex.foreach { case (r, i) => part.ingest(0, r, ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled) } val origPoolSize = myBufferPool.poolSize @@ -117,7 +122,8 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { val blockHolder = new BlockMemFactory(blockStore, schema1.data.blockMetaSize, dummyContext) // Task needs to fully iterate over the chunks, to release the shared lock. val flushFut = Future(part.makeFlushChunks(blockHolder).toBuffer) - data.drop(10).zipWithIndex.foreach { case (r, i) => part.ingest(0, r, ingestBlockHolder) } + data.drop(10).zipWithIndex.foreach { case (r, i) => part.ingest(0, r, ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled) } val chunkSets = flushFut.futureValue // After flush, the old writebuffers should be returned to pool, but new one allocated for ingesting @@ -146,27 +152,63 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { it("should enforce user time length in each chunk") { part = makePart(0, dataset1) // user time maximum is not enforced, so just one chunk - singleSeriesReaders().take(35).foreach { r => part.ingest(0, r, ingestBlockHolder, Long.MaxValue) } + singleSeriesReaders().take(35).foreach { r => part.ingest(0, r, ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled) } part.numChunks shouldEqual 1 part = makePart(0, dataset1) // 11 samples per chunk since maxChunkTime is 10 seconds - singleSeriesReaders().take(33).foreach { r => part.ingest(0, r, ingestBlockHolder, 10000) } + singleSeriesReaders().take(33).foreach { r => part.ingest(0, r, ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled, maxChunkTime = 10000) } part.numChunks shouldEqual 3 part = makePart(0, dataset1) // 11 samples per chunk since maxChunkTime is 10 seconds - singleSeriesReaders().take(34).foreach { r => part.ingest(0, r, ingestBlockHolder, 10000) } + singleSeriesReaders().take(34).foreach { r => part.ingest(0, r, ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled, maxChunkTime = 10000) } part.numChunks shouldEqual 4 } + it("should enforce write-buffer-switching/chunk-creation at flush boundary when the functionality is enabled") { + val currentTIme = System.currentTimeMillis() + def timeAlignedSeriesReaders(): Stream[RowReader] = + singleSeriesData(initTs = currentTIme - currentTIme%(1 minutes).toMillis).map(TupleRowReader) + part = makePart(0, dataset1) + // chunk creation on crossing flush boundary is not enforced, so will create 1 chunk + timeAlignedSeriesReaders().take(70).foreach { r => part.ingest(0, r, ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled) } + part.numChunks shouldEqual 1 + + // chunk creation on crossing flush boundary is enabled with flushInterval of 1 min. + // Each chunk will contain 60 samples. + part = makePart(0, dataset1) + timeAlignedSeriesReaders().take(61).foreach { r => part.ingest(0, r, ingestBlockHolder, + flushIntervalMillis = Option((1 minutes).toMillis), createChunkAtFlushBoundary = true) } + part.numChunks shouldEqual 2 // ingesting 61 samples results into 2 chunks + + // chunk creation on crossing flush boundary is enabled with flushInterval of 1 min. + // Each chunk will contain 60 samples, + part = makePart(0, dataset1) + timeAlignedSeriesReaders().take(180).foreach { r => part.ingest(0, r, ingestBlockHolder, + flushIntervalMillis = Option((1 minutes).toMillis), createChunkAtFlushBoundary = true, maxChunkTime = Long.MaxValue) } + part.numChunks shouldEqual 3 // ingesting 180 samples results into 2 chunks + + // chunk creation on crossing flush boundary is enabled with flushInterval of 1 min. + // Each chunk will contain 60 samples, + part = makePart(0, dataset1) + timeAlignedSeriesReaders().take(200).foreach { r => part.ingest(0, r, ingestBlockHolder, + flushIntervalMillis = Option((1 minutes).toMillis), createChunkAtFlushBoundary = true, maxChunkTime = Long.MaxValue) } + part.numChunks shouldEqual 4 // ingesting 200 samples results into 4 chunks + } + it("should be able to read a time range of ingested data") { part = makePart(0, dataset1) val data = singleSeriesReaders().take(11) val initTS = data(0).getLong(0) val appendingTS = data.last.getLong(0) val minData = data.map(_.getDouble(1)) - data.take(10).foreach { r => part.ingest(0, r, ingestBlockHolder) } + data.take(10).foreach { r => part.ingest(0, r, ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled) } // First 10 rows ingested. Now flush in a separate Future while ingesting the remaining row part.switchBuffers(ingestBlockHolder) @@ -175,7 +217,8 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { val blockHolder = new BlockMemFactory(blockStore, schema1.data.blockMetaSize, dummyContext) // Task needs to fully iterate over the chunks, to release the shared lock. val flushFut = Future(part.makeFlushChunks(blockHolder).toBuffer) - data.drop(10).zipWithIndex.foreach { case (r, i) => part.ingest(0, r, ingestBlockHolder) } + data.drop(10).zipWithIndex.foreach { case (r, i) => part.ingest(0, r, ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled) } // there should be a frozen chunk of 10 records plus 1 record in currently appending chunks part.numChunks shouldEqual 2 @@ -222,7 +265,8 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { part = makePart(0, dataset1) val data = singleSeriesReaders().take(21) val minData = data.map(_.getDouble(1)) - data.take(10).zipWithIndex.foreach { case (r, i) => part.ingest(0, r, ingestBlockHolder) } + data.take(10).zipWithIndex.foreach { case (r, i) => part.ingest(0, r, ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled) } val origPoolSize = myBufferPool.poolSize @@ -232,7 +276,8 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { val blockHolder = new BlockMemFactory(blockStore, schema1.data.blockMetaSize, dummyContext) // Task needs to fully iterate over the chunks, to release the shared lock. val flushFut = Future(part.makeFlushChunks(blockHolder).toBuffer) - data.drop(10).take(6).zipWithIndex.foreach { case (r, i) => part.ingest(0, r, ingestBlockHolder) } + data.drop(10).take(6).zipWithIndex.foreach { case (r, i) => part.ingest(0, r, ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled) } val chunkSets = flushFut.futureValue // After flush, the old writebuffers should be returned to pool, but new one allocated too @@ -263,7 +308,8 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { val holder2 = new BlockMemFactory(blockStore, schema1.data.blockMetaSize, dummyContext) // Task needs to fully iterate over the chunks, to release the shared lock. val flushFut2 = Future(part.makeFlushChunks(holder2).toBuffer) - data.drop(16).zipWithIndex.foreach { case (r, i) => part.ingest(0, r, ingestBlockHolder) } + data.drop(16).zipWithIndex.foreach { case (r, i) => part.ingest(0, r, ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled) } val chunkSets2 = flushFut2.futureValue part.numChunks shouldEqual 3 @@ -283,7 +329,8 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { it("should not switch buffers and flush when current chunks are empty") { part = makePart(0, dataset1) val data = singleSeriesReaders().take(11) - data.zipWithIndex.foreach { case (r, i) => part.ingest(0, r, ingestBlockHolder) } + data.zipWithIndex.foreach { case (r, i) => part.ingest(0, r, ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled) } part.numChunks shouldEqual 1 part.appendingChunkLen shouldEqual 11 @@ -318,7 +365,8 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { makePart(partNo, dataset1) } (0 to 9).foreach { i => - data.foreach { case d => partitions(i).ingest(0, d, ingestBlockHolder) } + data.foreach { case d => partitions(i).ingest(0, d, ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled) } partitions(i).numChunks shouldEqual 1 partitions(i).appendingChunkLen shouldEqual 10 val infos = partitions(i).infos(AllChunkScan) @@ -339,7 +387,8 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { // Do this 4 more times so that we get old recycled metadata back (0 until 4).foreach { n => (0 to 9).foreach { i => - moreData.drop(n*10).take(10).foreach { case d => partitions(i).ingest(0, d, ingestBlockHolder) } + moreData.drop(n*10).take(10).foreach { case d => partitions(i).ingest(0, d, ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled) } partitions(i).appendingChunkLen shouldEqual 10 partitions(i).switchBuffers(ingestBlockHolder, true) } @@ -347,7 +396,8 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { // Now ingest again but don't switch buffers. Ensure appendingChunkLen is appropriate. (0 to 9).foreach { i => - moreData.drop(40).foreach { case d => partitions(i).ingest(0, d, ingestBlockHolder) } + moreData.drop(40).foreach { case d => partitions(i).ingest(0, d, ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled) } partitions(i).appendingChunkLen shouldEqual 10 } } @@ -358,7 +408,8 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { part = makePart(0, dataset1) val data = singleSeriesReaders().take(maxChunkSize + 10) - data.take(maxChunkSize - 10).zipWithIndex.foreach { case (r, i) => part.ingest(0, r, ingestBlockHolder) } + data.take(maxChunkSize - 10).zipWithIndex.foreach { case (r, i) => part.ingest(0, r, ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled) } part.numChunks shouldEqual 1 part.appendingChunkLen shouldEqual (maxChunkSize - 10) part.unflushedChunksets shouldEqual 1 @@ -366,7 +417,8 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { myBufferPool.poolSize shouldEqual (origPoolSize - 1) // Now ingest 20 more. Verify new chunks encoded. 10 rows after switch at 100. Verify can read everything. - data.drop(maxChunkSize - 10).zipWithIndex.foreach { case (r, i) => part.ingest(0, r, ingestBlockHolder) } + data.drop(maxChunkSize - 10).zipWithIndex.foreach { case (r, i) => part.ingest(0, r, ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled) } part.numChunks shouldEqual 2 part.appendingChunkLen shouldEqual 10 part.unflushedChunksets shouldEqual 2 @@ -393,15 +445,22 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { val minData = data.map(_.getDouble(1)) // Ingest first 5, then: 8th, 6th, 7th, 9th, 10th - data.take(5).foreach { r => part.ingest(0, r, ingestBlockHolder) } - part.ingest(0, data(7), ingestBlockHolder) - part.ingest(0, data(5), ingestBlockHolder) - part.ingest(0, data(6), ingestBlockHolder) - part.ingest(0, data(8), ingestBlockHolder) - part.ingest(0, data(9), ingestBlockHolder) + data.take(5).foreach { r => part.ingest(0, r, ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled) } + part.ingest(0, data(7), ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled) + part.ingest(0, data(5), ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled) + part.ingest(0, data(6), ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled) + part.ingest(0, data(8), ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled) + part.ingest(0, data(9), ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled) // Try ingesting old sample now at the end. Verify that end time of chunkInfo is not incorrectly changed. - part.ingest(0, data(2), ingestBlockHolder) + part.ingest(0, data(2), ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled) // 8 of first 10 ingested, 2 should be dropped. Switch buffers, and try ingesting out of order again. part.appendingChunkLen shouldEqual 8 part.infoLast.numRows shouldEqual 8 @@ -411,10 +470,13 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { part.appendingChunkLen shouldEqual 0 // Now try ingesting an old smaple again at first element of next chunk. - part.ingest(0, data(8), ingestBlockHolder) // This one should be dropped + part.ingest(0, data(8), ingestBlockHolder, flushIntervalMillis = flushIntervalMillis, + createChunkAtFlushBoundary = timeAlignedChunksEnabled) // This one should be dropped part.appendingChunkLen shouldEqual 0 - part.ingest(0, data(10), ingestBlockHolder) - part.ingest(0, data(11), ingestBlockHolder) + part.ingest(0, data(10), ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled) + part.ingest(0, data(11), ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled) // there should be a frozen chunk of 10 records plus 2 records in currently appending chunks part.numChunks shouldEqual 2 @@ -431,7 +493,8 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { val data = singleSeriesReaders().take(11) val minData = data.map(_.getDouble(1)) val initTS = data(0).getLong(0) - data.take(10).zipWithIndex.foreach { case (r, i) => part.ingest(0, r, ingestBlockHolder) } + data.take(10).zipWithIndex.foreach { case (r, i) => part.ingest(0, r, ingestBlockHolder, + flushIntervalMillis = flushIntervalMillis, createChunkAtFlushBoundary = timeAlignedChunksEnabled) } val origPoolSize = myBufferPool.poolSize 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 db7154ddf6..48a2c18612 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala @@ -154,7 +154,8 @@ trait RawDataWindowingSpec extends AnyFunSpec with Matchers with BeforeAndAfter partKey: NativePointer = defaultPartKey): RawDataRangeVector = { val part = TimeSeriesPartitionSpec.makePart(0, timeseriesDatasetWithMetric, partKey, bufferPool = tsBufferPool) val readers = tuples.map { case (ts, d) => TupleRowReader((Some(ts), Some(d))) } - readers.foreach { row => part.ingest(0, row, ingestBlockHolder) } + readers.foreach { row => part.ingest(0, row, ingestBlockHolder, createChunkAtFlushBoundary = false, + flushIntervalMillis = Option.empty) } // Now flush and ingest the rest to ensure two separate chunks part.switchBuffers(ingestBlockHolder, encode = true) // part.encodeAndReleaseBuffers(ingestBlockHolder) @@ -167,7 +168,8 @@ trait RawDataWindowingSpec extends AnyFunSpec with Matchers with BeforeAndAfter val readers = tuples.map { case (ts, d1, d2, d3, d4, d5) => TupleRowReader((Some(ts), Some(d1), Some(d2), Some(d3), Some(d4), Some(d5))) } - readers.foreach { row => part.ingest(0, row, ingestBlockHolder2) } + readers.foreach { row => part.ingest(0, row, ingestBlockHolder2, createChunkAtFlushBoundary = false, + flushIntervalMillis = Option.empty) } // Now flush and ingest the rest to ensure two separate chunks part.switchBuffers(ingestBlockHolder2, encode = true) // part.encodeAndReleaseBuffers(ingestBlockHolder) @@ -184,7 +186,8 @@ trait RawDataWindowingSpec extends AnyFunSpec with Matchers with BeforeAndAfter val part = rv.partition.asInstanceOf[TimeSeriesPartition] val startingNumChunks = part.numChunks val readers = tuples.map { case (ts, d) => TupleRowReader((Some(ts), Some(d))) } - readers.foreach { row => part.ingest(0, row, ingestBlockHolder) } + readers.foreach { row => part.ingest(0, row, ingestBlockHolder, createChunkAtFlushBoundary = false, + flushIntervalMillis = Option.empty) } part.switchBuffers(ingestBlockHolder, encode = true) part.numChunks shouldEqual (startingNumChunks + 1) } 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 4f4013e0b3..e6d7c7a269 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/RateFunctionsSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/RateFunctionsSpec.scala @@ -227,7 +227,8 @@ class RateFunctionsSpec extends RawDataWindowingSpec { val dropData = data.map(d => (d.head.asInstanceOf[Long] + 70000L) +: d.drop(1)) val container = MachineMetricsData.records(promHistDS, dropData).records val bh = MachineMetricsData.histIngestBH - container.iterate(promHistDS.ingestionSchema).foreach { row => part.ingest(0, row, bh) } + container.iterate(promHistDS.ingestionSchema).foreach { row => part.ingest(0, row, bh, + createChunkAtFlushBoundary = false, flushIntervalMillis = Option.empty) } part.switchBuffers(bh, encode = true) diff --git a/spark-jobs/src/main/scala/filodb/downsampler/chunk/BatchDownsampler.scala b/spark-jobs/src/main/scala/filodb/downsampler/chunk/BatchDownsampler.scala index 7a7ffb970f..8f1c51c30e 100644 --- a/spark-jobs/src/main/scala/filodb/downsampler/chunk/BatchDownsampler.scala +++ b/spark-jobs/src/main/scala/filodb/downsampler/chunk/BatchDownsampler.scala @@ -328,7 +328,8 @@ class BatchDownsampler(settings: DownsamplerSettings) extends Instance with Seri for {c <- dsRecordBuilder.allContainers row <- c.iterate(part.schema.ingestionSchema) } { - part.ingest(userTimeEndExclusive, row, offHeapMem.blockMemFactory) + part.ingest(userTimeEndExclusive, row, offHeapMem.blockMemFactory, createChunkAtFlushBoundary = false, + flushIntervalMillis = Option.empty) } } catch { case e: Exception => diff --git a/spark-jobs/src/test/scala/filodb/downsampler/DownsamplerMainSpec.scala b/spark-jobs/src/test/scala/filodb/downsampler/DownsamplerMainSpec.scala index 7d0ff114c6..6a749b9ced 100644 --- a/spark-jobs/src/test/scala/filodb/downsampler/DownsamplerMainSpec.scala +++ b/spark-jobs/src/test/scala/filodb/downsampler/DownsamplerMainSpec.scala @@ -132,7 +132,8 @@ class DownsamplerMainSpec extends AnyFunSpec with Matchers with BeforeAndAfterAl MachineMetricsData.records(rawDataset, rawSamples).records.foreach { case (base, offset) => val rr = new BinaryRecordRowReader(Schemas.untyped.ingestionSchema, base, offset) - part.ingest( lastSampleTime, rr, offheapMem.blockMemFactory) + part.ingest(lastSampleTime, rr, offheapMem.blockMemFactory, createChunkAtFlushBoundary = false, + flushIntervalMillis = Option.empty) } part.switchBuffers(offheapMem.blockMemFactory, true) val chunks = part.makeFlushChunks(offheapMem.blockMemFactory) @@ -174,7 +175,8 @@ class DownsamplerMainSpec extends AnyFunSpec with Matchers with BeforeAndAfterAl MachineMetricsData.records(rawDataset, rawSamples).records.foreach { case (base, offset) => val rr = new BinaryRecordRowReader(Schemas.gauge.ingestionSchema, base, offset) - part.ingest( lastSampleTime, rr, offheapMem.blockMemFactory) + part.ingest( lastSampleTime, rr, offheapMem.blockMemFactory, createChunkAtFlushBoundary = false, + flushIntervalMillis = Option.empty) } part.switchBuffers(offheapMem.blockMemFactory, true) val chunks = part.makeFlushChunks(offheapMem.blockMemFactory) @@ -214,7 +216,8 @@ class DownsamplerMainSpec extends AnyFunSpec with Matchers with BeforeAndAfterAl MachineMetricsData.records(rawDataset, rawSamples).records.foreach { case (base, offset) => val rr = new BinaryRecordRowReader(Schemas.gauge.ingestionSchema, base, offset) - part.ingest( lastSampleTime, rr, offheapMem.blockMemFactory) + part.ingest( lastSampleTime, rr, offheapMem.blockMemFactory, createChunkAtFlushBoundary = false, + flushIntervalMillis = Option.empty) } part.switchBuffers(offheapMem.blockMemFactory, true) val chunks = part.makeFlushChunks(offheapMem.blockMemFactory) @@ -260,7 +263,8 @@ class DownsamplerMainSpec extends AnyFunSpec with Matchers with BeforeAndAfterAl MachineMetricsData.records(rawDataset, rawSamples).records.foreach { case (base, offset) => val rr = new BinaryRecordRowReader(Schemas.promCounter.ingestionSchema, base, offset) - part.ingest( lastSampleTime, rr, offheapMem.blockMemFactory) + part.ingest( lastSampleTime, rr, offheapMem.blockMemFactory, createChunkAtFlushBoundary = false, + flushIntervalMillis = Option.empty) } part.switchBuffers(offheapMem.blockMemFactory, true) val chunks = part.makeFlushChunks(offheapMem.blockMemFactory) @@ -307,7 +311,8 @@ class DownsamplerMainSpec extends AnyFunSpec with Matchers with BeforeAndAfterAl MachineMetricsData.records(rawDataset, rawSamples).records.foreach { case (base, offset) => val rr = new BinaryRecordRowReader(Schemas.promHistogram.ingestionSchema, base, offset) - part.ingest( lastSampleTime, rr, offheapMem.blockMemFactory) + part.ingest( lastSampleTime, rr, offheapMem.blockMemFactory, createChunkAtFlushBoundary = false, + flushIntervalMillis = Option.empty) } part.switchBuffers(offheapMem.blockMemFactory, true) val chunks = part.makeFlushChunks(offheapMem.blockMemFactory) From 6665cbba05c55b55fa4081651086a2b551e19a9e Mon Sep 17 00:00:00 2001 From: TanviBhavsar Date: Thu, 3 Dec 2020 12:28:32 -0800 Subject: [PATCH 23/33] bug(query): "next on empty iterator" when using sum over topk result (#955) --- .../queryplanner/ShardKeyRegexPlanner.scala | 6 +- .../queryplanner/SingleClusterPlanner.scala | 9 ++- .../NodeCoordinatorActorSpec.scala | 6 +- .../client/SerializationSpec.scala | 16 ++-- .../SingleClusterPlannerSpec.scala | 3 +- .../SingleClusterPlannerSplitSpec.scala | 3 +- .../query/exec/AggrOverRangeVectors.scala | 8 +- .../exec/aggregator/AvgRowAggregator.scala | 2 +- .../exec/aggregator/CountRowAggregator.scala | 2 +- .../aggregator/CountValuesRowAggregator.scala | 2 +- .../aggregator/HistMaxSumAggregator.scala | 4 +- .../aggregator/HistSumRowAggregator.scala | 4 +- .../exec/aggregator/MaxRowAggregator.scala | 2 +- .../exec/aggregator/MinRowAggregator.scala | 2 +- .../aggregator/QuantileRowAggregator.scala | 2 +- .../query/exec/aggregator/RowAggregator.scala | 4 +- .../exec/aggregator/StddevRowAggregator.scala | 2 +- .../exec/aggregator/StdvarRowAggregator.scala | 2 +- .../exec/aggregator/SumRowAggregator.scala | 4 +- .../aggregator/TopBottomKRowAggregator.scala | 46 ++++++++--- .../query/exec/AggrOverRangeVectorsSpec.scala | 81 +++++++++++++------ 21 files changed, 143 insertions(+), 67 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlanner.scala index 568cc9f230..b6a60187dc 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlanner.scala @@ -1,7 +1,7 @@ package filodb.coordinator.queryplanner import filodb.core.metadata.{Dataset, Schemas} -import filodb.core.query.{ColumnFilter, PromQlQueryParams, QueryContext} +import filodb.core.query.{ColumnFilter, PromQlQueryParams, QueryContext, RangeParams} import filodb.query._ import filodb.query.exec._ @@ -98,7 +98,9 @@ class ShardKeyRegexPlanner(dataset: Dataset, else { val reducer = MultiPartitionReduceAggregateExec(queryContext, InProcessPlanDispatcher, execPlans.sortWith((x, y) => !x.isInstanceOf[PromQlRemoteExec]), aggregate.operator, aggregate.params) - reducer.addRangeVectorTransformer(AggregatePresenter(aggregate.operator, aggregate.params)) + val promQlQueryParams = queryContext.origQueryParams.asInstanceOf[PromQlQueryParams] + reducer.addRangeVectorTransformer(AggregatePresenter(aggregate.operator, aggregate.params, + RangeParams(promQlQueryParams.startSecs, promQlQueryParams.stepSecs, promQlQueryParams.endSecs))) reducer } PlanResult(Seq(exec)) diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala index c708e91162..590e037dbe 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala @@ -11,10 +11,10 @@ import filodb.coordinator.client.QueryCommands.StaticSpreadProvider import filodb.core.{DatasetRef, SpreadProvider} import filodb.core.binaryrecord2.RecordBuilder import filodb.core.metadata.Schemas -import filodb.core.query.{ColumnFilter, Filter, QueryConfig, QueryContext} +import filodb.core.query.{ColumnFilter, Filter, PromQlQueryParams, QueryConfig, QueryContext, RangeParams} import filodb.core.store.{AllChunkScan, ChunkScanMethod, InMemoryChunkScan, TimeRangeChunkScan, WriteBufferChunkScan} import filodb.prometheus.ast.Vectors.{PromMetricLabel, TypeLabel} -import filodb.prometheus.ast.WindowConstants +import filodb.prometheus.ast. WindowConstants import filodb.query.{exec, _} import filodb.query.exec.{LocalPartitionDistConcatExec, _} @@ -258,8 +258,11 @@ class SingleClusterPlanner(dsRef: DatasetRef, val reduceDispatcher = pickDispatcher(toReduceLevel2) val reducer = LocalPartitionReduceAggregateExec(qContext, reduceDispatcher, toReduceLevel2, lp.operator, lp.params) + val promQlQueryParams = qContext.origQueryParams.asInstanceOf[PromQlQueryParams] + if (!qContext.plannerParams.skipAggregatePresent) - reducer.addRangeVectorTransformer(AggregatePresenter(lp.operator, lp.params)) + reducer.addRangeVectorTransformer(AggregatePresenter(lp.operator, lp.params, RangeParams( + promQlQueryParams.startSecs, promQlQueryParams.stepSecs, promQlQueryParams.endSecs))) PlanResult(Seq(reducer), false) // since we have aggregated, no stitching } diff --git a/coordinator/src/test/scala/filodb.coordinator/NodeCoordinatorActorSpec.scala b/coordinator/src/test/scala/filodb.coordinator/NodeCoordinatorActorSpec.scala index 71affd04f8..d3a74a002e 100644 --- a/coordinator/src/test/scala/filodb.coordinator/NodeCoordinatorActorSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/NodeCoordinatorActorSpec.scala @@ -122,7 +122,8 @@ class NodeCoordinatorActorSpec extends ActorTest(NodeCoordinatorActorSpec.getNew val timeMinSchema = ResultSchema(Seq(ColumnInfo("timestamp", TimestampColumn), ColumnInfo("min", DoubleColumn)), 1) val countSchema = ResultSchema(Seq(ColumnInfo("timestamp", TimestampColumn), ColumnInfo("count", DoubleColumn)), 1) val valueSchema = ResultSchema(Seq(ColumnInfo("timestamp", TimestampColumn), ColumnInfo("value", DoubleColumn)), 1) - val qOpt = QueryContext(plannerParams = PlannerParams(shardOverrides = Some(Seq(0)))) + val qOpt = QueryContext(plannerParams = PlannerParams(shardOverrides = Some(Seq(0))), origQueryParams = + PromQlQueryParams("", 1000, 1, 1000)) describe("QueryActor commands and responses") { import MachineMetricsData._ @@ -275,7 +276,8 @@ class NodeCoordinatorActorSpec extends ActorTest(NodeCoordinatorActorSpec.getNew // Should return results from both shards // shard 1 - timestamps 110000 -< 130000; shard 2 - timestamps 130000 <- 1400000 - val queryOpt = QueryContext(plannerParams = PlannerParams(shardOverrides = Some(Seq(0, 1)))) + val queryOpt = QueryContext(plannerParams = PlannerParams(shardOverrides = Some(Seq(0, 1))), + origQueryParams = PromQlQueryParams("", 1000, 1, 1000)) val series2 = (2 to 4).map(n => s"Series $n").toSet.asInstanceOf[Set[Any]] val multiFilter = Seq(ColumnFilter("series", Filter.In(series2))) val q2 = LogicalPlan2Query(ref, diff --git a/coordinator/src/test/scala/filodb.coordinator/client/SerializationSpec.scala b/coordinator/src/test/scala/filodb.coordinator/client/SerializationSpec.scala index dc9d9a5ca1..3aeb63553f 100644 --- a/coordinator/src/test/scala/filodb.coordinator/client/SerializationSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/client/SerializationSpec.scala @@ -204,8 +204,8 @@ class SerializationSpec extends ActorTest(SerializationSpecConfig.getNewSystem) val windowed2 = PeriodicSeriesWithWindowing(raw2, from, 1000, to, 5000, RangeFunctionId.Rate) val summed2 = Aggregate(AggregationOperator.Sum, windowed2, Nil, Seq("job")) val logicalPlan = BinaryJoin(summed1, BinaryOperator.DIV, Cardinality.OneToOne, summed2) - val execPlan = engine.materialize(logicalPlan, QueryContext(Some(StaticSpreadProvider(SpreadChange(0, 0))), - 100)) + val execPlan = engine.materialize(logicalPlan, QueryContext(plannerParams = PlannerParams(Some(StaticSpreadProvider + (SpreadChange(0, 0))), 100), origQueryParams = PromQlQueryParams("", from/1000, 1000, to/1000))) roundTrip(execPlan) shouldEqual execPlan } @@ -224,8 +224,10 @@ class SerializationSpec extends ActorTest(SerializationSpecConfig.getNewSystem) val logicalPlan1 = Parser.queryRangeToLogicalPlan( s"""sum(rate(http_request_duration_seconds_bucket{job="prometheus",$shardKeyStr}[20s])) by (handler)""", qParams) - val execPlan1 = engine.materialize(logicalPlan1, QueryContext(Some(new StaticSpreadProvider(SpreadChange(0, 0))), - 100)) + val execPlan1 = engine.materialize(logicalPlan1, QueryContext(origQueryParams = PromQlQueryParams( + s"""sum(rate(http_request_duration_seconds_bucket{job="prometheus",$shardKeyStr}[20s])) by (handler)""", + from, 10, to), plannerParams = PlannerParams(Some(new StaticSpreadProvider(SpreadChange(0, 0))), + 100))) roundTrip(execPlan1) shouldEqual execPlan1 // scalastyle:off @@ -233,7 +235,11 @@ class SerializationSpec extends ActorTest(SerializationSpecConfig.getNewSystem) s"""sum(rate(http_request_duration_microseconds_sum{job="prometheus",$shardKeyStr}[5m])) by (handler) / sum(rate(http_request_duration_microseconds_count{job="prometheus",$shardKeyStr}[5m])) by (handler)""", qParams) // scalastyle:on - val execPlan2 = engine.materialize(logicalPlan2, QueryContext(Some(new StaticSpreadProvider(SpreadChange(0, 0))), 100)) + val execPlan2 = engine.materialize(logicalPlan2, QueryContext(origQueryParams = PromQlQueryParams( + s"""sum(rate(http_request_duration_microseconds_sum{job="prometheus",$shardKeyStr}[5m])) by (handler) / + |sum(rate(http_request_duration_microseconds_count{job="prometheus",$shardKeyStr}[5m])) by (handler)""".stripMargin, + from, 10, to), plannerParams = PlannerParams(Some(new StaticSpreadProvider(SpreadChange(0, 0))), + 100))) roundTrip(execPlan2) shouldEqual execPlan2 } diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/SingleClusterPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/SingleClusterPlannerSpec.scala index 832f9929c5..94b8478da0 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/SingleClusterPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/SingleClusterPlannerSpec.scala @@ -322,7 +322,8 @@ class SingleClusterPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture val logicalPlan = Parser.queryRangeToLogicalPlan("""sum(rate(foo{job="bar"}[3d]))""", TimeStepParams(nowSeconds, 1.minute.toSeconds, nowSeconds)) - val ep = planner.materialize(logicalPlan, QueryContext()).asInstanceOf[LocalPartitionReduceAggregateExec] + val ep = planner.materialize(logicalPlan, QueryContext(origQueryParams = PromQlQueryParams + ("""sum(rate(foo{job="bar"}[3d]))""",1000, 100, 1000))).asInstanceOf[LocalPartitionReduceAggregateExec] val psm = ep.children.head.asInstanceOf[MultiSchemaPartitionsExec] .rangeVectorTransformers.head.asInstanceOf[PeriodicSamplesMapper] psm.start shouldEqual (nowSeconds * 1000) diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/SingleClusterPlannerSplitSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/SingleClusterPlannerSplitSpec.scala index 6393fa9b9f..c6ca2bf9d8 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/SingleClusterPlannerSplitSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/SingleClusterPlannerSplitSpec.scala @@ -471,7 +471,8 @@ class SingleClusterPlannerSplitSpec extends AnyFunSpec with Matchers with ScalaF val logicalPlan = Parser.queryRangeToLogicalPlan("""sum(rate(foo{job="bar"}[3d]))""", TimeStepParams(nowSeconds, 1.minute.toSeconds, nowSeconds)) - val ep = planner.materialize(logicalPlan, QueryContext()).asInstanceOf[LocalPartitionReduceAggregateExec] + val ep = planner.materialize(logicalPlan, QueryContext(origQueryParams = PromQlQueryParams + ("""sum(rate(foo{job="bar"}[3d]))""",1000, 100, 1000))).asInstanceOf[LocalPartitionReduceAggregateExec] val psm = ep.children.head.asInstanceOf[MultiSchemaPartitionsExec] .rangeVectorTransformers.head.asInstanceOf[PeriodicSamplesMapper] psm.start shouldEqual (nowSeconds * 1000) diff --git a/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala b/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala index 161f255746..63b0e209a6 100644 --- a/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala +++ b/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala @@ -120,6 +120,7 @@ final case class AggregateMapReduce(aggrOp: AggregationOperator, final case class AggregatePresenter(aggrOp: AggregationOperator, aggrParams: Seq[Any], + rangeParams: RangeParams, funcParams: Seq[FuncArgs] = Nil) extends RangeVectorTransformer { protected[exec] def args: String = s"aggrOp=$aggrOp, aggrParams=$aggrParams" @@ -130,7 +131,7 @@ final case class AggregatePresenter(aggrOp: AggregationOperator, sourceSchema: ResultSchema, paramResponse: Seq[Observable[ScalarRangeVector]]): Observable[RangeVector] = { val aggregator = RowAggregator(aggrOp, aggrParams, sourceSchema) - RangeVectorAggregator.present(aggregator, source, limit) + RangeVectorAggregator.present(aggregator, source, limit, rangeParams) } override def schema(source: ResultSchema): ResultSchema = { @@ -184,8 +185,9 @@ object RangeVectorAggregator extends StrictLogging { */ def present(aggregator: RowAggregator, source: Observable[RangeVector], - limit: Int): Observable[RangeVector] = { - source.flatMap(rv => Observable.fromIterable(aggregator.present(rv, limit))) + limit: Int, + rangeParams: RangeParams): Observable[RangeVector] = { + source.flatMap(rv => Observable.fromIterable(aggregator.present(rv, limit, rangeParams))) } private def mapReduceInternal(rvs: List[RangeVector], diff --git a/query/src/main/scala/filodb/query/exec/aggregator/AvgRowAggregator.scala b/query/src/main/scala/filodb/query/exec/aggregator/AvgRowAggregator.scala index 499bae88bb..8ea6894f00 100644 --- a/query/src/main/scala/filodb/query/exec/aggregator/AvgRowAggregator.scala +++ b/query/src/main/scala/filodb/query/exec/aggregator/AvgRowAggregator.scala @@ -45,7 +45,7 @@ object AvgRowAggregator extends RowAggregator { acc } // ignore last count column. we rely on schema change - def present(aggRangeVector: RangeVector, limit: Int): Seq[RangeVector] = Seq(aggRangeVector) + def present(aggRangeVector: RangeVector, limit: Int, rangeParams: RangeParams): Seq[RangeVector] = Seq(aggRangeVector) def reductionSchema(source: ResultSchema): ResultSchema = { source.copy(columns = source.columns :+ ColumnInfo("count", ColumnType.LongColumn)) } diff --git a/query/src/main/scala/filodb/query/exec/aggregator/CountRowAggregator.scala b/query/src/main/scala/filodb/query/exec/aggregator/CountRowAggregator.scala index d684c0be47..9bb3899fd7 100644 --- a/query/src/main/scala/filodb/query/exec/aggregator/CountRowAggregator.scala +++ b/query/src/main/scala/filodb/query/exec/aggregator/CountRowAggregator.scala @@ -34,7 +34,7 @@ abstract class CountRowAggregator extends RowAggregator { acc.count += aggRes.getDouble(1) acc } - def present(aggRangeVector: RangeVector, limit: Int): Seq[RangeVector] = Seq(aggRangeVector) + def present(aggRangeVector: RangeVector, limit: Int, rangeParams: RangeParams): Seq[RangeVector] = Seq(aggRangeVector) def reductionSchema(source: ResultSchema): ResultSchema = source def presentationSchema(reductionSchema: ResultSchema): ResultSchema = reductionSchema } 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 6cdd4704fd..1b64d62a49 100644 --- a/query/src/main/scala/filodb/query/exec/aggregator/CountValuesRowAggregator.scala +++ b/query/src/main/scala/filodb/query/exec/aggregator/CountValuesRowAggregator.scala @@ -81,7 +81,7 @@ class CountValuesRowAggregator(label: String, limit: Int = 1000) extends RowAggr acc } - def present(aggRangeVector: RangeVector, limit: Int): Seq[RangeVector] = { + def present(aggRangeVector: RangeVector, limit: Int, rangeParams: RangeParams): Seq[RangeVector] = { val colSchema = Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), ColumnInfo("value", ColumnType.DoubleColumn)) val recSchema = SerializedRangeVector.toSchema(colSchema) diff --git a/query/src/main/scala/filodb/query/exec/aggregator/HistMaxSumAggregator.scala b/query/src/main/scala/filodb/query/exec/aggregator/HistMaxSumAggregator.scala index 8d89465bfc..e81b9a143a 100644 --- a/query/src/main/scala/filodb/query/exec/aggregator/HistMaxSumAggregator.scala +++ b/query/src/main/scala/filodb/query/exec/aggregator/HistMaxSumAggregator.scala @@ -1,6 +1,6 @@ package filodb.query.exec.aggregator -import filodb.core.query.{MutableRowReader, RangeVector, RangeVectorKey, ResultSchema, TransientHistMaxRow} +import filodb.core.query.{MutableRowReader, RangeParams, RangeVector, RangeVectorKey, ResultSchema, TransientHistMaxRow} import filodb.memory.format.RowReader object HistMaxSumAggregator extends RowAggregator { @@ -29,7 +29,7 @@ object HistMaxSumAggregator extends RowAggregator { acc.m = if (acc.m.isNaN) aggRes.getDouble(2) else Math.max(acc.m, aggRes.getDouble(2)) acc } - def present(aggRangeVector: RangeVector, limit: Int): Seq[RangeVector] = Seq(aggRangeVector) + def present(aggRangeVector: RangeVector, limit: Int, rangeParams: RangeParams): Seq[RangeVector] = Seq(aggRangeVector) def reductionSchema(source: ResultSchema): ResultSchema = source def presentationSchema(reductionSchema: ResultSchema): ResultSchema = reductionSchema } diff --git a/query/src/main/scala/filodb/query/exec/aggregator/HistSumRowAggregator.scala b/query/src/main/scala/filodb/query/exec/aggregator/HistSumRowAggregator.scala index 307fe5380a..fb6920a7bc 100644 --- a/query/src/main/scala/filodb/query/exec/aggregator/HistSumRowAggregator.scala +++ b/query/src/main/scala/filodb/query/exec/aggregator/HistSumRowAggregator.scala @@ -1,6 +1,6 @@ package filodb.query.exec.aggregator -import filodb.core.query.{MutableRowReader, RangeVector, RangeVectorKey, ResultSchema, TransientHistRow} +import filodb.core.query.{MutableRowReader, RangeParams, RangeVector, RangeVectorKey, ResultSchema, TransientHistRow} import filodb.memory.format.RowReader object HistSumRowAggregator extends RowAggregator { @@ -27,7 +27,7 @@ object HistSumRowAggregator extends RowAggregator { } acc } - def present(aggRangeVector: RangeVector, limit: Int): Seq[RangeVector] = Seq(aggRangeVector) + def present(aggRangeVector: RangeVector, limit: Int, rangeParams: RangeParams): Seq[RangeVector] = Seq(aggRangeVector) def reductionSchema(source: ResultSchema): ResultSchema = source def presentationSchema(reductionSchema: ResultSchema): ResultSchema = reductionSchema } \ No newline at end of file diff --git a/query/src/main/scala/filodb/query/exec/aggregator/MaxRowAggregator.scala b/query/src/main/scala/filodb/query/exec/aggregator/MaxRowAggregator.scala index 2799066ef1..fae8f0b299 100644 --- a/query/src/main/scala/filodb/query/exec/aggregator/MaxRowAggregator.scala +++ b/query/src/main/scala/filodb/query/exec/aggregator/MaxRowAggregator.scala @@ -27,7 +27,7 @@ object MaxRowAggregator extends RowAggregator { } acc } - def present(aggRangeVector: RangeVector, limit: Int): Seq[RangeVector] = Seq(aggRangeVector) + def present(aggRangeVector: RangeVector, limit: Int, rangeParams: RangeParams): Seq[RangeVector] = Seq(aggRangeVector) def reductionSchema(source: ResultSchema): ResultSchema = source def presentationSchema(reductionSchema: ResultSchema): ResultSchema = reductionSchema } diff --git a/query/src/main/scala/filodb/query/exec/aggregator/MinRowAggregator.scala b/query/src/main/scala/filodb/query/exec/aggregator/MinRowAggregator.scala index d41764a01e..d5914aa6b6 100644 --- a/query/src/main/scala/filodb/query/exec/aggregator/MinRowAggregator.scala +++ b/query/src/main/scala/filodb/query/exec/aggregator/MinRowAggregator.scala @@ -28,7 +28,7 @@ object MinRowAggregator extends RowAggregator { } acc } - def present(aggRangeVector: RangeVector, limit: Int): Seq[RangeVector] = Seq(aggRangeVector) + def present(aggRangeVector: RangeVector, limit: Int, rangeParams: RangeParams): Seq[RangeVector] = Seq(aggRangeVector) def reductionSchema(source: ResultSchema): ResultSchema = source def presentationSchema(reductionSchema: ResultSchema): ResultSchema = reductionSchema } 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 df7fb687ee..a3ae73ffa7 100644 --- a/query/src/main/scala/filodb/query/exec/aggregator/QuantileRowAggregator.scala +++ b/query/src/main/scala/filodb/query/exec/aggregator/QuantileRowAggregator.scala @@ -63,7 +63,7 @@ class QuantileRowAggregator(q: Double) extends RowAggregator { acc } - def present(aggRangeVector: RangeVector, limit: Int): Seq[RangeVector] = { + def present(aggRangeVector: RangeVector, limit: Int, rangeParams: RangeParams): Seq[RangeVector] = { val mutRow = new TransientRow() val result = aggRangeVector.rows.mapRow { r => val qVal = ArrayDigest.fromBytes(r.getBuffer(1)).quantile(q) diff --git a/query/src/main/scala/filodb/query/exec/aggregator/RowAggregator.scala b/query/src/main/scala/filodb/query/exec/aggregator/RowAggregator.scala index 23c258fc74..470d3334fc 100644 --- a/query/src/main/scala/filodb/query/exec/aggregator/RowAggregator.scala +++ b/query/src/main/scala/filodb/query/exec/aggregator/RowAggregator.scala @@ -1,7 +1,7 @@ package filodb.query.exec.aggregator import filodb.core.metadata.Column.ColumnType -import filodb.core.query.{MutableRowReader, RangeVector, RangeVectorKey, ResultSchema} +import filodb.core.query.{MutableRowReader, RangeParams, RangeVector, RangeVectorKey, ResultSchema} import filodb.memory.format.RowReader import filodb.query.AggregationOperator import filodb.query.AggregationOperator._ @@ -97,7 +97,7 @@ trait RowAggregator { * Apply limit only on iterators that are NOT lazy and need to be * materialized. */ - def present(aggRangeVector: RangeVector, limit: Int): Seq[RangeVector] + def present(aggRangeVector: RangeVector, limit: Int, rangeParams: RangeParams): Seq[RangeVector] /** * Schema of the RowReader returned by toRowReader diff --git a/query/src/main/scala/filodb/query/exec/aggregator/StddevRowAggregator.scala b/query/src/main/scala/filodb/query/exec/aggregator/StddevRowAggregator.scala index c19f7ec10e..8cffe29ddc 100644 --- a/query/src/main/scala/filodb/query/exec/aggregator/StddevRowAggregator.scala +++ b/query/src/main/scala/filodb/query/exec/aggregator/StddevRowAggregator.scala @@ -57,7 +57,7 @@ object StddevRowAggregator extends RowAggregator { acc } // ignore last two column. we rely on schema change - def present(aggRangeVector: RangeVector, limit: Int): Seq[RangeVector] = Seq(aggRangeVector) + def present(aggRangeVector: RangeVector, limit: Int, rangeParams: RangeParams): Seq[RangeVector] = Seq(aggRangeVector) def reductionSchema(source: ResultSchema): ResultSchema = { source.copy(source.columns :+ ColumnInfo("mean", ColumnType.DoubleColumn) :+ ColumnInfo("count", ColumnType.LongColumn)) diff --git a/query/src/main/scala/filodb/query/exec/aggregator/StdvarRowAggregator.scala b/query/src/main/scala/filodb/query/exec/aggregator/StdvarRowAggregator.scala index 4360873964..43fa7476f1 100644 --- a/query/src/main/scala/filodb/query/exec/aggregator/StdvarRowAggregator.scala +++ b/query/src/main/scala/filodb/query/exec/aggregator/StdvarRowAggregator.scala @@ -71,7 +71,7 @@ object StdvarRowAggregator extends RowAggregator { acc } // ignore last two column. we rely on schema change - def present(aggRangeVector: RangeVector, limit: Int): Seq[RangeVector] = Seq(aggRangeVector) + def present(aggRangeVector: RangeVector, limit: Int, rangeParams: RangeParams): Seq[RangeVector] = Seq(aggRangeVector) def reductionSchema(source: ResultSchema): ResultSchema = { source.copy(source.columns :+ ColumnInfo("mean", ColumnType.DoubleColumn) :+ ColumnInfo("count", ColumnType.LongColumn)) diff --git a/query/src/main/scala/filodb/query/exec/aggregator/SumRowAggregator.scala b/query/src/main/scala/filodb/query/exec/aggregator/SumRowAggregator.scala index 06ea90d089..617f9d99f6 100644 --- a/query/src/main/scala/filodb/query/exec/aggregator/SumRowAggregator.scala +++ b/query/src/main/scala/filodb/query/exec/aggregator/SumRowAggregator.scala @@ -1,6 +1,6 @@ package filodb.query.exec.aggregator -import filodb.core.query.{MutableRowReader, RangeVector, RangeVectorKey, ResultSchema, TransientRow} +import filodb.core.query.{MutableRowReader, RangeParams, RangeVector, RangeVectorKey, ResultSchema, TransientRow} import filodb.memory.format.RowReader /** * Map: Every sample is mapped to itself @@ -26,7 +26,7 @@ object SumRowAggregator extends RowAggregator { } acc } - def present(aggRangeVector: RangeVector, limit: Int): Seq[RangeVector] = Seq(aggRangeVector) + def present(aggRangeVector: RangeVector, limit: Int, rangeParams: RangeParams): Seq[RangeVector] = Seq(aggRangeVector) def reductionSchema(source: ResultSchema): ResultSchema = source def presentationSchema(reductionSchema: ResultSchema): ResultSchema = reductionSchema } \ No newline at end of file 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 b1a32abe99..3aca39ebe5 100644 --- a/query/src/main/scala/filodb/query/exec/aggregator/TopBottomKRowAggregator.scala +++ b/query/src/main/scala/filodb/query/exec/aggregator/TopBottomKRowAggregator.scala @@ -1,6 +1,7 @@ package filodb.query.exec.aggregator import scala.collection.mutable +import scala.collection.mutable.ListBuffer import filodb.core.binaryrecord2.RecordBuilder import filodb.core.memstore.FiloSchedulers @@ -24,6 +25,10 @@ class TopBottomKRowAggregator(k: Int, bottomK: Boolean) extends RowAggregator { private val rvkStringCache = mutable.HashMap[RangeVectorKey, ZeroCopyUTF8String]() case class RVKeyAndValue(rvk: ZeroCopyUTF8String, value: Double) + val colSchema = Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), + ColumnInfo("value", ColumnType.DoubleColumn)) + val recSchema = SerializedRangeVector.toSchema(colSchema) + class TopKHolder(var timestamp: Long = 0L) extends AggregateHolder { val valueOrdering = Ordering.by[RVKeyAndValue, Double](kr => kr.value) implicit val ordering = if (bottomK) valueOrdering else valueOrdering.reverse @@ -56,7 +61,7 @@ class TopBottomKRowAggregator(k: Int, bottomK: Boolean) extends RowAggregator { def map(rvk: RangeVectorKey, item: RowReader, mapInto: MutableRowReader): RowReader = { val rvkString = rvkStringCache.getOrElseUpdate(rvk, CustomRangeVectorKey.toZcUtf8(rvk)) mapInto.setLong(0, item.getLong(0)) - // TODO: Use setBlob instead of setString once RowReeder has the support for blob + // TODO: Use setBlob instead of setString once RowReader has the support for blob mapInto.setString(1, rvkString) mapInto.setDouble(2, item.getDouble(1)) var i = 3 @@ -81,28 +86,47 @@ class TopBottomKRowAggregator(k: Int, bottomK: Boolean) extends RowAggregator { acc } - def present(aggRangeVector: RangeVector, limit: Int): Seq[RangeVector] = { - val colSchema = Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), - ColumnInfo("value", ColumnType.DoubleColumn)) - val recSchema = SerializedRangeVector.toSchema(colSchema) + private def addRecordToBuilder(builder: RecordBuilder, timeStampMs: Long, value: Double): Unit = { + builder.startNewRecord(recSchema) + builder.addLong(timeStampMs) + builder.addDouble(value) + builder.endRecord() + } + + /** + Create new builder and add NaN till current time + */ + private def createBuilder(rangeParams: RangeParams, currentTime: Long): RecordBuilder= { + val builder = SerializedRangeVector.newBuilder(); + for (t <- rangeParams.startSecs to (currentTime - rangeParams.stepSecs) by rangeParams.stepSecs) { + addRecordToBuilder(builder, t * 1000, Double.NaN) + } + builder + } + + def present(aggRangeVector: RangeVector, limit: Int, rangeParams: RangeParams): Seq[RangeVector] = { val resRvs = mutable.Map[RangeVectorKey, RecordBuilder]() try { FiloSchedulers.assertThreadName(QuerySchedName) ChunkMap.validateNoSharedLocks(s"TopkQuery-$k-$bottomK") // We limit the results wherever it is materialized first. So it is done here. - aggRangeVector.rows.take(limit).foreach { row => + val rows = aggRangeVector.rows.take(limit) + for (t <- rangeParams.startSecs to rangeParams.endSecs by rangeParams.stepSecs) { + val rvkSeen = new ListBuffer[RangeVectorKey] + val row = rows.next() var i = 1 while (row.notNull(i)) { if (row.filoUTF8String(i) != CustomRangeVectorKey.emptyAsZcUtf8) { val rvk = CustomRangeVectorKey.fromZcUtf8(row.filoUTF8String(i)) - val builder = resRvs.getOrElseUpdate(rvk, SerializedRangeVector.newBuilder()) - builder.startNewRecord(recSchema) - builder.addLong(row.getLong(0)) - builder.addDouble(row.getDouble(i + 1)) - builder.endRecord() + rvkSeen += rvk + val builder = resRvs.getOrElseUpdate(rvk, createBuilder(rangeParams, t)) + addRecordToBuilder(builder, t, row.getDouble(i + 1)) } i += 2 } + resRvs.keySet.foreach { rvs => + if (!rvkSeen.contains(rvs)) addRecordToBuilder(resRvs.get(rvs).get, t * 1000, Double.NaN) + } } } finally { aggRangeVector.rows().close() diff --git a/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala b/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala index 99c2c109dd..be5d8d190e 100644 --- a/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala +++ b/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala @@ -41,6 +41,8 @@ class AggrOverRangeVectorsSpec extends RawDataWindowingSpec with ScalaFutures { override def rows(): RangeVectorCursor = data.iterator }) + val rangeParams = RangeParams(0, 1, 0) + // Sum val agg1 = RowAggregator(AggregationOperator.Sum, Nil, tvSchema) val resultObs = RangeVectorAggregator.mapReduce(agg1, false, Observable.fromIterable(samples), noGrouping) @@ -111,7 +113,7 @@ class AggrOverRangeVectorsSpec extends RawDataWindowingSpec with ScalaFutures { val agg7 = RowAggregator(AggregationOperator.Quantile, Seq(0.70), tvSchema) val resultObs7a = RangeVectorAggregator.mapReduce(agg7, false, Observable.fromIterable(samples), noGrouping) val resultObs7 = RangeVectorAggregator.mapReduce(agg7, true, resultObs7a, rv=>rv.key) - val resultObs7b = RangeVectorAggregator.present(agg7, resultObs7, 1000) + val resultObs7b = RangeVectorAggregator.present(agg7, resultObs7, 1000, rangeParams) val result7 = resultObs7b.toListL.runAsync.futureValue result7.size shouldEqual 1 result7(0).key shouldEqual noKey @@ -166,17 +168,22 @@ class AggrOverRangeVectorsSpec extends RawDataWindowingSpec with ScalaFutures { val ignoreKey = CustomRangeVectorKey( Map("ignore".utf8 -> "ignore".utf8)) + val ignoreKey2 = CustomRangeVectorKey( + Map("ignore2".utf8 -> "ignore2".utf8)) + val noKey = CustomRangeVectorKey(Map.empty) def noGrouping(rv: RangeVector): RangeVectorKey = noKey it ("should ignore NaN while aggregating") { val samples: Array[RangeVector] = Array( - toRv(Seq((1L, Double.NaN), (2L, 5.6d))), - toRv(Seq((1L, 4.6d), (2L, 4.4d))), - toRv(Seq((1L, 2.1d), (2L, 5.4d))) + toRv(Seq((1000L, Double.NaN), (2000L, 5.6d))), + toRv(Seq((1000L, 4.6d), (2000L, 4.4d))), + toRv(Seq((1000L, 2.1d), (2000L, 5.4d))) ) + val rangeParams = RangeParams(1,1,2) + // Sum val agg1 = RowAggregator(AggregationOperator.Sum, Nil, tvSchema) val resultObs = RangeVectorAggregator.mapReduce(agg1, false, Observable.fromIterable(samples), noGrouping) @@ -214,8 +221,8 @@ class AggrOverRangeVectorsSpec extends RawDataWindowingSpec with ScalaFutures { // BottomK val agg5 = RowAggregator(AggregationOperator.BottomK, Seq(2.0), tvSchema) val resultObs5a = RangeVectorAggregator.mapReduce(agg5, false, Observable.fromIterable(samples), noGrouping) - val resultObs5 = RangeVectorAggregator.mapReduce(agg5, true, resultObs5a, rv=>rv.key) - val resultObs5b = RangeVectorAggregator.present(agg5, resultObs5, 1000) + val resultObs5 = RangeVectorAggregator.mapReduce(agg5,true, resultObs5a, rv=>rv.key) + val resultObs5b = RangeVectorAggregator.present(agg5, resultObs5, 1000, rangeParams) val result5 = resultObs5.toListL.runAsync.futureValue result5.size shouldEqual 1 result5(0).key shouldEqual noKey @@ -230,7 +237,7 @@ class AggrOverRangeVectorsSpec extends RawDataWindowingSpec with ScalaFutures { val agg6 = RowAggregator(AggregationOperator.TopK, Seq(2.0), tvSchema) val resultObs6a = RangeVectorAggregator.mapReduce(agg6, false, Observable.fromIterable(samples), noGrouping) val resultObs6 = RangeVectorAggregator.mapReduce(agg6, true, resultObs6a, rv=>rv.key) - val resultObs6b = RangeVectorAggregator.present(agg6, resultObs6, 1000) + val resultObs6b = RangeVectorAggregator.present(agg6, resultObs6, 1000, rangeParams) val result6 = resultObs6.toListL.runAsync.futureValue result6.size shouldEqual 1 result6(0).key shouldEqual noKey @@ -245,7 +252,7 @@ class AggrOverRangeVectorsSpec extends RawDataWindowingSpec with ScalaFutures { val agg7 = RowAggregator(AggregationOperator.Quantile, Seq(0.5), tvSchema) val resultObs7a = RangeVectorAggregator.mapReduce(agg7, false, Observable.fromIterable(samples), noGrouping) val resultObs7 = RangeVectorAggregator.mapReduce(agg7, true, resultObs7a, rv=>rv.key) - val resultObs7b = RangeVectorAggregator.present(agg7, resultObs7, 1000) + val resultObs7b = RangeVectorAggregator.present(agg7, resultObs7, 1000, rangeParams) val result7 = resultObs7b.toListL.runAsync.futureValue result7.size shouldEqual 1 result7(0).key shouldEqual noKey @@ -289,16 +296,16 @@ class AggrOverRangeVectorsSpec extends RawDataWindowingSpec with ScalaFutures { val builder = SerializedRangeVector.newBuilder() val srv = SerializedRangeVector(result7(0), builder, recSchema, "AggrOverRangeVectorsSpec") - val resultObs7b = RangeVectorAggregator.present(agg7, Observable.now(srv), 1000) + val resultObs7b = RangeVectorAggregator.present(agg7, Observable.now(srv), 1000, RangeParams(0,1,0)) val finalResult = resultObs7b.toListL.runAsync.futureValue compareIter(finalResult(0).rows.map(_.getDouble(1)), Seq(3.35d, 5.4d).iterator) } - private def toRv(samples: Seq[(Long, Double)]): RangeVector = { + private def toRv(samples: Seq[(Long, Double)], rangeVectorKey: RangeVectorKey = ignoreKey): RangeVector = { new RangeVector { import NoCloseCursor._ - override def key: RangeVectorKey = ignoreKey + override def key: RangeVectorKey = rangeVectorKey override def rows(): RangeVectorCursor = samples.map(r => new TransientRow(r._1, r._2)).iterator } } @@ -359,9 +366,9 @@ class AggrOverRangeVectorsSpec extends RawDataWindowingSpec with ScalaFutures { it("should return NaN when all values are NaN for a timestamp ") { val samples: Array[RangeVector] = Array( - toRv(Seq((1L, Double.NaN), (2L, 5.6d))), - toRv(Seq((1L, Double.NaN), (2L, 4.4d))), - toRv(Seq((1L, Double.NaN), (2L, 5.4d))) + toRv(Seq((1000L, Double.NaN), (2000L, 5.6d))), + toRv(Seq((1000L, Double.NaN), (2000L, 4.4d))), + toRv(Seq((1000L, Double.NaN), (2000L, 5.4d))) ) // Sum @@ -402,7 +409,7 @@ class AggrOverRangeVectorsSpec extends RawDataWindowingSpec with ScalaFutures { val agg5 = RowAggregator(AggregationOperator.BottomK, Seq(2.0), tvSchema) val resultObs5a = RangeVectorAggregator.mapReduce(agg5, false, Observable.fromIterable(samples), noGrouping) val resultObs5 = RangeVectorAggregator.mapReduce(agg5, true, resultObs5a, rv=>rv.key) - val resultObs5b = RangeVectorAggregator.present(agg5, resultObs5, 1000) + val resultObs5b = RangeVectorAggregator.present(agg5, resultObs5, 1000, RangeParams(1,1,2)) val result5 = resultObs5.toListL.runAsync.futureValue result5.size shouldEqual 1 result5(0).key shouldEqual noKey @@ -412,14 +419,14 @@ class AggrOverRangeVectorsSpec extends RawDataWindowingSpec with ScalaFutures { val result5b = resultObs5b.toListL.runAsync.futureValue result5b.size shouldEqual 1 result5b(0).key shouldEqual ignoreKey - // present removes the range vector which has all values as Double.Max - compareIter(result5b(0).rows.map(_.getDouble(1)), Seq(5.4d, 4.4d).iterator) + + compareIter(result5b(0).rows.map(_.getDouble(1)), Seq(Double.NaN, 5.4d, 4.4d).iterator) // TopK val agg6 = RowAggregator(AggregationOperator.TopK, Seq(2.0), tvSchema) val resultObs6a = RangeVectorAggregator.mapReduce(agg6, false, Observable.fromIterable(samples), noGrouping) val resultObs6 = RangeVectorAggregator.mapReduce(agg6, true, resultObs6a, rv=>rv.key) - val resultObs6b = RangeVectorAggregator.present(agg6, resultObs6, 1000) + val resultObs6b = RangeVectorAggregator.present(agg6, resultObs6, 1000, RangeParams(1,1,2)) val result6 = resultObs6.toListL.runAsync.futureValue result6.size shouldEqual 1 result6(0).key shouldEqual noKey @@ -428,7 +435,7 @@ class AggrOverRangeVectorsSpec extends RawDataWindowingSpec with ScalaFutures { val result6b = resultObs6b.toListL.runAsync.futureValue result6b.size shouldEqual 1 result6b(0).key shouldEqual ignoreKey - compareIter(result6b(0).rows.map(_.getDouble(1)), Seq(5.4d,5.6d).iterator) + compareIter(result6b(0).rows.map(_.getDouble(1)), Seq(Double.NaN, 5.4d, 5.6d).iterator) // Stdvar val agg8 = RowAggregator(AggregationOperator.Stdvar, Nil, tvSchema) @@ -453,14 +460,14 @@ class AggrOverRangeVectorsSpec extends RawDataWindowingSpec with ScalaFutures { // The value before NaN should not get carried over. Topk result for timestamp 1556744173L should have Double.NaN val samples: Array[RangeVector] = Array( - toRv(Seq((1556744143L, 42d), (1556744158L, 42d),(1556744173L, Double.NaN))) + toRv(Seq((1556744143L, 42d), (1556745158L, 42d),(1556745173L, Double.NaN))) ) val agg6 = RowAggregator(AggregationOperator.TopK, Seq(5.0), tvSchema) val resultObs6a = RangeVectorAggregator.mapReduce(agg6, false, Observable.fromIterable(samples), noGrouping) val resultObs6 = RangeVectorAggregator.mapReduce(agg6, true, resultObs6a, rv=>rv .key) - val resultObs6b = RangeVectorAggregator.present(agg6, resultObs6, 1000) + val resultObs6b = RangeVectorAggregator.present(agg6, resultObs6, 1000, RangeParams(1556744,1,1556745)) val result6 = resultObs6.toListL.runAsync.futureValue result6(0).key shouldEqual noKey val result6b = resultObs6b.toListL.runAsync.futureValue @@ -518,7 +525,35 @@ class AggrOverRangeVectorsSpec extends RawDataWindowingSpec with ScalaFutures { result(0).rows.map(_.getDouble(1)).toList shouldEqual counts } - it("should sum and compute max of histogram & max RVs") { + it ("should add NaN in topK") { + + val samples: Array[RangeVector] = Array( + toRv(Seq((1000L, Double.NaN), (2000L, 5.1), (3000L, Double.NaN), (4000L, 5.6d), (5000L, 4.0d), (6000L, + Double.NaN))), + toRv(Seq((1000L, Double.NaN), (2000L, 5.0), (3000L, Double.NaN), (4000L, 5.7d), (5000L, 4.4d), + (6000L, Double.NaN)), ignoreKey2) + ) + + val rangeParams = RangeParams(1,1,6) + + val agg = RowAggregator(AggregationOperator.TopK, Seq(1.0), tvSchema) + val resultObsa = RangeVectorAggregator.mapReduce(agg, false, Observable.fromIterable(samples), + noGrouping) + val resultObsb = RangeVectorAggregator.mapReduce(agg, true, resultObsa, rv=>rv.key) + val resultObsc = RangeVectorAggregator.present(agg, resultObsb, 1000, rangeParams) + val result = resultObsc.toListL.runAsync.futureValue + + result.size shouldEqual 2 + result(0).key shouldEqual ignoreKey2 + result(1).key shouldEqual ignoreKey + + compareIter(result(0).rows.map(_.getDouble(1)).toIterator, + Seq(Double.NaN, Double.NaN, Double.NaN, 5.7, 4.4, Double.NaN).toIterator) + compareIter(result(1).rows.map(_.getDouble(1)).toIterator, + Seq(Double.NaN, 5.1, Double.NaN, Double.NaN, Double.NaN, Double.NaN).toIterator) + } + + it("should sum and compute max of histogram & max RVs") { val (data1, rv1) = MMD.histMaxRV(100000L, numSamples = 5) val (data2, rv2) = MMD.histMaxRV(100000L, numSamples = 5) val samples: Array[RangeVector] = Array(rv1, rv2) @@ -560,7 +595,7 @@ class AggrOverRangeVectorsSpec extends RawDataWindowingSpec with ScalaFutures { val resultObs = RangeVectorAggregator.mapReduce(agg, false, Observable.fromIterable(samples), noGrouping) val resultObs1 = RangeVectorAggregator.mapReduce(agg, true, resultObs, rv=>rv.key) - val resultObs2 = RangeVectorAggregator.present(agg, resultObs1, 1000) + val resultObs2 = RangeVectorAggregator.present(agg, resultObs1, 1000, RangeParams(0,1,0) ) val result = resultObs2.toListL.runAsync.futureValue result.size.shouldEqual(4) result.map(_.key.labelValues).sameElements(expectedLabels) shouldEqual true From 960c7ad00d007e22d9db41a9818b1e01fdbfbacd Mon Sep 17 00:00:00 2001 From: TanviBhavsar Date: Thu, 3 Dec 2020 21:01:22 -0800 Subject: [PATCH 24/33] bug(query): time added to builder in TopBottomKRowAggregator is in seconds (#965) --- .../query/exec/aggregator/TopBottomKRowAggregator.scala | 4 +++- .../scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala | 4 ++++ 2 files changed, 7 insertions(+), 1 deletion(-) 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 3aca39ebe5..438b9c2e53 100644 --- a/query/src/main/scala/filodb/query/exec/aggregator/TopBottomKRowAggregator.scala +++ b/query/src/main/scala/filodb/query/exec/aggregator/TopBottomKRowAggregator.scala @@ -1,5 +1,7 @@ package filodb.query.exec.aggregator +import java.util.concurrent.TimeUnit + import scala.collection.mutable import scala.collection.mutable.ListBuffer @@ -120,7 +122,7 @@ class TopBottomKRowAggregator(k: Int, bottomK: Boolean) extends RowAggregator { val rvk = CustomRangeVectorKey.fromZcUtf8(row.filoUTF8String(i)) rvkSeen += rvk val builder = resRvs.getOrElseUpdate(rvk, createBuilder(rangeParams, t)) - addRecordToBuilder(builder, t, row.getDouble(i + 1)) + addRecordToBuilder(builder, TimeUnit.SECONDS.toMillis(t), row.getDouble(i + 1)) } i += 2 } diff --git a/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala b/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala index be5d8d190e..a79cdacc7f 100644 --- a/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala +++ b/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala @@ -547,6 +547,10 @@ class AggrOverRangeVectorsSpec extends RawDataWindowingSpec with ScalaFutures { result(0).key shouldEqual ignoreKey2 result(1).key shouldEqual ignoreKey + result(0).rows.map(_.getLong(0)). + sameElements(Seq(1000L, 2000L, 3000L, 4000L, 5000L, 6000L).toIterator) shouldEqual true + result(1).rows.map(_.getLong(0)). + sameElements(Seq(1000L, 2000L, 3000L, 4000L, 5000L, 6000L).toIterator) shouldEqual true compareIter(result(0).rows.map(_.getDouble(1)).toIterator, Seq(Double.NaN, Double.NaN, Double.NaN, 5.7, 4.4, Double.NaN).toIterator) compareIter(result(1).rows.map(_.getDouble(1)).toIterator, From 35699f5cecb7154e9200c2c0ae2d544c55b84778 Mon Sep 17 00:00:00 2001 From: TanviBhavsar Date: Fri, 4 Dec 2020 13:50:02 -0800 Subject: [PATCH 25/33] feat(query): make skipAggregatePresent false when there is single matching value for shard key regex (#968) --- .../queryplanner/ShardKeyRegexPlanner.scala | 9 +++++-- .../ShardKeyRegexPlannerSpec.scala | 27 +++++++++++++++++++ 2 files changed, 34 insertions(+), 2 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlanner.scala index b6a60187dc..140828da81 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlanner.scala @@ -65,14 +65,16 @@ class ShardKeyRegexPlanner(dataset: Dataset, private def generateExecWithoutRegex(logicalPlan: LogicalPlan, nonMetricShardKeyFilters: Seq[ColumnFilter], qContext: QueryContext): Seq[ExecPlan] = { val queryParams = qContext.origQueryParams.asInstanceOf[PromQlQueryParams] - shardKeyMatcher(nonMetricShardKeyFilters).map { result => + val shardKeyMatches = shardKeyMatcher(nonMetricShardKeyFilters) + val skipAggregatePresentValue = if (shardKeyMatches.length == 1) false else true + shardKeyMatches.map { result => val newLogicalPlan = logicalPlan.replaceFilters(result) // Querycontext should just have the part of query which has regex // For example for exp(sum(test{_ws_ = "demo", _ns_ =~ "App.*"})), sub queries should be // sum(test{_ws_ = "demo", _ns_ = "App-1"}), sum(test{_ws_ = "demo", _ns_ = "App-2"}) etc val newQueryParams = queryParams.copy(promQl = LogicalPlanParser.convertToQuery(newLogicalPlan)) val newQueryContext = qContext.copy(origQueryParams = newQueryParams, plannerParams = qContext.plannerParams. - copy(skipAggregatePresent = true)) + copy(skipAggregatePresent = skipAggregatePresentValue)) queryPlanner.materialize(logicalPlan.replaceFilters(result), newQueryContext) } } @@ -96,6 +98,9 @@ class ShardKeyRegexPlanner(dataset: Dataset, LogicalPlan.getNonMetricShardKeyFilters(aggregate, dataset.options.nonMetricShardColumns).head, queryContext) val exec = if (execPlans.size == 1) execPlans.head else { + if (aggregate.operator.equals(AggregationOperator.TopK) || aggregate.operator.equals(AggregationOperator.BottomK) + || aggregate.operator.equals(AggregationOperator.CountValues)) + throw new UnsupportedOperationException(s"Shard Key regex not supported for ${aggregate.operator}") val reducer = MultiPartitionReduceAggregateExec(queryContext, InProcessPlanDispatcher, execPlans.sortWith((x, y) => !x.isInstanceOf[PromQlRemoteExec]), aggregate.operator, aggregate.params) val promQlQueryParams = queryContext.origQueryParams.asInstanceOf[PromQlQueryParams] diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlannerSpec.scala index edb8d3088b..c8e3ae0f7b 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlannerSpec.scala @@ -249,4 +249,31 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture execPlan.isInstanceOf[LabelValuesDistConcatExec] shouldEqual (true) } + + it("should generate Exec plan for topk query with single matching value for regex") { + val lp = Parser.queryToLogicalPlan(s"""topk(2, test{_ws_ = "demo", _ns_ =~ "App-1"})""", + 1000, 1000) + val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { + Seq(Seq(ColumnFilter("_ws_", Equals("demo")), + ColumnFilter("_ns_", Equals("App-1")))) + } + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn) + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + execPlan.isInstanceOf[LocalPartitionReduceAggregateExec] shouldEqual (true) + } + + it("should throw UnsupportedOperationException for topk query with multiple matching values for regex") { + val lp = Parser.queryToLogicalPlan(s"""topk(2, test{_ws_ = "demo", _ns_ =~ "App.*"})""", + 1000, 1000) + val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { + Seq(Seq(ColumnFilter("_ws_", Equals("demo")), + ColumnFilter("_ns_", Equals("App-1"))), + Seq(ColumnFilter("_ws_", Equals("demo")), + ColumnFilter("_ns_", Equals("App-2")))) + } + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn) + the[UnsupportedOperationException] thrownBy { + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + } should have message "Shard Key regex not supported for TopK" + } } From 674552d7f77acc397a21d67d2a8bb2c0b6063f28 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Fri, 4 Dec 2020 17:48:05 -0800 Subject: [PATCH 26/33] maint(all): Using too many spans; replace with histograms & markers (#967) --- .../columnstore/CassandraColumnStore.scala | 75 ++++++++++--------- .../filodb.coordinator/IngestionActor.scala | 16 ++-- .../scala/filodb.coordinator/QueryActor.scala | 58 +++++++------- .../queryplanner/QueryPlanner.scala | 1 + .../DownsampledTimeSeriesShard.scala | 20 +++-- .../downsample/ShardDownsampler.scala | 6 -- .../memstore/IndexBootstrapper.scala | 24 +++--- .../memstore/PartKeyLuceneIndex.scala | 48 +++++++----- .../memstore/TimeSeriesShard.scala | 13 ++-- .../scala/filodb/query/exec/ExecPlan.scala | 35 ++++----- .../query/exec/InProcessPlanDispatcher.scala | 1 + .../query/exec/MetadataRemoteExec.scala | 9 +-- .../filodb/query/exec/PromQlRemoteExec.scala | 7 -- .../scala/filodb/query/exec/RemoteExec.scala | 11 +-- .../exec/ScalarBinaryOperationExec.scala | 20 +---- .../query/exec/ScalarFixedDoubleExec.scala | 12 +-- .../query/exec/SelectRawPartitionsExec.scala | 5 -- .../query/exec/TimeScalarGeneratorExec.scala | 13 +--- .../downsampler/chunk/BatchDownsampler.scala | 19 +++-- .../filodb/downsampler/index/DSIndexJob.scala | 10 +-- 20 files changed, 186 insertions(+), 217 deletions(-) diff --git a/cassandra/src/main/scala/filodb.cassandra/columnstore/CassandraColumnStore.scala b/cassandra/src/main/scala/filodb.cassandra/columnstore/CassandraColumnStore.scala index de2f9059cc..c0fb415d06 100644 --- a/cassandra/src/main/scala/filodb.cassandra/columnstore/CassandraColumnStore.scala +++ b/cassandra/src/main/scala/filodb.cassandra/columnstore/CassandraColumnStore.scala @@ -13,6 +13,7 @@ import com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap import com.typesafe.config.Config import com.typesafe.scalalogging.StrictLogging import kamon.Kamon +import kamon.metric.MeasurementUnit import monix.eval.Task import monix.execution.Scheduler import monix.reactive.Observable @@ -56,7 +57,6 @@ extends ColumnStore with CassandraChunkSource with StrictLogging { import collection.JavaConverters._ import filodb.core.store._ - import Perftools._ logger.info(s"Starting CassandraColumnStore with config ${cassandraConfig.withoutPath("password")}") @@ -68,6 +68,13 @@ extends ColumnStore with CassandraChunkSource with StrictLogging { val sinkStats = new ChunkSinkStats + val writeChunksetLatency = Kamon.histogram("cass-write-chunkset-latency", MeasurementUnit.time.milliseconds) + .withoutTags() + val writePksLatency = Kamon.histogram("cass-write-part-keys-latency", MeasurementUnit.time.milliseconds) + .withoutTags() + val readChunksBatchLatency = Kamon.histogram("cassandra-per-batch-chunk-read-latency", + MeasurementUnit.time.milliseconds).withoutTags() + def initialize(dataset: DatasetRef, numShards: Int): Future[Response] = { val chunkTable = getOrCreateChunkTable(dataset) val partitionKeysByUpdateTimeTable = getOrCreatePartitionKeysByUpdateTimeTable(dataset) @@ -133,49 +140,45 @@ extends ColumnStore with CassandraChunkSource with StrictLogging { chunksets: Observable[ChunkSet], diskTimeToLiveSeconds: Int = 259200): Future[Response] = { chunksets.mapAsync(writeParallelism) { chunkset => - val span = Kamon.spanBuilder("write-chunkset").asChildOf(Kamon.currentSpan()).start() - val partBytes = BinaryRegionLarge.asNewByteArray(chunkset.partition) - val future = - for { writeChunksResp <- writeChunks(ref, partBytes, chunkset, diskTimeToLiveSeconds) - if writeChunksResp == Success - writeIndicesResp <- writeIndices(ref, partBytes, chunkset, diskTimeToLiveSeconds) - if writeIndicesResp == Success - } yield { - span.finish() - sinkStats.chunksetWrite() - writeIndicesResp - } - Task.fromFuture(future) - } - .countL.runAsync - .map { chunksWritten => - if (chunksWritten > 0) Success else NotApplied + val start = System.currentTimeMillis() + val partBytes = BinaryRegionLarge.asNewByteArray(chunkset.partition) + val future = + for { writeChunksResp <- writeChunks(ref, partBytes, chunkset, diskTimeToLiveSeconds) + if writeChunksResp == Success + writeIndicesResp <- writeIndices(ref, partBytes, chunkset, diskTimeToLiveSeconds) + if writeIndicesResp == Success + } yield { + writeChunksetLatency.record(System.currentTimeMillis() - start) + sinkStats.chunksetWrite() + writeIndicesResp } + Task.fromFuture(future) + } + .countL.runAsync + .map { chunksWritten => + if (chunksWritten > 0) Success else NotApplied + } } private def writeChunks(ref: DatasetRef, partition: Array[Byte], chunkset: ChunkSet, diskTimeToLiveSeconds: Int): Future[Response] = { - asyncSubtrace("write-chunks", "ingestion") { - val chunkTable = getOrCreateChunkTable(ref) - chunkTable.writeChunks(partition, chunkset.info, chunkset.chunks, sinkStats, diskTimeToLiveSeconds) - .collect { - case Success => chunkset.invokeFlushListener(); Success - } - } + val chunkTable = getOrCreateChunkTable(ref) + chunkTable.writeChunks(partition, chunkset.info, chunkset.chunks, sinkStats, diskTimeToLiveSeconds) + .collect { + case Success => chunkset.invokeFlushListener(); Success + } } private def writeIndices(ref: DatasetRef, partition: Array[Byte], chunkset: ChunkSet, diskTimeToLiveSeconds: Int): Future[Response] = { - asyncSubtrace("write-index", "ingestion") { - val indexTable = getOrCreateIngestionTimeIndexTable(ref) - val info = chunkset.info - val infos = Seq((info.ingestionTime, info.startTime, ChunkSetInfo.toBytes(info))) - indexTable.writeIndices(partition, infos, sinkStats, diskTimeToLiveSeconds) - } + val indexTable = getOrCreateIngestionTimeIndexTable(ref) + val info = chunkset.info + val infos = Seq((info.ingestionTime, info.startTime, ChunkSetInfo.toBytes(info))) + indexTable.writeIndices(partition, infos, sinkStats, diskTimeToLiveSeconds) } /** @@ -212,11 +215,11 @@ extends ColumnStore with CassandraChunkSource with StrictLogging { s"endTimeExclusive=$endTimeExclusive maxChunkTime=$maxChunkTime") // 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() + val start = System.currentTimeMillis() try { chunksTable.readRawPartitionRangeBBNoAsync(parts, userTimeStart - maxChunkTime, endTimeExclusive) } finally { - batchReadSpan.finish() + readChunksBatchLatency.record(System.currentTimeMillis() - start) } } } @@ -376,7 +379,7 @@ extends ColumnStore with CassandraChunkSource with StrictLogging { writeToPkUTTable: Boolean = true): Future[Response] = { val pkTable = getOrCreatePartitionKeysTable(ref, shard) val pkByUTTable = getOrCreatePartitionKeysByUpdateTimeTable(ref) - val span = Kamon.spanBuilder("write-part-keys").asChildOf(Kamon.currentSpan()).start() + val start = System.currentTimeMillis() val ret = partKeys.mapAsync(writeParallelism) { pk => val ttl = if (pk.endTime == Long.MaxValue) -1 else diskTTLSeconds // caller needs to supply hash for partKey - cannot be None @@ -393,7 +396,9 @@ extends ColumnStore with CassandraChunkSource with StrictLogging { resp } }.findL(_.isInstanceOf[ErrorResponse]).map(_.getOrElse(Success)).runAsync - ret.onComplete(_ => span.finish()) + ret.onComplete { _ => + writePksLatency.record(System.currentTimeMillis() - start) + } ret } diff --git a/coordinator/src/main/scala/filodb.coordinator/IngestionActor.scala b/coordinator/src/main/scala/filodb.coordinator/IngestionActor.scala index 7a9f21a412..ee4186673d 100644 --- a/coordinator/src/main/scala/filodb.coordinator/IngestionActor.scala +++ b/coordinator/src/main/scala/filodb.coordinator/IngestionActor.scala @@ -11,6 +11,7 @@ import scala.util.control.NonFatal import akka.actor.{ActorRef, Props} import akka.event.LoggingReceive import kamon.Kamon +import kamon.metric.MeasurementUnit import monix.eval.Task import monix.execution.{CancelableFuture, Scheduler, UncaughtExceptionReporter} import monix.reactive.Observable @@ -293,10 +294,12 @@ private[filodb] final class IngestionActor(ref: DatasetRef, private def doRecovery(shard: Int, startOffset: Long, endOffset: Long, interval: Long, checkpoints: Map[Int, Long]): Future[Option[Long]] = { val futTry = create(shard, Some(startOffset)) map { ingestionStream => - val recoveryTrace = Kamon.spanBuilder("ingestion-recovery-trace") - .asChildOf(Kamon.currentSpan()) - .tag("shard", shard.toString) - .tag("dataset", ref.toString).start() + + val ingestionRecoveryLatency = Kamon.histogram("ingestion-recovery-latency", MeasurementUnit.time.milliseconds) + .withTag("dataset", ref.dataset) + .withTag("shard", shard) + + val recoveryStart = System.currentTimeMillis() val stream = ingestionStream.get statusActor ! RecoveryInProgress(ref, shard, nodeCoord, 0) @@ -317,12 +320,11 @@ private[filodb] final class IngestionActor(ref: DatasetRef, logger.info(s"Finished recovery for dataset=$ref shard=$shard") ingestionStream.teardown() streams.remove(shard) - recoveryTrace.finish() + ingestionRecoveryLatency.record(System.currentTimeMillis() - recoveryStart) case Failure(ex) => - recoveryTrace.fail(s"Recovery failed for dataset=$ref shard=$shard", ex) logger.error(s"Recovery failed for dataset=$ref shard=$shard", ex) handleError(ref, shard, ex) - recoveryTrace.finish() + ingestionRecoveryLatency.record(System.currentTimeMillis() - recoveryStart) }(actorDispatcher) fut } diff --git a/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala b/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala index 3b53aa60e1..6883e1d81e 100644 --- a/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala +++ b/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala @@ -117,31 +117,39 @@ final class QueryActor(memStore: MemStore, def execPhysicalPlan2(q: ExecPlan, replyTo: ActorRef): Unit = { if (checkTimeout(q.queryContext, replyTo)) { epRequests.increment() - Kamon.currentSpan().tag("query", q.getClass.getSimpleName) - Kamon.currentSpan().tag("query-id", q.queryContext.queryId) - val querySession = QuerySession(q.queryContext, queryConfig) - Kamon.currentSpan().mark("query-actor-received-scheduling-into-executor") - q.execute(memStore, querySession)(queryScheduler) - .foreach { res => - FiloSchedulers.assertThreadName(QuerySchedName) - querySession.close() - replyTo ! res - res match { - case QueryResult(_, _, vectors) => resultVectors.record(vectors.length) - case e: QueryError => - queryErrors.increment() - logger.debug(s"queryId ${q.queryContext.queryId} Normal QueryError returned from query execution: $e") - e.t match { - case cve: CorruptVectorException => memStore.analyzeAndLogCorruptPtr(dsRef, cve) - case t: Throwable => - } - } - }(queryScheduler).recover { case ex => - querySession.close() - // Unhandled exception in query, should be rare - logger.error(s"queryId ${q.queryContext.queryId} Unhandled Query Error: ", ex) - replyTo ! QueryError(q.queryContext.queryId, ex) - }(queryScheduler) + val queryExecuteSpan = Kamon.spanBuilder(s"query-actor-exec-plan-execute-${q.getClass.getSimpleName}") + .asChildOf(Kamon.currentSpan()) + .start() + // Dont finish span since we finish it asynchronously when response is received + Kamon.runWithSpan(queryExecuteSpan, false) { + Kamon.currentSpan().tag("query", q.getClass.getSimpleName) + Kamon.currentSpan().tag("query-id", q.queryContext.queryId) + val querySession = QuerySession(q.queryContext, queryConfig) + Kamon.currentSpan().mark("query-actor-received-execute-start") + q.execute(memStore, querySession)(queryScheduler) + .foreach { res => + FiloSchedulers.assertThreadName(QuerySchedName) + querySession.close() + queryExecuteSpan.finish() + replyTo ! res + res match { + case QueryResult(_, _, vectors) => resultVectors.record(vectors.length) + case e: QueryError => + queryErrors.increment() + logger.debug(s"queryId ${q.queryContext.queryId} Normal QueryError returned from query execution: $e") + e.t match { + case cve: CorruptVectorException => memStore.analyzeAndLogCorruptPtr(dsRef, cve) + case t: Throwable => + } + } + }(queryScheduler).recover { case ex => + querySession.close() + // Unhandled exception in query, should be rare + logger.error(s"queryId ${q.queryContext.queryId} Unhandled Query Error: ", ex) + queryExecuteSpan.finish() + replyTo ! QueryError(q.queryContext.queryId, ex) + }(queryScheduler) + } } } diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/QueryPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/QueryPlanner.scala index 3182059c01..4e3f03b892 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/QueryPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/QueryPlanner.scala @@ -33,6 +33,7 @@ trait QueryPlanner { // 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. + // Dont finish span since this code didnt create it Kamon.runWithSpan(parentSpan, false) { execPlan.dispatcher.dispatch(execPlan) } diff --git a/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesShard.scala b/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesShard.scala index 237b358679..95cb95dbbe 100644 --- a/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesShard.scala @@ -9,6 +9,7 @@ import scala.concurrent.duration._ import com.typesafe.config.Config import com.typesafe.scalalogging.StrictLogging import kamon.Kamon +import kamon.metric.MeasurementUnit import kamon.tag.TagSet import monix.eval.Task import monix.execution.{CancelableFuture, Scheduler, UncaughtExceptionReporter} @@ -35,6 +36,10 @@ class DownsampledTimeSeriesShardStats(dataset: DatasetRef, shardNum: Int) { val indexPurgeFailed = Kamon.counter("index-purge-failed").withTags(TagSet.from(tags)) val indexEntries = Kamon.gauge("downsample-store-index-entries").withTags(TagSet.from(tags)) val indexRamBytes = Kamon.gauge("downsample-store-index-ram-bytes").withTags(TagSet.from(tags)) + val singlePartCassFetchLatency = Kamon.histogram("single-partition-cassandra-latency", + MeasurementUnit.time.milliseconds).withTags(TagSet.from(tags)) + val purgeIndexEntriesLatency = Kamon.histogram("downsample-store-purge-index-entries-latency", + MeasurementUnit.time.milliseconds).withTags(TagSet.from(tags)) } class DownsampledTimeSeriesShard(rawDatasetRef: DatasetRef, @@ -138,10 +143,7 @@ class DownsampledTimeSeriesShard(rawDatasetRef: DatasetRef, } private def purgeExpiredIndexEntries(): Unit = { - val tracer = Kamon.spanBuilder("downsample-store-purge-index-entries-latency") - .asChildOf(Kamon.currentSpan()) - .tag("dataset", rawDatasetRef.toString) - .tag("shard", shardNum).start() + val start = System.currentTimeMillis() try { val partsToPurge = partKeyIndex.partIdsEndedBefore(System.currentTimeMillis() - downsampleTtls.last.toMillis) partKeyIndex.removePartKeys(partsToPurge) @@ -152,7 +154,7 @@ class DownsampledTimeSeriesShard(rawDatasetRef: DatasetRef, logger.error(s"Error occurred when purging index entries dataset=$rawDatasetRef shard=$shardNum", e) stats.indexPurgeFailed.increment() } finally { - tracer.finish() + stats.purgeIndexEntriesLatency.record(System.currentTimeMillis() - start) } } @@ -251,11 +253,7 @@ class DownsampledTimeSeriesShard(rawDatasetRef: DatasetRef, // when query is complete. Observable.fromIterable(lookup.pkRecords) .mapAsync(downsampleStoreConfig.demandPagingParallelism) { partRec => - val partLoadSpan = Kamon.spanBuilder(s"single-partition-cassandra-latency") - .asChildOf(Kamon.currentSpan()) - .tag("dataset", rawDatasetRef.toString) - .tag("shard", shardNum) - .start() + val startExecute = System.currentTimeMillis() // TODO test multi-partition scan if latencies are high store.readRawPartitions(downsampledDataset, downsampleStoreConfig.maxChunkTime.toMillis, @@ -265,7 +263,7 @@ class DownsampledTimeSeriesShard(rawDatasetRef: DatasetRef, val part = makePagedPartition(pd, lookup.firstSchemaId.get, Some(resolution), colIds) stats.partitionsQueried.increment() stats.chunksQueried.increment(part.numChunks) - partLoadSpan.finish() + stats.singlePartCassFetchLatency.record(Math.max(0, System.currentTimeMillis - startExecute)) part } .defaultIfEmpty(makePagedPartition(RawPartData(partRec.partKey, Seq.empty), diff --git a/core/src/main/scala/filodb.core/downsample/ShardDownsampler.scala b/core/src/main/scala/filodb.core/downsample/ShardDownsampler.scala index 1277f5bb35..81ac896123 100644 --- a/core/src/main/scala/filodb.core/downsample/ShardDownsampler.scala +++ b/core/src/main/scala/filodb.core/downsample/ShardDownsampler.scala @@ -1,7 +1,6 @@ package filodb.core.downsample import com.typesafe.scalalogging.StrictLogging -import kamon.Kamon import filodb.core.binaryrecord2.RecordBuilder import filodb.core.memstore.{TimeSeriesPartition, TimeSeriesShardStats} @@ -64,10 +63,6 @@ class ShardDownsampler(datasetName: String, chunksets: ChunkInfoIterator, records: Seq[DownsampleRecords]): Unit = { if (enabled) { - val downsampleTrace = Kamon.spanBuilder("memstore-downsample-records-trace") - .asChildOf(Kamon.currentSpan()) - .tag("dataset", datasetName) - .tag("shard", shardNum).start() while (chunksets.hasNext) { val chunkset = chunksets.nextInfoReader val startTime = chunkset.startTime @@ -103,7 +98,6 @@ class ShardDownsampler(datasetName: String, } } } - downsampleTrace.finish() } } } diff --git a/core/src/main/scala/filodb.core/memstore/IndexBootstrapper.scala b/core/src/main/scala/filodb.core/memstore/IndexBootstrapper.scala index 048b548821..7872d90f30 100644 --- a/core/src/main/scala/filodb.core/memstore/IndexBootstrapper.scala +++ b/core/src/main/scala/filodb.core/memstore/IndexBootstrapper.scala @@ -1,6 +1,7 @@ package filodb.core.memstore import kamon.Kamon +import kamon.metric.MeasurementUnit import monix.eval.Task import monix.reactive.Observable @@ -28,11 +29,12 @@ class IndexBootstrapper(colStore: ColumnStore) { shardNum: Int, ref: DatasetRef) (assignPartId: PartKeyRecord => Int): Task[Long] = { - val tracer = Kamon.spanBuilder("memstore-recover-index-latency") - .asChildOf(Kamon.currentSpan()) - .tag("dataset", ref.dataset) - .tag("shard", shardNum).start() + val recoverIndexLatency = Kamon.histogram("shard-recover-index-latency", + MeasurementUnit.time.milliseconds) + .withTag("dataset", ref.dataset) + .withTag("shard", shardNum) + val start = System.currentTimeMillis() colStore.scanPartKeys(ref, shardNum) .map { pk => val partId = assignPartId(pk) @@ -41,7 +43,7 @@ class IndexBootstrapper(colStore: ColumnStore) { .countL .map { count => index.refreshReadersBlocking() - tracer.finish() + recoverIndexLatency.record(System.currentTimeMillis() - start) count } } @@ -64,11 +66,11 @@ class IndexBootstrapper(colStore: ColumnStore) { schemas: Schemas, parallelism: Int = Runtime.getRuntime.availableProcessors()) (lookUpOrAssignPartId: Array[Byte] => Int): Task[Long] = { - val tracer = Kamon.spanBuilder("downsample-store-refresh-index-latency") - .asChildOf(Kamon.currentSpan()) - .tag("dataset", ref.dataset) - .tag("shard", shardNum).start() - + val recoverIndexLatency = Kamon.histogram("shard-recover-index-latency", + MeasurementUnit.time.milliseconds) + .withTag("dataset", ref.dataset) + .withTag("shard", shardNum) + val start = System.currentTimeMillis() Observable.fromIterable(fromHour to toHour).flatMap { hour => colStore.getPartKeysByUpdateHour(ref, shardNum, hour) }.mapAsync(parallelism) { pk => @@ -83,7 +85,7 @@ class IndexBootstrapper(colStore: ColumnStore) { .countL .map { count => index.refreshReadersBlocking() - tracer.finish() + recoverIndexLatency.record(System.currentTimeMillis() - start) count } } diff --git a/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala b/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala index 77e340dec3..6204a12ca1 100644 --- a/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala +++ b/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala @@ -12,6 +12,7 @@ import com.googlecode.javaewah.{EWAHCompressedBitmap, IntIterator} import com.typesafe.scalalogging.StrictLogging import java.util import kamon.Kamon +import kamon.metric.MeasurementUnit import org.apache.lucene.analysis.standard.StandardAnalyzer import org.apache.lucene.document._ import org.apache.lucene.document.Field.Store @@ -76,6 +77,21 @@ class PartKeyLuceneIndex(ref: DatasetRef, import PartKeyLuceneIndex._ + val startTimeLookupLatency = Kamon.histogram("index-startTimes-for-odp-lookup-latency", + MeasurementUnit.time.milliseconds) + .withTag("dataset", ref.dataset) + .withTag("shard", shardNum) + + val queryIndexLookupLatency = Kamon.histogram("index-partition-lookup-latency", + MeasurementUnit.time.milliseconds) + .withTag("dataset", ref.dataset) + .withTag("shard", shardNum) + + val partIdFromPartKeyLookupLatency = Kamon.histogram("index-ingestion-partId-lookup-latency", + MeasurementUnit.time.milliseconds) + .withTag("dataset", ref.dataset) + .withTag("shard", shardNum) + private val numPartColumns = schema.columns.length private val indexDiskLocation = diskLocation.getOrElse(createTempDir(ref, shardNum)).toPath private val mMapDirectory = new MMapDirectory(indexDiskLocation) @@ -360,11 +376,9 @@ class PartKeyLuceneIndex(ref: DatasetRef, * Called when a document is updated with new endTime */ def startTimeFromPartIds(partIds: Iterator[Int]): debox.Map[Int, Long] = { - val span = Kamon.spanBuilder("index-startTimes-for-odp-lookup-latency") - .asChildOf(Kamon.currentSpan()) - .tag("dataset", ref.dataset) - .tag("shard", shardNum) - .start() + + val startExecute = System.currentTimeMillis() + val span = Kamon.currentSpan() val collector = new PartIdStartTimeCollector() val terms = new util.ArrayList[BytesRef]() partIds.foreach { pId => @@ -374,7 +388,9 @@ class PartKeyLuceneIndex(ref: DatasetRef, // more efficient within Lucene withNewSearcher(s => s.search(new TermInSetQuery(PART_ID, terms), collector)) span.tag(s"num-partitions-to-page", terms.size()) - span.finish() + val latency = System.currentTimeMillis - startExecute + span.mark(s"index-startTimes-for-odp-lookup-latency=${latency}ms") + startTimeLookupLatency.record(latency) collector.startTimes } @@ -493,11 +509,9 @@ class PartKeyLuceneIndex(ref: DatasetRef, startTime: Long, endTime: Long, collector: Collector): Unit = { - val partKeySpan = Kamon.spanBuilder("index-partition-lookup-latency") - .tag("dataset", ref.dataset) - .tag("shard", shardNum) - .asChildOf(Kamon.currentSpan()) - .start() + + val startExecute = System.currentTimeMillis() + val span = Kamon.currentSpan() val booleanQuery = new BooleanQuery.Builder columnFilters.foreach { filter => val q = leafFilter(filter.column, filter.filter) @@ -508,7 +522,9 @@ class PartKeyLuceneIndex(ref: DatasetRef, val query = booleanQuery.build() logger.debug(s"Querying dataset=$ref shard=$shardNum partKeyIndex with: $query") withNewSearcher(s => s.search(query, collector)) - partKeySpan.finish() + val latency = System.currentTimeMillis - startExecute + span.mark(s"index-partition-lookup-latency=${latency}ms") + queryIndexLookupLatency.record(latency) } def partIdFromPartKeySlow(partKeyBase: Any, @@ -517,11 +533,7 @@ class PartKeyLuceneIndex(ref: DatasetRef, val columnFilters = schema.binSchema.toStringPairs(partKeyBase, partKeyOffset) .map { pair => ColumnFilter(pair._1, Filter.Equals(pair._2)) } - val partKeySpan = Kamon.spanBuilder("index-partition-lookup-latency") - .asChildOf(Kamon.currentSpan()) - .tag("dataset", ref.dataset) - .tag("shard", shardNum) - .start() + val startExecute = System.currentTimeMillis() val booleanQuery = new BooleanQuery.Builder columnFilters.foreach { filter => val q = leafFilter(filter.column, filter.filter) @@ -542,7 +554,7 @@ class PartKeyLuceneIndex(ref: DatasetRef, } val collector = new ActionCollector(handleMatch) withNewSearcher(s => s.search(query, collector)) - partKeySpan.finish() + partIdFromPartKeyLookupLatency.record(Math.max(0, System.currentTimeMillis - startExecute)) chosenPartId } } diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala index 30bb4fc38e..be2e4cc3fc 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala @@ -105,7 +105,10 @@ class TimeSeriesShardStats(dataset: DatasetRef, shardNum: Int) { * expected), then the delay reflects the delay between the generation of the samples and * receiving them, assuming that the clocks are in sync. */ - val ingestionClockDelay = Kamon.gauge("ingestion-clock-delay").withTags(TagSet.from(tags)) + val ingestionClockDelay = Kamon.gauge("ingestion-clock-delay", + MeasurementUnit.time.milliseconds).withTags(TagSet.from(tags)) + val chunkFlushTaskLatency = Kamon.histogram("chunk-flush-task-latency-after-retries", + MeasurementUnit.time.milliseconds).withTags(TagSet.from(tags)) } object TimeSeriesShard { @@ -895,11 +898,7 @@ class TimeSeriesShard(val ref: DatasetRef, private def doFlushSteps(flushGroup: FlushGroup, partitionIt: Iterator[TimeSeriesPartition]): Task[Response] = { assertThreadName(IngestSchedName) - - val tracer = Kamon.spanBuilder("chunk-flush-task-latency-after-retries") - .asChildOf(Kamon.currentSpan()) - .tag("dataset", ref.dataset) - .tag("shard", shardNum).start() + val flushStart = System.currentTimeMillis() // Only allocate the blockHolder when we actually have chunks/partitions to flush val blockHolder = blockFactoryPool.checkout(Map("flushGroup" -> flushGroup.groupNum.toString)) @@ -958,7 +957,7 @@ class TimeSeriesShard(val ref: DatasetRef, blockHolder.markFullBlocksReclaimable() blockFactoryPool.release(blockHolder) flushDoneTasks(flushGroup, resp) - tracer.finish() + shardStats.chunkFlushTaskLatency.record(System.currentTimeMillis() - flushStart) } catch { case e: Throwable => logger.error(s"Error when wrapping up doFlushSteps in dataset=$ref shard=$shardNum", e) } diff --git a/query/src/main/scala/filodb/query/exec/ExecPlan.scala b/query/src/main/scala/filodb/query/exec/ExecPlan.scala index f87c7e73e0..fa886e8103 100644 --- a/query/src/main/scala/filodb/query/exec/ExecPlan.scala +++ b/query/src/main/scala/filodb/query/exec/ExecPlan.scala @@ -99,27 +99,26 @@ trait ExecPlan extends QueryCommand { val startExecute = querySession.qContext.submitTime - val parentSpan = Kamon.currentSpan() + val span = Kamon.currentSpan() // NOTE: we launch the preparatory steps as a Task too. This is important because scanPartitions, // Lucene index lookup, and On-Demand Paging orchestration work could suck up nontrivial time and // we don't want these to happen in a single thread. // Step 1: initiate doExecute: make result schema and set up the async monix pipeline to create RVs lazy val step1 = Task { - val span = Kamon.spanBuilder(s"execute-step1-${getClass.getSimpleName}") - .asChildOf(parentSpan) - .tag("query-id", queryContext.queryId) - .start() + span.mark(s"execute-step1-start-${getClass.getSimpleName}") FiloSchedulers.assertThreadName(QuerySchedName) // 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(span, true) { + // Dont finish span since this code didnt create it + Kamon.runWithSpan(span, false) { val doEx = doExecute(source, querySession) Kamon.histogram("query-execute-time-elapsed-step1-done", MeasurementUnit.time.milliseconds) .withTag("plan", getClass.getSimpleName) .record(Math.max(0, System.currentTimeMillis - startExecute)) + span.mark(s"execute-step1-end-${getClass.getSimpleName}") doEx } } @@ -129,17 +128,14 @@ trait ExecPlan extends QueryCommand { Kamon.histogram("query-execute-time-elapsed-step2-start", MeasurementUnit.time.milliseconds) .withTag("plan", getClass.getSimpleName) .record(Math.max(0, System.currentTimeMillis - startExecute)) - val span = Kamon.spanBuilder(s"execute-step2-${getClass.getSimpleName}") - .asChildOf(parentSpan) - .tag("query-id", queryContext.queryId) - .start() + span.mark(s"execute-step2-start-${getClass.getSimpleName}") FiloSchedulers.assertThreadName(QuerySchedName) val dontRunTransformers = if (allTransformers.isEmpty) true else !allTransformers.forall(_.canHandleEmptySchemas) - span.tag("dontRunTransformers", dontRunTransformers) // It is possible a null schema is returned (due to no time series). In that case just return empty results val resultTask = if (resSchema == ResultSchema.empty && dontRunTransformers) { qLogger.debug(s"queryId: ${queryContext.queryId} Empty plan $this, returning empty results") span.mark("empty-plan") + span.mark(s"execute-step2-end-${getClass.getSimpleName}") Task.eval(QueryResult(queryContext.queryId, resSchema, Nil)) } else { val finalRes = allTransformers.foldLeft((res.rvs, resSchema)) { (acc, transf) => @@ -152,7 +148,6 @@ trait ExecPlan extends QueryCommand { MeasurementUnit.time.milliseconds) .withTag("plan", getClass.getSimpleName) .record(Math.max(0, System.currentTimeMillis - startExecute)) - span.mark("step2-transformer-pipeline-setup") val builder = SerializedRangeVector.newBuilder() @volatile var numResultSamples = 0 // BEWARE - do not modify concurrently!! finalRes._1 @@ -194,7 +189,7 @@ trait ExecPlan extends QueryCommand { } span.mark(s"num-result-samples: $numResultSamples") span.mark(s"num-range-vectors: ${r.size}") - span.finish() + span.mark(s"execute-step2-end-${getClass.getSimpleName}") QueryResult(queryContext.queryId, finalRes._2, r) } } @@ -374,6 +369,7 @@ abstract class NonLeafExecPlan extends ExecPlan { // 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. + // Dont finish span since this code didnt create it Kamon.runWithSpan(span, false) { plan.dispatcher.dispatch(plan).onErrorHandle { case ex: Throwable => qLogger.error(s"queryId: ${queryContext.queryId} Execution failed for sub-query ${plan.printTree()}", ex) @@ -392,13 +388,9 @@ abstract class NonLeafExecPlan extends ExecPlan { final def doExecute(source: ChunkSource, querySession: QuerySession) (implicit sched: Scheduler): ExecResult = { - val parentSpan = Kamon.currentSpan() - - val span = Kamon.spanBuilder(s"execute-step1-child-result-composition-${getClass.getSimpleName}") - .asChildOf(parentSpan) - .tag("query-id", queryContext.queryId) - .start() + val span = Kamon.currentSpan() + span.mark(s"execute-step1-child-result-composition-start-${getClass.getSimpleName}") // whether child tasks need to be executed sequentially. // parallelism 1 means, only one worker thread to process underlying tasks. val parallelism: Int = if (parallelChildTasks) @@ -411,7 +403,7 @@ abstract class NonLeafExecPlan extends ExecPlan { val childTasks = Observable.fromIterable(children.zipWithIndex) .mapAsync(parallelism) { case (plan, i) => val task = dispatchRemotePlan(plan, span).map((_, i)) - span.mark(s"plan-dispatched-${plan.getClass.getSimpleName}") + span.mark(s"child-plan-$i-dispatched-${plan.getClass.getSimpleName}") task } @@ -433,9 +425,10 @@ abstract class NonLeafExecPlan extends ExecPlan { val outputSchema = processedTasks.collect { case (QueryResult(_, schema, _), _) => schema }.firstOptionL.map(_.getOrElse(ResultSchema.empty)) + // Dont finish span since this code didnt create it Kamon.runWithSpan(span, false) { val outputRvs = compose(processedTasks, outputSchema, querySession) - .doOnTerminate(_ => span.finish()) + .doOnTerminate(_ => span.mark(s"execute-step1-child-result-composition-end-${getClass.getSimpleName}")) ExecResult(outputRvs, outputSchema) } } diff --git a/query/src/main/scala/filodb/query/exec/InProcessPlanDispatcher.scala b/query/src/main/scala/filodb/query/exec/InProcessPlanDispatcher.scala index db7883adb7..92333b40d4 100644 --- a/query/src/main/scala/filodb/query/exec/InProcessPlanDispatcher.scala +++ b/query/src/main/scala/filodb/query/exec/InProcessPlanDispatcher.scala @@ -30,6 +30,7 @@ case object InProcessPlanDispatcher extends PlanDispatcher { // 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. + // Dont finish span since this code didnt create it Kamon.runWithSpan(Kamon.currentSpan(), false) { // translate implicit ExecutionContext to monix.Scheduler val querySession = QuerySession(plan.queryContext, queryConfig) diff --git a/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala b/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala index f6e4d69488..408cdd3df2 100644 --- a/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala +++ b/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala @@ -2,7 +2,6 @@ package filodb.query.exec import scala.concurrent.Future -import kamon.Kamon import kamon.trace.Span import monix.execution.Scheduler @@ -39,20 +38,14 @@ case class MetadataRemoteExec(queryEndpoint: String, } 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) }} import NoCloseCursor._ val rangeVector = IteratorBackedRangeVector(new CustomRangeVectorKey(Map.empty), - new UTF8MapIteratorRowReader(iteratorMap.toIterator)) + 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/PromQlRemoteExec.scala b/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala index 03ed7077e7..a9ded9049b 100644 --- a/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala +++ b/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala @@ -2,7 +2,6 @@ package filodb.query.exec import scala.concurrent.Future -import kamon.Kamon import kamon.trace.Span import monix.execution.Scheduler @@ -60,11 +59,6 @@ case class PromQlRemoteExec(queryEndpoint: String, // 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 queryResponse = if (data.result.isEmpty) { logger.debug("PromQlRemoteExec generating empty QueryResult as result is empty") QueryResult(id, ResultSchema.empty, Seq.empty) @@ -84,7 +78,6 @@ case class PromQlRemoteExec(queryEndpoint: String, } } } - span.finish() queryResponse } diff --git a/query/src/main/scala/filodb/query/exec/RemoteExec.scala b/query/src/main/scala/filodb/query/exec/RemoteExec.scala index 229e605e04..a0a7b0387b 100644 --- a/query/src/main/scala/filodb/query/exec/RemoteExec.scala +++ b/query/src/main/scala/filodb/query/exec/RemoteExec.scala @@ -49,11 +49,6 @@ trait RemoteExec extends LeafExecPlan with StrictLogging { 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.") } @@ -61,8 +56,10 @@ trait RemoteExec extends LeafExecPlan with StrictLogging { // 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, requestTimeoutMs)) + val span = Kamon.currentSpan() + // Dont finish span since this code didnt create it + Kamon.runWithSpan(span, false) { + Task.fromFuture(sendHttpRequest(span, requestTimeoutMs)) } } diff --git a/query/src/main/scala/filodb/query/exec/ScalarBinaryOperationExec.scala b/query/src/main/scala/filodb/query/exec/ScalarBinaryOperationExec.scala index 012f2c92d1..f8d444cc4b 100644 --- a/query/src/main/scala/filodb/query/exec/ScalarBinaryOperationExec.scala +++ b/query/src/main/scala/filodb/query/exec/ScalarBinaryOperationExec.scala @@ -1,6 +1,5 @@ package filodb.query.exec -import kamon.Kamon import monix.eval.Task import monix.execution.Scheduler @@ -57,24 +56,9 @@ case class ScalarBinaryOperationExec(queryContext: QueryContext, 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 rangeVectors : Seq[RangeVector] = Seq(ScalarFixedDouble(params, evaluate)) - // 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 { - val span = Kamon.spanBuilder(s"transform-${getClass.getSimpleName}") - .asChildOf(execPlan2Span) - .tag("query-id", queryContext.queryId) - .start() - span.finish() - QueryResult(queryContext.queryId, resultSchema, rangeVectors) - } + Task { + QueryResult(queryContext.queryId, resultSchema, rangeVectors) } } diff --git a/query/src/main/scala/filodb/query/exec/ScalarFixedDoubleExec.scala b/query/src/main/scala/filodb/query/exec/ScalarFixedDoubleExec.scala index 16ed0971a2..6991d0c795 100644 --- a/query/src/main/scala/filodb/query/exec/ScalarFixedDoubleExec.scala +++ b/query/src/main/scala/filodb/query/exec/ScalarFixedDoubleExec.scala @@ -46,27 +46,19 @@ case class ScalarFixedDoubleExec(queryContext: QueryContext, 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 resultSchema = ResultSchema(columns, 1) val rangeVectors : Seq[RangeVector] = Seq(ScalarFixedDouble(params, value)) // 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) { + // Dont finish span since this code didnt create it + Kamon.runWithSpan(Kamon.currentSpan(), false) { Task { - val span = Kamon.spanBuilder(s"transform-${getClass.getSimpleName}") - .asChildOf(execPlan2Span) - .tag("query-id", queryContext.queryId) - .start() rangeVectorTransformers.foldLeft((Observable.fromIterable(rangeVectors), resultSchema)) { (acc, transf) => val paramRangeVector: Seq[Observable[ScalarRangeVector]] = transf.funcParams.map(_.getResult) (transf.apply(acc._1, querySession, queryContext.plannerParams.sampleLimit, acc._2, paramRangeVector), transf.schema(acc._2)) }._1.toListL.map({ - span.finish() QueryResult(queryContext.queryId, resultSchema, _) }) }.flatten diff --git a/query/src/main/scala/filodb/query/exec/SelectRawPartitionsExec.scala b/query/src/main/scala/filodb/query/exec/SelectRawPartitionsExec.scala index a18a668baa..b7e9867029 100644 --- a/query/src/main/scala/filodb/query/exec/SelectRawPartitionsExec.scala +++ b/query/src/main/scala/filodb/query/exec/SelectRawPartitionsExec.scala @@ -1,6 +1,5 @@ package filodb.query.exec -import kamon.Kamon import monix.eval.Task import monix.execution.Scheduler import monix.reactive.Observable @@ -124,16 +123,12 @@ final case class SelectRawPartitionsExec(queryContext: QueryContext, def doExecute(source: ChunkSource, querySession: QuerySession) (implicit sched: Scheduler): ExecResult = { - val span = Kamon.spanBuilder(s"execute-${getClass.getSimpleName}") - .asChildOf(Kamon.currentSpan()) - .start() Query.qLogger.debug(s"queryId=${queryContext.queryId} on dataset=$datasetRef shard=" + s"${lookupRes.map(_.shard).getOrElse("")} " + s"schema=" + s"${dataSchema.map(_.name)} is configured to use columnIDs=$colIds") val rvs = dataSchema.map { sch => source.rangeVectors(datasetRef, lookupRes.get, colIds, sch, filterSchemas, querySession) }.getOrElse(Observable.empty) - span.finish() ExecResult(rvs, Task.eval(schemaOfDoExecute())) } diff --git a/query/src/main/scala/filodb/query/exec/TimeScalarGeneratorExec.scala b/query/src/main/scala/filodb/query/exec/TimeScalarGeneratorExec.scala index 220d8b3c5f..dea5ac001c 100644 --- a/query/src/main/scala/filodb/query/exec/TimeScalarGeneratorExec.scala +++ b/query/src/main/scala/filodb/query/exec/TimeScalarGeneratorExec.scala @@ -44,10 +44,7 @@ case class TimeScalarGeneratorExec(queryContext: QueryContext, 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 span = Kamon.currentSpan() val resultSchema = ResultSchema(columns, 1) val rangeVectors : Seq[RangeVector] = function match { case Time => Seq(TimeScalar(params)) @@ -63,18 +60,14 @@ case class TimeScalarGeneratorExec(queryContext: QueryContext, // 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) { + // Dont finish span since this code didnt create it + Kamon.runWithSpan(span, false) { Task { - val span = Kamon.spanBuilder(s"transform-${getClass.getSimpleName}") - .asChildOf(execPlan2Span) - .tag("query-id", queryContext.queryId) - .start() rangeVectorTransformers.foldLeft((Observable.fromIterable(rangeVectors), resultSchema)) { (acc, transf) => val paramRangeVector: Seq[Observable[ScalarRangeVector]] = transf.funcParams.map(_.getResult) (transf.apply(acc._1, querySession, queryContext.plannerParams.sampleLimit, acc._2, paramRangeVector), transf.schema(acc._2)) }._1.toListL.map({ - span.finish() QueryResult(queryContext.queryId, resultSchema, _) }) }.flatten diff --git a/spark-jobs/src/main/scala/filodb/downsampler/chunk/BatchDownsampler.scala b/spark-jobs/src/main/scala/filodb/downsampler/chunk/BatchDownsampler.scala index 8f1c51c30e..acc992aad1 100644 --- a/spark-jobs/src/main/scala/filodb/downsampler/chunk/BatchDownsampler.scala +++ b/spark-jobs/src/main/scala/filodb/downsampler/chunk/BatchDownsampler.scala @@ -5,6 +5,7 @@ import scala.concurrent.Await import scala.concurrent.duration.FiniteDuration import kamon.Kamon +import kamon.metric.MeasurementUnit import monix.reactive.Observable import spire.syntax.cfor._ @@ -49,6 +50,13 @@ class BatchDownsampler(settings: DownsamplerSettings) extends Instance with Seri @transient lazy val numRawChunksDownsampled = Kamon.counter("num-raw-chunks-downsampled").withoutTags() @transient lazy val numDownsampledChunksWritten = Kamon.counter("num-downsampled-chunks-written").withoutTags() + @transient lazy val downsampleBatchLatency = Kamon.histogram("downsample-batch-latency", + MeasurementUnit.time.milliseconds).withoutTags() + @transient lazy val downsampleSinglePartLatency = Kamon.histogram("downsample-single-partition-latency", + MeasurementUnit.time.milliseconds).withoutTags() + @transient lazy val downsampleBatchPersistLatency = Kamon.histogram("cassandra-downsample-batch-persist-latency", + MeasurementUnit.time.milliseconds).withoutTags() + @transient lazy private val session = DownsamplerContext.getOrCreateCassandraSession(settings.cassandraConfig) @transient lazy private[downsampler] val downsampleCassandraColStore = @@ -110,7 +118,6 @@ class BatchDownsampler(settings: DownsamplerSettings) extends Instance with Seri def downsampleBatch(rawPartsBatch: Seq[RawPartData], userTimeStart: Long, userTimeEndExclusive: Long): Unit = { - val batchSpan = Kamon.spanBuilder("downsample-batch-latency").start() DownsamplerContext.dsLogger.info(s"Starting to downsample batchSize=${rawPartsBatch.size} partitions " + s"rawDataset=${settings.rawDatasetName} for " + s"userTimeStart=${java.time.Instant.ofEpochMilli(userTimeStart)} " + @@ -158,7 +165,7 @@ class BatchDownsampler(settings: DownsamplerSettings) extends Instance with Seri numBatchesFailed.increment() throw e // will be logged by spark } finally { - batchSpan.finish() + downsampleBatchLatency.record(System.currentTimeMillis() - startedAt) offHeapMem.free() // free offheap mem pagedPartsToFree.clear() downsampledPartsToFree.clear() @@ -221,7 +228,7 @@ class BatchDownsampler(settings: DownsamplerSettings) extends Instance with Seri res -> part }.toMap - val downsamplePartSpan = Kamon.spanBuilder("downsample-single-partition-latency").start() + val downsamplePartStart = System.currentTimeMillis() downsampleChunks(offHeapMem, rawReadablePart, downsamplers, periodMarker, downsampledParts, userTimeStart, userTimeEndExclusive, dsRecordBuilder, shouldTrace) @@ -233,7 +240,7 @@ class BatchDownsampler(settings: DownsamplerSettings) extends Instance with Seri val newIt = downsampledChunksToPersist(res) ++ dsPartition.makeFlushChunks(offHeapMem.blockMemFactory) downsampledChunksToPersist(res) = newIt } - downsamplePartSpan.finish() + downsampleSinglePartLatency.record(System.currentTimeMillis() - downsamplePartStart) case None => numPartitionsNoDownsampleSchema.increment() DownsamplerContext.dsLogger.debug(s"Skipping downsampling of partition " + @@ -363,7 +370,7 @@ class BatchDownsampler(settings: DownsamplerSettings) extends Instance with Seri * Persist chunks in `downsampledChunksToPersist` to Cassandra. */ private def persistDownsampledChunks(downsampledChunksToPersist: MMap[FiniteDuration, Iterator[ChunkSet]]): Int = { - val batchWriteSpan = Kamon.spanBuilder("cassandra-downsample-batch-persist-latency").start() + val start = System.currentTimeMillis() @volatile var numChunks = 0 // write all chunks to cassandra val writeFut = downsampledChunksToPersist.map { case (res, chunks) => @@ -385,7 +392,7 @@ class BatchDownsampler(settings: DownsamplerSettings) extends Instance with Seri DownsamplerContext.dsLogger.error(s"Got response $response when writing to Cassandra") } numDownsampledChunksWritten.increment(numChunks) - batchWriteSpan.finish() + downsampleBatchPersistLatency.record(System.currentTimeMillis() - start) numChunks } diff --git a/spark-jobs/src/main/scala/filodb/downsampler/index/DSIndexJob.scala b/spark-jobs/src/main/scala/filodb/downsampler/index/DSIndexJob.scala index 61ff5a43ca..41bffa740c 100644 --- a/spark-jobs/src/main/scala/filodb/downsampler/index/DSIndexJob.scala +++ b/spark-jobs/src/main/scala/filodb/downsampler/index/DSIndexJob.scala @@ -3,6 +3,7 @@ package filodb.downsampler.index import scala.concurrent.Await import kamon.Kamon +import kamon.metric.MeasurementUnit import monix.reactive.Observable import filodb.cassandra.columnstore.CassandraColumnStore @@ -24,6 +25,8 @@ class DSIndexJob(dsSettings: DownsamplerSettings, @transient lazy private val numPartKeysNoDownsampleSchema = Kamon.counter("num-partkeys-no-downsample").withoutTags() @transient lazy private val numPartKeysMigrated = Kamon.counter("num-partkeys-migrated").withoutTags() @transient lazy private val numPartKeysBlocked = Kamon.counter("num-partkeys-blocked").withoutTags() + @transient lazy val perShardIndexMigrationLatency = Kamon.histogram("per-shard-index-migration-latency", + MeasurementUnit.time.milliseconds).withoutTags() @transient lazy private[downsampler] val schemas = Schemas.fromConfig(dsSettings.filodbConfig).get @@ -59,10 +62,7 @@ class DSIndexJob(dsSettings: DownsamplerSettings, val rawDataSource = rawCassandraColStore @volatile var count = 0 try { - val span = Kamon.spanBuilder("per-shard-index-migration-latency") - .asChildOf(Kamon.currentSpan()) - .tag("shard", shard) - .start + val start = System.currentTimeMillis() if (fullIndexMigration) { DownsamplerContext.dsLogger.info(s"Starting Full PartKey Migration for shard=$shard") val partKeys = rawDataSource.scanPartKeys(ref = rawDatasetRef, @@ -80,7 +80,7 @@ class DSIndexJob(dsSettings: DownsamplerSettings, s"count=$count fromHour=$fromHour toHourExcl=$toHourExcl") } sparkForeachTasksCompleted.increment() - span.finish() + perShardIndexMigrationLatency.record(System.currentTimeMillis() - start) } catch { case e: Exception => DownsamplerContext.dsLogger.error(s"Exception in task count=$count " + s"shard=$shard fromHour=$fromHour toHourExcl=$toHourExcl fullIndexMigration=$fullIndexMigration", e) From 594dd0b66fc3e3371afa047b749ecb0ff30684d4 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Wed, 9 Dec 2020 12:03:56 -0800 Subject: [PATCH 27/33] debug(memory): Additional debugging for debugging negative lock state (#970) --- .../main/scala/filodb.memory/data/ChunkMap.scala | 16 ++++++++++++++-- 1 file changed, 14 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 d44295aa3d..5106590625 100644 --- a/memory/src/main/scala/filodb.memory/data/ChunkMap.scala +++ b/memory/src/main/scala/filodb.memory/data/ChunkMap.scala @@ -92,7 +92,6 @@ object ChunkMap extends StrictLogging { */ //scalastyle:off null def releaseAllSharedLocks(): Int = { - execPlanTracker.remove(Thread.currentThread()) var total = 0 val countMap = sharedLockCounts.get if (countMap != null) { @@ -100,16 +99,24 @@ object ChunkMap extends StrictLogging { if (amt > 0) { total += amt sharedLockLingering.increment(amt) - _logger.warn(s"Releasing all shared locks for: $inst, amount: $amt") + _logger.error(s"Lingering locks while releasing all shared locks for pk $inst, amount=$amt " + + s"Contents of execPlanTracker for current thread: ${execPlanTracker.get(Thread.currentThread())}", + new RuntimeException) var lockState = 0 do { lockState = UnsafeUtils.getIntVolatile(inst, lockStateOffset) + if (lockState - amt < 0) { + _logger.error(s"Negative lock state while releasing all shared locks for pk: $inst, amount=$amt " + + s"Contents of execPlanTracker for current thread: ${execPlanTracker.get(Thread.currentThread())}", + new RuntimeException) + } } while (!UnsafeUtils.unsafe.compareAndSwapInt(inst, lockStateOffset, lockState, lockState - amt)) } } countMap.clear } + execPlanTracker.remove(Thread.currentThread()) total } //scalastyle:on null @@ -397,6 +404,11 @@ class ChunkMap(val memFactory: NativeMemoryManager, var capacity: Int) { var lockState = 0 do { lockState = UnsafeUtils.getIntVolatile(this, lockStateOffset) + if (lockState - 1 < 0) { + _logger.error(s"Negative lock state while releasing single shared lock for pk: $this " + + s"Contents of execPlanTracker for current thread: ${execPlanTracker.get(Thread.currentThread())}", + new RuntimeException) + } } while (!UnsafeUtils.unsafe.compareAndSwapInt(this, lockStateOffset, lockState, lockState - 1)) adjustSharedLockCount(this, -1) } From 54319e0800be29d7fd0ec6f70dca50d3babfada8 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Wed, 9 Dec 2020 17:20:32 -0800 Subject: [PATCH 28/33] bug(query): Fast Reduce closed RV in a different thread (#971) Range vector locks were released in a different thread than the one where it was acquired. This leads to lock state corruption. Closing immediately in same lambda. --- .../query/exec/AggrOverRangeVectors.scala | 30 +++++++++++-------- 1 file changed, 17 insertions(+), 13 deletions(-) diff --git a/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala b/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala index 63b0e209a6..050046ec2d 100644 --- a/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala +++ b/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala @@ -1,7 +1,5 @@ package filodb.query.exec -import scala.collection.mutable.ArrayBuffer - import com.typesafe.scalalogging.StrictLogging import monix.eval.Task import monix.reactive.Observable @@ -237,8 +235,6 @@ 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. :) @@ -246,9 +242,12 @@ object RangeVectorAggregator extends StrictLogging { source.foldLeftF(accs) { case (_, rv) => count += 1 val rowIter = rv.rows - toClose += rowIter - cforRange { 0 until outputLen } { i => - accs(i) = rowAgg.reduceAggregate(accs(i), rowIter.next) + try { + cforRange { 0 until outputLen } { i => + accs(i) = rowAgg.reduceAggregate(accs(i), rowIter.next) + } + } finally { + rowIter.close() } accs } @@ -257,19 +256,24 @@ object RangeVectorAggregator extends StrictLogging { source.foldLeftF(accs) { case (_, rv) => count += 1 val rowIter = rv.rows - toClose += rowIter - cforRange { 0 until outputLen } { i => - val mapped = rowAgg.map(rv.key, rowIter.next, mapIntos(i)) - accs(i) = rowAgg.reduceMappedRow(accs(i), mapped) + try { + cforRange { 0 until outputLen } { i => + val mapped = rowAgg.map(rv.key, rowIter.next, mapIntos(i)) + accs(i) = rowAgg.reduceMappedRow(accs(i), mapped) + } + } finally { + rowIter.close() } accs } } + // convert the aggregations to range vectors aggObs.flatMap { _ => if (count > 0) { - val iter = new CustomCloseCursor(accs.toIterator.map(_.toRowReader))(toClose.foreach(_.close())) - Observable.now(IteratorBackedRangeVector(CustomRangeVectorKey.empty, iter)) + import NoCloseCursor._ // The base range vectors are already closed, so no close propagation needed + Observable.now(IteratorBackedRangeVector(CustomRangeVectorKey.empty, + NoCloseCursor(accs.toIterator.map(_.toRowReader)))) } else { Observable.empty } From 48b341dc4832e714a5b6e4703b5a54b83ceb2736 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Wed, 9 Dec 2020 18:42:31 -0800 Subject: [PATCH 29/33] perf(query): Remove logging of plans on error to reduce logging I/O (#972) We use printTree that logs plan on errors. Now that we have original promQL query, we dont need to log full exec plan - we can log the query. ExecPlan logging is huge, esp with spread=8. This reduces logging pressure. We found that in some nodes that are backed on the scheduler, a lot of time is spent logging timeout errors --- .../scala/filodb.coordinator/QueryActor.scala | 3 ++- .../scala/filodb/query/exec/ExecPlan.scala | 18 ++++++++++++------ .../filodb/query/exec/MetadataRemoteExec.scala | 3 ++- .../filodb/query/exec/PromQlRemoteExec.scala | 11 +++++++---- 4 files changed, 23 insertions(+), 12 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala b/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala index 6883e1d81e..9ec142a1c3 100644 --- a/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala +++ b/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala @@ -145,7 +145,8 @@ final class QueryActor(memStore: MemStore, }(queryScheduler).recover { case ex => querySession.close() // Unhandled exception in query, should be rare - logger.error(s"queryId ${q.queryContext.queryId} Unhandled Query Error: ", ex) + logger.error(s"queryId ${q.queryContext.queryId} Unhandled Query Error," + + s" query was ${q.queryContext.origQueryParams}", ex) queryExecuteSpan.finish() replyTo ! QueryError(q.queryContext.queryId, ex) }(queryScheduler) diff --git a/query/src/main/scala/filodb/query/exec/ExecPlan.scala b/query/src/main/scala/filodb/query/exec/ExecPlan.scala index fa886e8103..b77f6b842e 100644 --- a/query/src/main/scala/filodb/query/exec/ExecPlan.scala +++ b/query/src/main/scala/filodb/query/exec/ExecPlan.scala @@ -162,7 +162,7 @@ trait ExecPlan extends QueryCommand { srv case rv: RangeVector => // materialize, and limit rows per RV - val srv = SerializedRangeVector(rv, builder, recSchema, printTree(false)) + val srv = SerializedRangeVector(rv, builder, recSchema, queryWithPlanName(queryContext)) numResultSamples += srv.numRowsInt // fail the query instead of limiting range vectors and returning incomplete/inaccurate results if (enforceLimit && numResultSamples > queryContext.plannerParams.sampleLimit) @@ -185,7 +185,8 @@ trait ExecPlan extends QueryCommand { // 250 RVs * (250 bytes for RV-Key + 200 samples * 32 bytes per sample) // is < 2MB qLogger.warn(s"queryId: ${queryContext.queryId} result was large size $numBytes. May need to " + - s"tweak limits. ExecPlan was: ${printTree()} ; Limit was: ${queryContext.plannerParams.sampleLimit}") + s"tweak limits. Query was: ${queryContext.origQueryParams}" + + s"; Limit was: ${queryContext.plannerParams.sampleLimit}") } span.mark(s"num-result-samples: $numResultSamples") span.mark(s"num-range-vectors: ${r.size}") @@ -196,7 +197,7 @@ trait ExecPlan extends QueryCommand { resultTask.onErrorHandle { case ex: Throwable => if (!ex.isInstanceOf[BadQueryException]) // dont log user errors qLogger.error(s"queryId: ${queryContext.queryId} Exception during execution of query: " + - s"${printTree(false)}", ex) + s"${queryContext.origQueryParams}", ex) span.fail(ex) QueryError(queryContext.queryId, ex) } @@ -204,7 +205,7 @@ trait ExecPlan extends QueryCommand { .onErrorRecover { case NonFatal(ex) => if (!ex.isInstanceOf[BadQueryException]) // dont log user errors qLogger.error(s"queryId: ${queryContext.queryId} Exception during orchestration of query:" + - s" ${printTree(false)}", ex) + s" ${queryContext.origQueryParams}", ex) QueryError(queryContext.queryId, ex) } @@ -245,6 +246,10 @@ trait ExecPlan extends QueryCommand { ((transf :+ curNode) ++ childr).mkString(if (useNewline) "\n" else " @@@ ") } + protected def queryWithPlanName(queryContext: QueryContext): String = { + s"${this.getClass.getSimpleName}-${queryContext.origQueryParams}" + } + def curNodeText(level: Int): String = s"${"-"*level}E~${getClass.getSimpleName}($args) on ${dispatcher}" @@ -314,7 +319,7 @@ final case class ExecPlanFuncArgs(execPlan: ExecPlan, timeStepParams: RangeParam Observable.fromTask( execPlan.dispatcher.dispatch(execPlan).onErrorHandle { case ex: Throwable => qLogger.error(s"queryId: ${execPlan.queryContext.queryId} Execution failed for sub-query" + - s" ${execPlan.printTree()}", ex) + s" ${execPlan.queryContext.origQueryParams}", ex) QueryError(execPlan.queryContext.queryId, ex) }.map { case QueryResult(_, _, result) => // Result is empty because of NaN so create ScalarFixedDouble with NaN @@ -372,7 +377,8 @@ abstract class NonLeafExecPlan extends ExecPlan { // Dont finish span since this code didnt create it Kamon.runWithSpan(span, false) { plan.dispatcher.dispatch(plan).onErrorHandle { case ex: Throwable => - qLogger.error(s"queryId: ${queryContext.queryId} Execution failed for sub-query ${plan.printTree()}", ex) + qLogger.error(s"queryId: ${queryContext.queryId} Execution failed for sub-query " + + s"${queryContext.origQueryParams}", ex) QueryError(queryContext.queryId, ex) } } diff --git a/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala b/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala index 408cdd3df2..477f08cd8b 100644 --- a/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala +++ b/query/src/main/scala/filodb/query/exec/MetadataRemoteExec.scala @@ -44,7 +44,8 @@ case class MetadataRemoteExec(queryEndpoint: String, val rangeVector = IteratorBackedRangeVector(new CustomRangeVectorKey(Map.empty), UTF8MapIteratorRowReader(iteratorMap.toIterator)) - val srvSeq = Seq(SerializedRangeVector(rangeVector, builder, recordSchema, printTree(false))) + val srvSeq = Seq(SerializedRangeVector(rangeVector, builder, recordSchema, + queryWithPlanName(queryContext))) QueryResult(id, resultSchema, srvSeq) } diff --git a/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala b/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala index a9ded9049b..a9cc3cbc59 100644 --- a/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala +++ b/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala @@ -113,7 +113,8 @@ case class PromQlRemoteExec(queryEndpoint: String, override def numRows: Option[Int] = Option(samples.size) } - SerializedRangeVector(rv, builder, recordSchema.get("default").get, printTree(false)) + SerializedRangeVector(rv, builder, recordSchema.get("default").get, + queryWithPlanName(queryContext)) // TODO: Handle stitching with verbose flag } QueryResult(id, resultSchema.get("default").get, rangeVectors) @@ -147,7 +148,7 @@ case class PromQlRemoteExec(queryEndpoint: String, override def numRows: Option[Int] = Option(samples.size) } - SerializedRangeVector(rv, builder, recordSchema.get("histogram").get, printTree(false)) + SerializedRangeVector(rv, builder, recordSchema.get("histogram").get, queryContext.origQueryParams.toString) // TODO: Handle stitching with verbose flag } QueryResult(id, resultSchema.get("histogram").get, rangeVectors) @@ -171,7 +172,8 @@ case class PromQlRemoteExec(queryEndpoint: String, } override def numRows: Option[Int] = Option(d.aggregateResponse.get.aggregateSampl.size) } - SerializedRangeVector(rv, builder, recordSchema.get(Avg.entryName).get, printTree(false)) + SerializedRangeVector(rv, builder, recordSchema.get(Avg.entryName).get, + queryWithPlanName(queryContext)) } // TODO: Handle stitching with verbose flag @@ -197,7 +199,8 @@ case class PromQlRemoteExec(queryEndpoint: String, } override def numRows: Option[Int] = Option(d.aggregateResponse.get.aggregateSampl.size) } - SerializedRangeVector(rv, builder, recordSchema.get(QueryFunctionConstants.stdVal).get, printTree(false)) + SerializedRangeVector(rv, builder, recordSchema.get(QueryFunctionConstants.stdVal).get, + queryWithPlanName(queryContext)) } // TODO: Handle stitching with verbose flag From 65a9c63756507c36b8275359d2086c6c14e86add Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Thu, 10 Dec 2020 10:21:58 -0800 Subject: [PATCH 30/33] bug(memory): Mask out high bit of lockState to get shared lock count (#973) Invalid warning seen because the lock loop may be observing a write lock. Mask the high bit of lockState to get shared lock count. --- memory/src/main/scala/filodb.memory/data/ChunkMap.scala | 4 ++-- 1 file changed, 2 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 5106590625..609c1391a1 100644 --- a/memory/src/main/scala/filodb.memory/data/ChunkMap.scala +++ b/memory/src/main/scala/filodb.memory/data/ChunkMap.scala @@ -105,7 +105,7 @@ object ChunkMap extends StrictLogging { var lockState = 0 do { lockState = UnsafeUtils.getIntVolatile(inst, lockStateOffset) - if (lockState - amt < 0) { + if ((lockState & Int.MaxValue) - amt < 0) { _logger.error(s"Negative lock state while releasing all shared locks for pk: $inst, amount=$amt " + s"Contents of execPlanTracker for current thread: ${execPlanTracker.get(Thread.currentThread())}", new RuntimeException) @@ -404,7 +404,7 @@ class ChunkMap(val memFactory: NativeMemoryManager, var capacity: Int) { var lockState = 0 do { lockState = UnsafeUtils.getIntVolatile(this, lockStateOffset) - if (lockState - 1 < 0) { + if ((lockState & Int.MaxValue) - 1 < 0) { _logger.error(s"Negative lock state while releasing single shared lock for pk: $this " + s"Contents of execPlanTracker for current thread: ${execPlanTracker.get(Thread.currentThread())}", new RuntimeException) From 06e42368489cb72fb812201d2cc9a6fd1bb076cf Mon Sep 17 00:00:00 2001 From: TanviBhavsar Date: Thu, 10 Dec 2020 14:35:45 -0800 Subject: [PATCH 31/33] feat(query): binary join for metrics across multiple partitions and having shard key regex (#960) --- .../queryplanner/MultiPartitionPlanner.scala | 109 ++++++++++++++---- .../queryplanner/ShardKeyRegexPlanner.scala | 33 +++++- .../MultiPartitionPlannerSpec.scala | 42 ++++++- .../ShardKeyRegexPlannerSpec.scala | 74 +++++++++--- 4 files changed, 212 insertions(+), 46 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala index 720a7dbeaf..2241ae349b 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiPartitionPlanner.scala @@ -5,7 +5,7 @@ import com.typesafe.scalalogging.StrictLogging import filodb.coordinator.queryplanner.LogicalPlanUtils._ import filodb.core.metadata.Dataset import filodb.core.query.{PromQlQueryParams, QueryConfig, QueryContext} -import filodb.query.{BinaryJoin, LabelValues, LogicalPlan, SeriesKeysByFilters} +import filodb.query.{BinaryJoin, LabelValues, LogicalPlan, SeriesKeysByFilters, SetOperator} import filodb.query.exec._ case class PartitionAssignment(partitionName: String, endPoint: String, timeRange: TimeRange) @@ -29,6 +29,8 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider val remoteHttpTimeoutMs: Long = queryConfig.routingConfig.config.as[Option[Long]]("remote.http.timeout").getOrElse(60000) + val datasetMetricColumn: String = dataset.options.metricColumn + override def materialize(logicalPlan: LogicalPlan, qContext: QueryContext): ExecPlan = { val tsdbQueryParams = qContext.origQueryParams @@ -43,14 +45,14 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider case lp: LabelValues => materializeLabelValues(lp, qContext) case lp: SeriesKeysByFilters => materializeSeriesKeysFilters(lp, qContext) case _ => materializeSimpleQuery(logicalPlan, qContext) - } } private def getRoutingKeys(logicalPlan: LogicalPlan) = { val columnFilterGroup = LogicalPlan.getColumnFilterGroup(logicalPlan) - dataset.options.nonMetricShardColumns + val routingKeys = dataset.options.nonMetricShardColumns .map(x => (x, LogicalPlan.getColumnValues(columnFilterGroup, x))) + if (routingKeys.flatMap(_._2).isEmpty) Seq.empty else routingKeys } private def generateRemoteExecParams(queryContext: QueryContext, startMs: Long, endMs: Long) = { @@ -60,14 +62,15 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider } /** - * - * @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, Set[String])], queryParams: PromQlQueryParams, - logicalPlan: LogicalPlan) = { - val routingKeyMap = routingKeys.map(x => (x._1, x._2.head)).toMap + * + * @param logicalPlan Logical plan + * @param queryParams PromQlQueryParams having query details + * @return Returns PartitionAssignment, lookback, offset and routing keys + */ + private def partitionUtilNonBinaryJoin(logicalPlan: LogicalPlan, queryParams: PromQlQueryParams) = { + + val routingKeys = getRoutingKeys(logicalPlan) + val offsetMs = LogicalPlanUtils.getOffsetMillis(logicalPlan) val periodicSeriesTimeWithOffset = TimeRange((queryParams.startSecs * 1000) - offsetMs, (queryParams.endSecs * 1000) - offsetMs) @@ -78,26 +81,53 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider val queryTimeRange = TimeRange(periodicSeriesTimeWithOffset.startMs - lookBackMs, periodicSeriesTimeWithOffset.endMs) - val partitions = partitionLocationProvider.getPartitions(routingKeyMap, queryTimeRange). - sortBy(_.timeRange.startMs) - if (partitions.isEmpty) new UnsupportedOperationException("No partitions found for routing keys: " + routingKeyMap) + val partitions = if (routingKeys.isEmpty) List.empty + else { + val routingKeyMap = routingKeys.map(x => (x._1, x._2.head)).toMap + partitionLocationProvider.getPartitions(routingKeyMap, queryTimeRange). + sortBy(_.timeRange.startMs) + } + if (partitions.isEmpty && !routingKeys.isEmpty) + new UnsupportedOperationException("No partitions found for routing keys: " + routingKeys) + + (partitions, lookBackMs, offsetMs, routingKeys) + } + /** + * @param queryParams PromQlQueryParams having query details + * @param logicalPlan Logical plan + * @return Returns PartitionAssignment and routing keys + */ + private def partitionUtil(queryParams: PromQlQueryParams, + logicalPlan: BinaryJoin): (List[PartitionAssignment], Seq[(String, Set[String])]) = { + + val lhsPartitionsAndRoutingKeys = logicalPlan.lhs match { + case b: BinaryJoin => partitionUtil(queryParams, b) + case _ => val p = partitionUtilNonBinaryJoin(logicalPlan.lhs, queryParams) + (p._1, p._4) + } - (partitions, lookBackMs, offsetMs) + val rhsPartitionsAndRoutingKeys = logicalPlan.rhs match { + case b: BinaryJoin => partitionUtil(queryParams, b) + case _ => val p = partitionUtilNonBinaryJoin(logicalPlan.rhs, queryParams) + (p._1, p._4) + } + (lhsPartitionsAndRoutingKeys._1 ++ rhsPartitionsAndRoutingKeys._1, + lhsPartitionsAndRoutingKeys._2 ++ rhsPartitionsAndRoutingKeys._2) } + /** * Materialize all queries except Binary Join and Metadata */ def materializeSimpleQuery(logicalPlan: LogicalPlan, qContext: QueryContext): ExecPlan = { - val routingKeys = getRoutingKeys(logicalPlan) + val queryParams = qContext.origQueryParams.asInstanceOf[PromQlQueryParams] + val (partitions, lookBackMs, offsetMs, routingKeys) = partitionUtilNonBinaryJoin(logicalPlan, queryParams) 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 @@ -131,16 +161,44 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider } } - def materializeBinaryJoin(logicalPlan: LogicalPlan, qContext: QueryContext): ExecPlan = { + def materializeMultiPartitionBinaryJoin(logicalPlan: BinaryJoin, qContext: QueryContext): ExecPlan = { + val lhsQueryContext = qContext.copy(origQueryParams = qContext.origQueryParams.asInstanceOf[PromQlQueryParams]. + copy(promQl = LogicalPlanParser.convertToQuery(logicalPlan.lhs))) + val rhsQueryContext = qContext.copy(origQueryParams = qContext.origQueryParams.asInstanceOf[PromQlQueryParams]. + copy(promQl = LogicalPlanParser.convertToQuery(logicalPlan.rhs))) - val routingKeys = getRoutingKeys(logicalPlan) + val lhsExec = logicalPlan.lhs match { + case b: BinaryJoin => materializeBinaryJoin(b, lhsQueryContext) + case _ => materializeSimpleQuery(logicalPlan.lhs, lhsQueryContext) + } + + val rhsExec = logicalPlan.rhs match { + case b: BinaryJoin => materializeBinaryJoin(b, rhsQueryContext) + case _ => materializeSimpleQuery(logicalPlan.rhs, rhsQueryContext) + } + + val onKeysReal = ExtraOnByKeysUtil.getRealOnLabels(logicalPlan, queryConfig.addExtraOnByKeysTimeRanges) + + if (logicalPlan.operator.isInstanceOf[SetOperator]) + SetOperatorExec(qContext, InProcessPlanDispatcher, Seq(lhsExec), Seq(rhsExec), logicalPlan.operator, + LogicalPlanUtils.renameLabels(onKeysReal, datasetMetricColumn), + LogicalPlanUtils.renameLabels(logicalPlan.ignoring, datasetMetricColumn), datasetMetricColumn) + else + BinaryJoinExec(qContext, InProcessPlanDispatcher, Seq(lhsExec), Seq(rhsExec), logicalPlan.operator, + logicalPlan.cardinality, LogicalPlanUtils.renameLabels(onKeysReal, datasetMetricColumn), + LogicalPlanUtils.renameLabels(logicalPlan.ignoring, datasetMetricColumn), + LogicalPlanUtils.renameLabels(logicalPlan.include, datasetMetricColumn), datasetMetricColumn) + + } + + def materializeBinaryJoin(logicalPlan: BinaryJoin, qContext: QueryContext): ExecPlan = { + + val queryParams = qContext.origQueryParams.asInstanceOf[PromQlQueryParams] + val (partitions, routingKeys) = partitionUtil(queryParams, 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 + // Binary Join for single partition if (partitions.forall(_.partitionName.equals((partitionName)))) { if (partitionName.equals(localPartitionName)) localPartitionPlanner.materialize(logicalPlan, qContext) else { @@ -150,10 +208,11 @@ class MultiPartitionPlanner(partitionLocationProvider: PartitionLocationProvider InProcessPlanDispatcher, dataset.ref, remoteExecHttpClient) } } - else throw new UnsupportedOperationException("Binary Join across multiple partitions not supported") + else materializeMultiPartitionBinaryJoin(logicalPlan, qContext) } } + def materializeSeriesKeysFilters(lp: SeriesKeysByFilters, qContext: QueryContext): ExecPlan = { val queryParams = qContext.origQueryParams.asInstanceOf[PromQlQueryParams] val partitions = partitionLocationProvider.getAuthorizedPartitions( diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlanner.scala index 140828da81..8a97355633 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlanner.scala @@ -1,7 +1,7 @@ package filodb.coordinator.queryplanner import filodb.core.metadata.{Dataset, Schemas} -import filodb.core.query.{ColumnFilter, PromQlQueryParams, QueryContext, RangeParams} +import filodb.core.query.{ColumnFilter, PromQlQueryParams, QueryConfig, QueryContext, RangeParams} import filodb.query._ import filodb.query.exec._ @@ -26,10 +26,13 @@ case class ShardKeyMatcher(columnFilters: Seq[ColumnFilter], query: String) class ShardKeyRegexPlanner(dataset: Dataset, queryPlanner: QueryPlanner, - shardKeyMatcher: Seq[ColumnFilter] => Seq[Seq[ColumnFilter]]) + shardKeyMatcher: Seq[ColumnFilter] => Seq[Seq[ColumnFilter]], + queryConfig: QueryConfig) extends QueryPlanner with PlannerMaterializer { + val datasetMetricColumn = dataset.options.metricColumn override val schemas = Schemas(dataset.schema) + /** * Converts a logical plan to execution plan. * @@ -81,11 +84,29 @@ class ShardKeyRegexPlanner(dataset: Dataset, /** * For binary join queries like test1{_ws_ = "demo", _ns_ =~ "App.*"} + test2{_ws_ = "demo", _ns_ =~ "App.*"}) + * LHS and RHS could be across multiple partitions */ - private def materializeBinaryJoin(binaryJoin: BinaryJoin, qContext: QueryContext): PlanResult = { - if (LogicalPlan.hasShardKeyEqualsOnly(binaryJoin, dataset.options.nonMetricShardColumns)) - PlanResult(Seq(queryPlanner.materialize(binaryJoin, qContext))) - else throw new UnsupportedOperationException("Regex not supported for Binary Join") + private def materializeBinaryJoin(logicalPlan: BinaryJoin, qContext: QueryContext): PlanResult = { + val lhsQueryContext = qContext.copy(origQueryParams = qContext.origQueryParams.asInstanceOf[PromQlQueryParams]. + copy(promQl = LogicalPlanParser.convertToQuery(logicalPlan.lhs))) + val rhsQueryContext = qContext.copy(origQueryParams = qContext.origQueryParams.asInstanceOf[PromQlQueryParams]. + copy(promQl = LogicalPlanParser.convertToQuery(logicalPlan.rhs))) + + val lhsExec = materialize(logicalPlan.lhs, lhsQueryContext) + val rhsExec = materialize(logicalPlan.rhs, rhsQueryContext) + + val onKeysReal = ExtraOnByKeysUtil.getRealOnLabels(logicalPlan, queryConfig.addExtraOnByKeysTimeRanges) + + val execPlan = if (logicalPlan.operator.isInstanceOf[SetOperator]) + SetOperatorExec(qContext, InProcessPlanDispatcher, Seq(lhsExec), Seq(rhsExec), logicalPlan.operator, + LogicalPlanUtils.renameLabels(onKeysReal, datasetMetricColumn), + LogicalPlanUtils.renameLabels(logicalPlan.ignoring, datasetMetricColumn), datasetMetricColumn) + else + BinaryJoinExec(qContext, InProcessPlanDispatcher, Seq(lhsExec), Seq(rhsExec), logicalPlan.operator, + logicalPlan.cardinality, LogicalPlanUtils.renameLabels(onKeysReal, datasetMetricColumn), + LogicalPlanUtils.renameLabels(logicalPlan.ignoring, datasetMetricColumn), + LogicalPlanUtils.renameLabels(logicalPlan.include, datasetMetricColumn), datasetMetricColumn) + PlanResult(Seq(execPlan)) } /*** diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala index 33fac5fe45..9cf3767ef7 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/MultiPartitionPlannerSpec.scala @@ -5,11 +5,11 @@ import akka.testkit.TestProbe import com.typesafe.config.ConfigFactory import org.scalatest.funspec.AnyFunSpec import org.scalatest.matchers.should.Matchers - import filodb.coordinator.ShardMapper import filodb.core.MetricsTestData import filodb.core.metadata.Schemas -import filodb.core.query.{PlannerParams, PromQlQueryParams, QueryConfig, QueryContext} +import filodb.core.query.Filter.Equals +import filodb.core.query.{ColumnFilter, PlannerParams, PromQlQueryParams, QueryConfig, QueryContext} import filodb.prometheus.ast.TimeStepParams import filodb.prometheus.parse.Parser import filodb.query.LogicalPlan @@ -471,4 +471,42 @@ class MultiPartitionPlannerSpec extends AnyFunSpec with Matchers { (endSeconds * 1000) } + + it ("should generate multipartition BinaryJoin") { + 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] = { + if (routingKey.equals(Map("job" -> "app1"))) List( + PartitionAssignment("remote", "remote-url", TimeRange(timeRange.startMs, + timeRange.endMs))) + else List( + PartitionAssignment("local", "local-url", TimeRange(timeRange.startMs, + timeRange.endMs))) + } + + override def getAuthorizedPartitions(timeRange: TimeRange): List[PartitionAssignment] = + partitions(timeRange) + } + + val engine = new MultiPartitionPlanner(partitionLocationProvider, localPlanner, "local", dataset, queryConfig) + val lp = Parser.queryRangeToLogicalPlan("""test1{job = "app1"} + test2{job = "app2"}""", + TimeStepParams(1000, 100, 10000)) + + val promQlQueryParams = PromQlQueryParams("""test1{job = "app1"} + test2{job = "app2"}""", 1000, 100, 10000) + + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams, plannerParams = + PlannerParams(processMultiPartition = true))) + execPlan.isInstanceOf[BinaryJoinExec] shouldEqual (true) + execPlan.asInstanceOf[BinaryJoinExec].lhs.head.isInstanceOf[PromQlRemoteExec] shouldEqual(true) + execPlan.asInstanceOf[BinaryJoinExec].rhs.head.isInstanceOf[LocalPartitionDistConcatExec] shouldEqual(true) + + execPlan.asInstanceOf[BinaryJoinExec].lhs.head.asInstanceOf[PromQlRemoteExec].queryContext.origQueryParams.asInstanceOf + [PromQlQueryParams].promQl shouldEqual("""test1{job="app1"}""") + + execPlan.asInstanceOf[BinaryJoinExec].rhs.head.asInstanceOf[LocalPartitionDistConcatExec].children.head. + asInstanceOf[MultiSchemaPartitionsExec].filters.contains(ColumnFilter("job", Equals("app2"))) shouldEqual(true) + + } } diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlannerSpec.scala index c8e3ae0f7b..d7ababf217 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/ShardKeyRegexPlannerSpec.scala @@ -43,7 +43,7 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-2"))))} - val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn) + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) execPlan.isInstanceOf[MultiPartitionDistConcatExec] shouldEqual(true) execPlan.children(0).children.head.isInstanceOf[MultiSchemaPartitionsExec] @@ -59,7 +59,7 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-2"))))} - val engine = new ShardKeyRegexPlanner( dataset, localPlanner, shardKeyMatcherFn) + val engine = new ShardKeyRegexPlanner( dataset, localPlanner, shardKeyMatcherFn, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = PromQlQueryParams("sum(heap_usage)", 100, 1, 1000))) execPlan.isInstanceOf[MultiPartitionReduceAggregateExec] shouldEqual(true) @@ -73,7 +73,7 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture it("should generate Exec plan for time()") { val lp = Parser.queryToLogicalPlan("time()", 1000, 1000) val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq((Seq.empty)) } - val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn) + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) execPlan.isInstanceOf[TimeScalarGeneratorExec] shouldEqual(true) } @@ -84,7 +84,7 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-2"))))} - val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn) + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = PromQlQueryParams("1 + test{_ws_ = \"demo\"," + " _ns_ =~ \"App.*\", instance = \"Inst-1\" }", 100, 1, 1000))) execPlan.isInstanceOf[MultiPartitionDistConcatExec] shouldEqual(true) @@ -109,7 +109,7 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-2"))))} - val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn) + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) execPlan.isInstanceOf[BinaryJoinExec] shouldEqual(true) } @@ -120,7 +120,7 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-2"))))} - val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn) + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) execPlan.isInstanceOf[PartKeysDistConcatExec] shouldEqual (true) } @@ -131,7 +131,7 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-2"))))} - val engine = new ShardKeyRegexPlanner( dataset, localPlanner, shardKeyMatcherFn) + val engine = new ShardKeyRegexPlanner( dataset, localPlanner, shardKeyMatcherFn, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) execPlan.isInstanceOf[MultiPartitionReduceAggregateExec] shouldEqual(true) execPlan.asInstanceOf[MultiPartitionReduceAggregateExec].rangeVectorTransformers(0). @@ -165,7 +165,7 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-2"))))} - val engine = new ShardKeyRegexPlanner( dataset, localPlanner, shardKeyMatcherFn) + val engine = new ShardKeyRegexPlanner( dataset, localPlanner, shardKeyMatcherFn, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) execPlan.isInstanceOf[MultiPartitionReduceAggregateExec] shouldEqual(true) execPlan.asInstanceOf[MultiPartitionReduceAggregateExec].rangeVectorTransformers(0). @@ -198,7 +198,7 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-2"))))} - val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn) + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) execPlan.isInstanceOf[LocalPartitionDistConcatExec] shouldEqual(true) execPlan.children(0).isInstanceOf[MultiSchemaPartitionsExec] @@ -213,7 +213,7 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-2"))))} - val engine = new ShardKeyRegexPlanner( dataset, localPlanner, shardKeyMatcherFn) + val engine = new ShardKeyRegexPlanner( dataset, localPlanner, shardKeyMatcherFn, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) execPlan.isInstanceOf[TimeScalarGeneratorExec] shouldEqual(true) execPlan.rangeVectorTransformers.head.isInstanceOf[ScalarOperationMapper] shouldEqual true @@ -237,7 +237,7 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture it ("should generate Exec plan for Metadata Label values query") { val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => Seq.empty - val engine = new ShardKeyRegexPlanner( dataset, localPlanner, shardKeyMatcherFn) + val engine = new ShardKeyRegexPlanner( dataset, localPlanner, shardKeyMatcherFn, queryConfig) val lp = Parser.labelValuesQueryToLogicalPlan(Seq("""__metric__"""), Some("""_ws_="demo", _ns_=~".*" """), TimeStepParams(1000, 20, 5000) ) @@ -250,6 +250,54 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture execPlan.isInstanceOf[LabelValuesDistConcatExec] shouldEqual (true) } + it("should generate Exec plan for Binary join with regex") { + val lp = Parser.queryToLogicalPlan("test1{_ws_ = \"demo\", _ns_ =~ \"App.*\"} + " + + "test2{_ws_ = \"demo\", _ns_ =~ \"App.*\"}", 1000, 1000) + val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { Seq(Seq(ColumnFilter("_ws_", Equals("demo")), + ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), + ColumnFilter("_ns_", Equals("App-2"))))} + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + execPlan.isInstanceOf[BinaryJoinExec] shouldEqual(true) + execPlan.children(0).isInstanceOf[MultiPartitionDistConcatExec] shouldEqual(true) + val lhs = execPlan.children(0).asInstanceOf[MultiPartitionDistConcatExec] + lhs.children.length shouldEqual 2 + lhs.children(0).children.head.isInstanceOf[MultiSchemaPartitionsExec] shouldEqual true + lhs.children(1).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", Equals("App-1"))) shouldEqual(true) + lhs.children(0).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", Equals("App-2"))) shouldEqual(true) + val rhs = execPlan.children(1).asInstanceOf[MultiPartitionDistConcatExec] + rhs.children(0).children.head.isInstanceOf[MultiSchemaPartitionsExec] shouldEqual true + rhs.children(1).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", Equals("App-1"))) shouldEqual(true) + rhs.children(0).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", Equals("App-2"))) shouldEqual(true) + } + + it("should generate Exec plan for Binary join with regex only on one side") { + val lp = Parser.queryToLogicalPlan("test1{_ws_ = \"demo\", _ns_ = \"App-0\"} + " + + "test2{_ws_ = \"demo\", _ns_ =~ \"App.*\"}", 1000, 1000) + val shardKeyMatcherFn = (shardColumnFilters: Seq[ColumnFilter]) => { + Seq(Seq(ColumnFilter("_ws_", Equals("demo")), + ColumnFilter("_ns_", Equals("App-1"))), Seq(ColumnFilter("_ws_", Equals("demo")), + ColumnFilter("_ns_", Equals("App-2")))) + } + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + execPlan.isInstanceOf[BinaryJoinExec] shouldEqual (true) + execPlan.children(0).isInstanceOf[LocalPartitionDistConcatExec] shouldEqual (true) + val lhs = execPlan.children(0).asInstanceOf[LocalPartitionDistConcatExec] + lhs.children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", Equals("App-0"))) shouldEqual (true) + val rhs = execPlan.children(1).asInstanceOf[MultiPartitionDistConcatExec] + rhs.children(0).children.head.isInstanceOf[MultiSchemaPartitionsExec] shouldEqual true + rhs.children(1).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", Equals("App-1"))) shouldEqual (true) + rhs.children(0).children.head.asInstanceOf[MultiSchemaPartitionsExec].filters. + contains(ColumnFilter("_ns_", Equals("App-2"))) shouldEqual (true) + } + it("should generate Exec plan for topk query with single matching value for regex") { val lp = Parser.queryToLogicalPlan(s"""topk(2, test{_ws_ = "demo", _ns_ =~ "App-1"})""", 1000, 1000) @@ -257,7 +305,7 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture Seq(Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-1")))) } - val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn) + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) execPlan.isInstanceOf[LocalPartitionReduceAggregateExec] shouldEqual (true) } @@ -271,7 +319,7 @@ class ShardKeyRegexPlannerSpec extends AnyFunSpec with Matchers with ScalaFuture Seq(ColumnFilter("_ws_", Equals("demo")), ColumnFilter("_ns_", Equals("App-2")))) } - val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn) + val engine = new ShardKeyRegexPlanner(dataset, localPlanner, shardKeyMatcherFn, queryConfig) the[UnsupportedOperationException] thrownBy { val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) } should have message "Shard Key regex not supported for TopK" From 25daad719bc8a6c38d933d236ad5429885afe7d1 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Thu, 10 Dec 2020 14:56:56 -0800 Subject: [PATCH 32/33] perf(query): Consolidate and reduce query exception logging further (#974) * Dont log ask timeout exceptions * For query-timeout, log message only, not stack * Consolidate error logging calls in query path --- .../scala/filodb.coordinator/QueryActor.scala | 23 +++++++++--- .../memstore/IndexBootstrapper.scala | 2 +- .../scala/filodb/query/exec/ExecPlan.scala | 35 +++++++------------ .../query/exec/InProcessPlanDispatcher.scala | 1 - 4 files changed, 31 insertions(+), 30 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala b/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala index 9ec142a1c3..947ca94d79 100644 --- a/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala +++ b/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala @@ -6,6 +6,7 @@ import java.util.concurrent.{ForkJoinPool, ForkJoinWorkerThread} import scala.util.control.NonFatal import akka.actor.{ActorRef, Props} +import akka.pattern.AskTimeoutException import kamon.Kamon import kamon.instrumentation.executor.ExecutorInstrumentation import kamon.tag.TagSet @@ -122,25 +123,37 @@ final class QueryActor(memStore: MemStore, .start() // Dont finish span since we finish it asynchronously when response is received Kamon.runWithSpan(queryExecuteSpan, false) { - Kamon.currentSpan().tag("query", q.getClass.getSimpleName) - Kamon.currentSpan().tag("query-id", q.queryContext.queryId) + queryExecuteSpan.tag("query", q.getClass.getSimpleName) + queryExecuteSpan.tag("query-id", q.queryContext.queryId) val querySession = QuerySession(q.queryContext, queryConfig) - Kamon.currentSpan().mark("query-actor-received-execute-start") + queryExecuteSpan.mark("query-actor-received-execute-start") q.execute(memStore, querySession)(queryScheduler) .foreach { res => FiloSchedulers.assertThreadName(QuerySchedName) querySession.close() - queryExecuteSpan.finish() replyTo ! res res match { case QueryResult(_, _, vectors) => resultVectors.record(vectors.length) case e: QueryError => queryErrors.increment() - logger.debug(s"queryId ${q.queryContext.queryId} Normal QueryError returned from query execution: $e") + queryExecuteSpan.fail(e.t.getMessage) + // error logging + e.t match { + case _: BadQueryException => // dont log user errors + case _: AskTimeoutException => // dont log ask timeouts. useless - let it simply flow up + case e: QueryTimeoutException => // log just message, no need for stacktrace + logger.error(s"queryId: ${q.queryContext.queryId} QueryTimeoutException: " + + s"${q.queryContext.origQueryParams} ${e.getMessage}") + case e: Throwable => + logger.error(s"queryId: ${q.queryContext.queryId} Query Error: " + + s"${q.queryContext.origQueryParams}", e) + } + // debug logging e.t match { case cve: CorruptVectorException => memStore.analyzeAndLogCorruptPtr(dsRef, cve) case t: Throwable => } + queryExecuteSpan.finish() } }(queryScheduler).recover { case ex => querySession.close() diff --git a/core/src/main/scala/filodb.core/memstore/IndexBootstrapper.scala b/core/src/main/scala/filodb.core/memstore/IndexBootstrapper.scala index 7872d90f30..67ced68b4d 100644 --- a/core/src/main/scala/filodb.core/memstore/IndexBootstrapper.scala +++ b/core/src/main/scala/filodb.core/memstore/IndexBootstrapper.scala @@ -66,7 +66,7 @@ class IndexBootstrapper(colStore: ColumnStore) { schemas: Schemas, parallelism: Int = Runtime.getRuntime.availableProcessors()) (lookUpOrAssignPartId: Array[Byte] => Int): Task[Long] = { - val recoverIndexLatency = Kamon.histogram("shard-recover-index-latency", + val recoverIndexLatency = Kamon.histogram("downsample-store-refresh-index-latency", MeasurementUnit.time.milliseconds) .withTag("dataset", ref.dataset) .withTag("shard", shardNum) diff --git a/query/src/main/scala/filodb/query/exec/ExecPlan.scala b/query/src/main/scala/filodb/query/exec/ExecPlan.scala index b77f6b842e..ef3927c9a5 100644 --- a/query/src/main/scala/filodb/query/exec/ExecPlan.scala +++ b/query/src/main/scala/filodb/query/exec/ExecPlan.scala @@ -195,17 +195,10 @@ trait ExecPlan extends QueryCommand { } } resultTask.onErrorHandle { case ex: Throwable => - if (!ex.isInstanceOf[BadQueryException]) // dont log user errors - qLogger.error(s"queryId: ${queryContext.queryId} Exception during execution of query: " + - s"${queryContext.origQueryParams}", ex) - span.fail(ex) QueryError(queryContext.queryId, ex) } }.flatten .onErrorRecover { case NonFatal(ex) => - if (!ex.isInstanceOf[BadQueryException]) // dont log user errors - qLogger.error(s"queryId: ${queryContext.queryId} Exception during orchestration of query:" + - s" ${queryContext.origQueryParams}", ex) QueryError(queryContext.queryId, ex) } @@ -318,21 +311,19 @@ final case class ExecPlanFuncArgs(execPlan: ExecPlan, timeStepParams: RangeParam override def getResult(implicit sched: Scheduler): Observable[ScalarRangeVector] = { Observable.fromTask( execPlan.dispatcher.dispatch(execPlan).onErrorHandle { case ex: Throwable => - qLogger.error(s"queryId: ${execPlan.queryContext.queryId} Execution failed for sub-query" + - s" ${execPlan.queryContext.origQueryParams}", ex) - QueryError(execPlan.queryContext.queryId, ex) - }.map { - case QueryResult(_, _, result) => // Result is empty because of NaN so create ScalarFixedDouble with NaN - if (result.isEmpty) { - ScalarFixedDouble(timeStepParams, Double.NaN) - } else { - result.head match { - case f: ScalarFixedDouble => f - case s: ScalarVaryingDouble => s + QueryError(execPlan.queryContext.queryId, ex) + }.map { + case QueryResult(_, _, result) => // Result is empty because of NaN so create ScalarFixedDouble with NaN + if (result.isEmpty) { + ScalarFixedDouble(timeStepParams, Double.NaN) + } else { + result.head match { + case f: ScalarFixedDouble => f + case s: ScalarVaryingDouble => s + } } - } - case QueryError(_, ex) => throw ex - }) + case QueryError(_, ex) => throw ex + }) } override def toString: String = execPlan.printTree() + "\n" @@ -377,8 +368,6 @@ abstract class NonLeafExecPlan extends ExecPlan { // Dont finish span since this code didnt create it Kamon.runWithSpan(span, false) { plan.dispatcher.dispatch(plan).onErrorHandle { case ex: Throwable => - qLogger.error(s"queryId: ${queryContext.queryId} Execution failed for sub-query " + - s"${queryContext.origQueryParams}", ex) QueryError(queryContext.queryId, ex) } } diff --git a/query/src/main/scala/filodb/query/exec/InProcessPlanDispatcher.scala b/query/src/main/scala/filodb/query/exec/InProcessPlanDispatcher.scala index 92333b40d4..d9aa78f688 100644 --- a/query/src/main/scala/filodb/query/exec/InProcessPlanDispatcher.scala +++ b/query/src/main/scala/filodb/query/exec/InProcessPlanDispatcher.scala @@ -37,7 +37,6 @@ case object InProcessPlanDispatcher extends PlanDispatcher { plan.execute(source, querySession) } } - } /** From d7a39b5e8bca23e9ccb4ded615e5d76d77695f39 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Thu, 10 Dec 2020 17:35:40 -0800 Subject: [PATCH 33/33] bug(coord): Finish span properly (#976) --- coordinator/src/main/scala/filodb.coordinator/QueryActor.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala b/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala index 947ca94d79..54d282efb4 100644 --- a/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala +++ b/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala @@ -153,8 +153,8 @@ final class QueryActor(memStore: MemStore, case cve: CorruptVectorException => memStore.analyzeAndLogCorruptPtr(dsRef, cve) case t: Throwable => } - queryExecuteSpan.finish() } + queryExecuteSpan.finish() }(queryScheduler).recover { case ex => querySession.close() // Unhandled exception in query, should be rare