From a8a0807e42d8ca74e29015271064b7bc6885cee7 Mon Sep 17 00:00:00 2001 From: "Brian S. O'Neill" Date: Fri, 1 May 2020 08:07:44 -0700 Subject: [PATCH 01/36] fix(core): Bring back hole expansion fix. (#740) --- .../scala/filodb.core/memstore/TimeSeriesPartition.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala index d5361eacfa..a27b1ce90e 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala @@ -387,8 +387,10 @@ extends ChunkMap(memFactory, initMapSize) with ReadablePartition { //def dataChunkPointer(id: ChunkID, columnID: Int): BinaryVector.BinaryVectorPtr = infoGet(id).vectorPtr(columnID) final def removeChunksAt(id: ChunkID): Unit = { - chunkmapWithExclusive(chunkmapDoRemove(id)) - shardStats.chunkIdsEvicted.increment() + // Remove all chunks at and lower than the given chunk. Doing so prevents a hole from + // emerging in the middle which ODP can't easily cope with. + val amt = chunkmapWithExclusive(chunkmapDoRemoveFloor(id)) + shardStats.chunkIdsEvicted.increment(amt) } final def hasChunksAt(id: ChunkID): Boolean = chunkmapContains(id) From 1ced99ac2d28a5e6f2736ea200338f9fb7f8f608 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Mon, 4 May 2020 07:23:37 -0700 Subject: [PATCH 02/36] feat(query, core): Introduce query session to store mutable state per-query (#741) QuerySession object can now be used to store per-query lock state, trace information etc. --- .../scala/filodb.coordinator/QueryActor.scala | 5 +- .../queryplanner/CompositePlanner.scala | 2 +- .../HighAvailabilityPlanner.scala | 4 +- .../queryplanner/SingleClusterPlanner.scala | 2 +- .../client/SerializationSpec.scala | 4 +- .../HighAvailabilityPlannerSpec.scala | 3 +- .../LongTimeRangePlannerSpec.scala | 8 ++- .../queryplanner/ScalarQueriesSpec.scala | 2 +- .../SingleClusterPlannerSpec.scala | 2 +- .../DownsampledTimeSeriesShard.scala | 8 ++- .../DownsampledTimeSeriesStore.scala | 12 ++-- .../memstore/OnDemandPagingShard.scala | 4 +- .../memstore/TimeSeriesMemStore.scala | 12 ++-- .../memstore/TimeSeriesShard.scala | 8 ++- .../filodb.core}/query/QueryConfig.scala | 2 +- .../filodb.core/query/QueryContext.scala | 9 ++- .../scala/filodb.core/store/ChunkSource.scala | 18 +++-- .../scala/filodb.core/store/package.scala | 8 ++- .../memstore/TimeSeriesMemStoreSpec.scala | 15 ++-- .../filodb.core/store/ColumnStoreSpec.scala | 8 ++- .../filodb.jmh/HistogramQueryBenchmark.scala | 9 +-- .../QueryHiCardInMemoryBenchmark.scala | 9 +-- .../filodb.jmh/QueryInMemoryBenchmark.scala | 14 ++-- .../query/exec/AggrOverRangeVectors.scala | 8 +-- .../filodb/query/exec/BinaryJoinExec.scala | 2 +- .../filodb/query/exec/DistConcatExec.scala | 2 +- .../filodb/query/exec/EmptyResultExec.scala | 9 +-- .../scala/filodb/query/exec/ExecPlan.scala | 16 +++-- .../query/exec/HistogramQuantileMapper.scala | 3 +- .../query/exec/InProcessPlanDispatcher.scala | 12 ++-- .../filodb/query/exec/MetadataExecPlan.scala | 8 +-- .../exec/MultiSchemaPartitionsExec.scala | 14 ++-- .../query/exec/PeriodicSamplesMapper.scala | 35 +++++----- .../scala/filodb/query/exec/PromQlExec.scala | 5 +- .../query/exec/RangeVectorTransformer.scala | 32 ++++----- .../exec/ScalarBinaryOperationExec.scala | 7 +- .../query/exec/ScalarFixedDoubleExec.scala | 10 +-- .../query/exec/SelectChunkInfosExec.scala | 7 +- .../query/exec/SelectRawPartitionsExec.scala | 8 +-- .../filodb/query/exec/SetOperatorExec.scala | 2 +- .../filodb/query/exec/StitchRvsExec.scala | 4 +- .../query/exec/TimeScalarGeneratorExec.scala | 10 +-- .../exec/rangefn/AggrOverTimeFunctions.scala | 3 +- .../query/exec/rangefn/RangeFunction.scala | 3 +- .../exec/rangefn/RangeInstantFunctions.scala | 3 +- .../query/exec/rangefn/RateFunctions.scala | 3 +- .../query/exec/AggrOverRangeVectorsSpec.scala | 4 +- .../query/exec/BinaryJoinExecSpec.scala | 17 ++--- .../query/exec/BinaryJoinGroupingSpec.scala | 21 +++--- .../exec/BinaryJoinSetOperatorSpec.scala | 56 ++++++++------- .../exec/HistToPromSeriesMapperSpec.scala | 6 +- .../exec/HistogramQuantileMapperSpec.scala | 5 +- .../exec/InProcessPlanDispatcherSpec.scala | 13 ++-- .../query/exec/LastSampleFunctionSpec.scala | 12 ++-- .../filodb/query/exec/MetadataExecSpec.scala | 13 ++-- .../exec/MultiSchemaPartitionsExecSpec.scala | 43 ++++++------ .../exec/PeriodicSamplesMapperSpec.scala | 6 +- .../query/exec/TimestampFunctionSpec.scala | 4 +- .../query/exec/WindowIteratorSpec.scala | 22 +++--- .../exec/rangefn/AbsentFunctionSpec.scala | 26 +++---- .../rangefn/AggrOverTimeFunctionsSpec.scala | 70 ++++++++++--------- .../exec/rangefn/BinaryOperatorSpec.scala | 12 ++-- .../exec/rangefn/InstantFunctionSpec.scala | 16 ++--- .../query/exec/rangefn/LabelReplaceSpec.scala | 32 +++++---- .../query/exec/rangefn/LableJoinSpec.scala | 20 +++--- .../exec/rangefn/RateFunctionsSpec.scala | 24 +++---- .../exec/rangefn/ScalarFunctionSpec.scala | 13 ++-- .../query/exec/rangefn/SortFunctionSpec.scala | 13 ++-- .../downsampler/DownsamplerMainSpec.scala | 17 ++--- 69 files changed, 459 insertions(+), 380 deletions(-) rename {query/src/main/scala/filodb => core/src/main/scala/filodb.core}/query/QueryConfig.scala (97%) diff --git a/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala b/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala index c5db07aafa..e34f36103d 100644 --- a/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala +++ b/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala @@ -17,7 +17,7 @@ import filodb.coordinator.queryplanner.SingleClusterPlanner import filodb.core._ import filodb.core.memstore.{FiloSchedulers, MemStore, TermInfo} import filodb.core.metadata.Schemas -import filodb.core.query.QueryContext +import filodb.core.query.{QueryConfig, QueryContext, QuerySession} import filodb.core.store.CorruptVectorException import filodb.query._ import filodb.query.exec.ExecPlan @@ -101,7 +101,8 @@ final class QueryActor(memStore: MemStore, epRequests.increment() Kamon.currentSpan().tag("query", q.getClass.getSimpleName) Kamon.currentSpan().tag("query-id", q.queryContext.queryId) - q.execute(memStore, queryConfig)(queryScheduler) + val querySession = QuerySession(q.queryContext, queryConfig) + q.execute(memStore, querySession)(queryScheduler) .foreach { res => FiloSchedulers.assertThreadName(QuerySchedName) replyTo ! res diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/CompositePlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/CompositePlanner.scala index 8264ab6a24..1ca64fe039 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/CompositePlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/CompositePlanner.scala @@ -6,7 +6,7 @@ import filodb.coordinator.ShardMapper import filodb.coordinator.client.QueryCommands.StaticSpreadProvider import filodb.core.{DatasetRef, SpreadProvider} import filodb.core.metadata.Schemas -import filodb.core.query.QueryContext +import filodb.core.query.{QueryConfig, QueryContext} import filodb.query._ import filodb.query.exec._ diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/HighAvailabilityPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/HighAvailabilityPlanner.scala index ae9a8e8028..faa3b0074a 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/HighAvailabilityPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/HighAvailabilityPlanner.scala @@ -3,8 +3,8 @@ package filodb.coordinator.queryplanner import com.typesafe.scalalogging.StrictLogging import filodb.core.DatasetRef -import filodb.core.query.{PromQlQueryParams, QueryContext} -import filodb.query.{LogicalPlan, QueryConfig} +import filodb.core.query.{PromQlQueryParams, QueryConfig, QueryContext} +import filodb.query.LogicalPlan import filodb.query.exec.{ExecPlan, InProcessPlanDispatcher, PromQlExec, StitchRvsExec} /** diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala index 05f54b6bdb..981a2aa97e 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala @@ -11,7 +11,7 @@ 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, QueryContext, RangeParams} +import filodb.core.query.{ColumnFilter, Filter, QueryConfig, QueryContext, RangeParams} import filodb.core.store.{AllChunkScan, ChunkScanMethod, InMemoryChunkScan, TimeRangeChunkScan, WriteBufferChunkScan} import filodb.prometheus.ast.Vectors.{PromMetricLabel, TypeLabel} import filodb.prometheus.ast.WindowConstants diff --git a/coordinator/src/test/scala/filodb.coordinator/client/SerializationSpec.scala b/coordinator/src/test/scala/filodb.coordinator/client/SerializationSpec.scala index 117a8b8faf..952bc3142a 100644 --- a/coordinator/src/test/scala/filodb.coordinator/client/SerializationSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/client/SerializationSpec.scala @@ -5,12 +5,14 @@ import akka.serialization.SerializationExtension import akka.testkit.TestProbe import com.typesafe.config.ConfigFactory import org.scalatest.concurrent.ScalaFutures + import filodb.coordinator.{ActorSpecConfig, ActorTest, ShardMapper} import filodb.coordinator.queryplanner.SingleClusterPlanner -import filodb.core.{MachineMetricsData, SpreadChange, query} +import filodb.core.{query, MachineMetricsData, SpreadChange} import filodb.core.binaryrecord2.BinaryRecordRowReader import filodb.core.metadata.{Dataset, Schemas} import filodb.core.metadata.Column.ColumnType +import filodb.core.query.QueryConfig import filodb.core.store.IngestionConfig import filodb.memory.format.{RowReader, SeqRowReader, UTF8MapIteratorRowReader, ZeroCopyUTF8String => UTF8Str} import filodb.prometheus.ast.TimeStepParams diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/HighAvailabilityPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/HighAvailabilityPlannerSpec.scala index a20de35328..cd88071a23 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/HighAvailabilityPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/HighAvailabilityPlannerSpec.scala @@ -4,10 +4,11 @@ import akka.actor.ActorSystem import akka.testkit.TestProbe import com.typesafe.config.ConfigFactory import org.scalatest.{FunSpec, Matchers} + import filodb.coordinator.ShardMapper import filodb.core.{DatasetRef, MetricsTestData} import filodb.core.metadata.Schemas -import filodb.core.query.{ColumnFilter, Filter, PromQlQueryParams, QueryContext} +import filodb.core.query.{ColumnFilter, Filter, PromQlQueryParams, QueryConfig, QueryContext} import filodb.core.store.TimeRangeChunkScan import filodb.prometheus.ast.TimeStepParams import filodb.prometheus.parse.Parser diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/LongTimeRangePlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/LongTimeRangePlannerSpec.scala index b31f3d7d33..888425f7f5 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/LongTimeRangePlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/LongTimeRangePlannerSpec.scala @@ -1,14 +1,16 @@ package filodb.coordinator.queryplanner import scala.concurrent.duration._ + import monix.execution.Scheduler import org.scalatest.{FunSpec, Matchers} + import filodb.core.DatasetRef -import filodb.core.query.QueryContext +import filodb.core.query.{QueryContext, QuerySession} import filodb.core.store.ChunkSource import filodb.prometheus.ast.TimeStepParams import filodb.prometheus.parse.Parser -import filodb.query.{LogicalPlan, PeriodicSeries, PeriodicSeriesPlan, QueryConfig} +import filodb.query.{LogicalPlan, PeriodicSeries, PeriodicSeriesPlan} import filodb.query.exec._ class LongTimeRangePlannerSpec extends FunSpec with Matchers { @@ -19,7 +21,7 @@ class LongTimeRangePlannerSpec extends FunSpec with Matchers { override def submitTime: Long = ??? override def dataset: DatasetRef = ??? override def dispatcher: PlanDispatcher = ??? - override def doExecute(source: ChunkSource, queryConfig: QueryConfig) + override def doExecute(source: ChunkSource, querySession: QuerySession) (implicit sched: Scheduler): ExecResult = ??? override protected def args: String = ??? } diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/ScalarQueriesSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/ScalarQueriesSpec.scala index 49f0f94413..7f41b3ad8f 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/ScalarQueriesSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/ScalarQueriesSpec.scala @@ -8,7 +8,7 @@ import org.scalatest.{FunSpec, Matchers} import filodb.coordinator.ShardMapper import filodb.core.MetricsTestData import filodb.core.metadata.Schemas -import filodb.core.query.{ColumnFilter, Filter, PromQlQueryParams, QueryContext, RangeParams} +import filodb.core.query.{ColumnFilter, Filter, PromQlQueryParams, QueryConfig, QueryContext, RangeParams} import filodb.prometheus.parse.Parser import filodb.query._ import filodb.query.ScalarFunctionId.Time diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/SingleClusterPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/SingleClusterPlannerSpec.scala index 03e8931f3e..6d0e4eca14 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/SingleClusterPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/SingleClusterPlannerSpec.scala @@ -12,7 +12,7 @@ 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, QueryContext} +import filodb.core.query.{ColumnFilter, Filter, PromQlQueryParams, QueryConfig, QueryContext} import filodb.core.store.TimeRangeChunkScan import filodb.prometheus.ast.{TimeStepParams, WindowConstants} import filodb.prometheus.parse.Parser diff --git a/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesShard.scala b/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesShard.scala index f3faadf9d5..d3467acfb7 100644 --- a/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesShard.scala @@ -18,7 +18,7 @@ import filodb.core.DatasetRef import filodb.core.binaryrecord2.RecordSchema import filodb.core.memstore._ import filodb.core.metadata.Schemas -import filodb.core.query.ColumnFilter +import filodb.core.query.{ColumnFilter, QuerySession} import filodb.core.store._ import filodb.memory.format.{UnsafeUtils, ZeroCopyUTF8String} @@ -206,7 +206,8 @@ class DownsampledTimeSeriesShard(rawDatasetRef: DatasetRef, def refreshPartKeyIndexBlocking(): Unit = {} def lookupPartitions(partMethod: PartitionScanMethod, - chunkMethod: ChunkScanMethod): PartLookupResult = { + chunkMethod: ChunkScanMethod, + querySession: QuerySession): PartLookupResult = { partMethod match { case SinglePartitionScan(partition, _) => throw new UnsupportedOperationException case MultiPartitionScan(partKeys, _) => throw new UnsupportedOperationException @@ -230,7 +231,8 @@ class DownsampledTimeSeriesShard(rawDatasetRef: DatasetRef, } } - def scanPartitions(lookup: PartLookupResult): Observable[ReadablePartition] = { + def scanPartitions(lookup: PartLookupResult, + querySession: QuerySession): Observable[ReadablePartition] = { // Step 1: Choose the downsample level depending on the range requested val downsampledDataset = chooseDownsampleResolution(lookup.chunkMethod) diff --git a/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesStore.scala b/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesStore.scala index 3fe1736375..548fdd4fff 100644 --- a/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesStore.scala +++ b/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesStore.scala @@ -13,7 +13,7 @@ import org.jctools.maps.NonBlockingHashMapLong import filodb.core.{DatasetRef, Response} import filodb.core.memstore._ import filodb.core.metadata.Schemas -import filodb.core.query.ColumnFilter +import filodb.core.query.{ColumnFilter, QuerySession} import filodb.core.store._ import filodb.memory.format.{UnsafeUtils, ZeroCopyUTF8String} @@ -88,25 +88,27 @@ extends MemStore with StrictLogging { def lookupPartitions(ref: DatasetRef, partMethod: PartitionScanMethod, - chunkMethod: ChunkScanMethod): PartLookupResult = { + chunkMethod: ChunkScanMethod, + querySession: QuerySession): PartLookupResult = { val shard = datasets(ref).get(partMethod.shard) if (shard == UnsafeUtils.ZeroPointer) { throw new IllegalArgumentException(s"Shard $shard of dataset $ref is not assigned to " + s"this node. Was it was recently reassigned to another node? Prolonged occurrence indicates an issue.") } - shard.lookupPartitions(partMethod, chunkMethod) + shard.lookupPartitions(partMethod, chunkMethod, querySession) } def scanPartitions(ref: DatasetRef, - lookupRes: PartLookupResult): Observable[ReadablePartition] = { + lookupRes: PartLookupResult, + querySession: QuerySession): Observable[ReadablePartition] = { val shard = datasets(ref).get(lookupRes.shard) if (shard == UnsafeUtils.ZeroPointer) { throw new IllegalArgumentException(s"Shard $shard of dataset $ref is not assigned to " + s"this node. Was it was recently reassigned to another node? Prolonged occurrence indicates an issue.") } - shard.scanPartitions(lookupRes) + shard.scanPartitions(lookupRes, querySession) } def activeShards(dataset: DatasetRef): Seq[Int] = diff --git a/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala b/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala index ced379ce90..707d5b0086 100644 --- a/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala +++ b/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala @@ -14,6 +14,7 @@ import filodb.core.DatasetRef import filodb.core.binaryrecord2.RecordSchema import filodb.core.downsample.{DownsampleConfig, DownsamplePublisher} import filodb.core.metadata.Schemas +import filodb.core.query.QuerySession import filodb.core.store._ import filodb.memory.BinaryRegionLarge import filodb.memory.MemFactory @@ -57,7 +58,8 @@ TimeSeriesShard(ref, schemas, storeConfig, shardNum, bufferMemoryManager, rawSto // 4. upload to memory and return partition // Definitely room for improvement, such as fetching multiple partitions at once, more parallelism, etc. //scalastyle:off - override def scanPartitions(partLookupRes: PartLookupResult): Observable[ReadablePartition] = { + override def scanPartitions(partLookupRes: PartLookupResult, + querySession: QuerySession): Observable[ReadablePartition] = { // For now, always read every data column. // 1. We don't have a good way to update just some columns of a chunkset for ODP // 2. Timestamp column almost always needed diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala index 1b1af9b861..8458c22eea 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala @@ -13,7 +13,7 @@ import org.jctools.maps.NonBlockingHashMapLong import filodb.core.{DatasetRef, Response} import filodb.core.downsample.{DownsampleConfig, DownsamplePublisher} import filodb.core.metadata.Schemas -import filodb.core.query.ColumnFilter +import filodb.core.query.{ColumnFilter, QuerySession} import filodb.core.store._ import filodb.memory.MemFactory import filodb.memory.NativeMemoryManager @@ -199,26 +199,28 @@ extends MemStore with StrictLogging { chunkMethod: ChunkScanMethod = AllChunkScan): Observable[RawPartData] = Observable.empty def scanPartitions(ref: DatasetRef, - iter: PartLookupResult): Observable[ReadablePartition] = { + iter: PartLookupResult, + querySession: QuerySession): Observable[ReadablePartition] = { val shard = datasets(ref).get(iter.shard) if (shard == UnsafeUtils.ZeroPointer) { throw new IllegalArgumentException(s"Shard ${iter.shard} of dataset $ref is not assigned to " + s"this node. Was it was recently reassigned to another node? Prolonged occurrence indicates an issue.") } - shard.scanPartitions(iter) + shard.scanPartitions(iter, querySession) } def lookupPartitions(ref: DatasetRef, partMethod: PartitionScanMethod, - chunkMethod: ChunkScanMethod): PartLookupResult = { + chunkMethod: ChunkScanMethod, + querySession: QuerySession): PartLookupResult = { val shard = datasets(ref).get(partMethod.shard) if (shard == UnsafeUtils.ZeroPointer) { throw new IllegalArgumentException(s"Shard ${partMethod.shard} of dataset $ref is not assigned to " + s"this node. Was it was recently reassigned to another node? Prolonged occurrence indicates an issue.") } - shard.lookupPartitions(partMethod, chunkMethod) + shard.lookupPartitions(partMethod, chunkMethod, querySession) } def numRowsIngested(dataset: DatasetRef, shard: Int): Long = diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala index 4a947b1992..6d21b651cf 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala @@ -26,7 +26,7 @@ import filodb.core.{ErrorResponse, _} import filodb.core.binaryrecord2._ import filodb.core.downsample.{DownsampleConfig, DownsamplePublisher, ShardDownsampler} import filodb.core.metadata.{Schema, Schemas} -import filodb.core.query.ColumnFilter +import filodb.core.query.{ColumnFilter, QuerySession} import filodb.core.store._ import filodb.memory._ import filodb.memory.format.{UnsafeUtils, ZeroCopyUTF8String} @@ -1405,7 +1405,8 @@ class TimeSeriesShard(val ref: DatasetRef, * Also returns detailed information about what is in memory and not, and does schema discovery. */ def lookupPartitions(partMethod: PartitionScanMethod, - chunkMethod: ChunkScanMethod): PartLookupResult = partMethod match { + chunkMethod: ChunkScanMethod, + querySession: QuerySession): PartLookupResult = partMethod match { case SinglePartitionScan(partition, _) => val partIds = debox.Buffer.empty[Int] getPartition(partition).foreach(p => partIds += p.partID) @@ -1450,7 +1451,8 @@ class TimeSeriesShard(val ref: DatasetRef, PartLookupResult(shardNum, chunkMethod, matches, _schema, startTimes, partIdsNotInMem) } - def scanPartitions(iterResult: PartLookupResult): Observable[ReadablePartition] = { + def scanPartitions(iterResult: PartLookupResult, + querySession: QuerySession): Observable[ReadablePartition] = { val partIter = new InMemPartitionIterator2(iterResult.partsInMemory) Observable.fromIterator(partIter.map { p => shardStats.partitionsQueried.increment() diff --git a/query/src/main/scala/filodb/query/QueryConfig.scala b/core/src/main/scala/filodb.core/query/QueryConfig.scala similarity index 97% rename from query/src/main/scala/filodb/query/QueryConfig.scala rename to core/src/main/scala/filodb.core/query/QueryConfig.scala index d54c006ebe..08f14e29c4 100644 --- a/query/src/main/scala/filodb/query/QueryConfig.scala +++ b/core/src/main/scala/filodb.core/query/QueryConfig.scala @@ -1,4 +1,4 @@ -package filodb.query +package filodb.core.query import scala.concurrent.duration.FiniteDuration diff --git a/core/src/main/scala/filodb.core/query/QueryContext.scala b/core/src/main/scala/filodb.core/query/QueryContext.scala index e1b0df05a3..8b3e1247b8 100644 --- a/core/src/main/scala/filodb.core/query/QueryContext.scala +++ b/core/src/main/scala/filodb.core/query/QueryContext.scala @@ -9,7 +9,7 @@ import filodb.core.{SpreadChange, SpreadProvider} trait TsdbQueryParams /** - * This class provides PromQl query paramaters + * This class provides PromQl query parameters * Config has routing parameters */ case class PromQlQueryParams(config: Config, promQl: String, startSecs: Long, stepSecs: Long, endSecs: Long, @@ -57,3 +57,10 @@ object QueryContext { simpleMapSpreadFunc(shardKeyNames.asScala, spreadAssignment, defaultSpread) } } + +case class QuerySession(qContext: QueryContext, + queryConfig: QueryConfig) + +object QuerySession { + def forTestingOnly: QuerySession = QuerySession(QueryContext(), EmptyQueryConfig) +} \ No newline at end of file diff --git a/core/src/main/scala/filodb.core/store/ChunkSource.scala b/core/src/main/scala/filodb.core/store/ChunkSource.scala index e04284dc73..b42161fb0b 100644 --- a/core/src/main/scala/filodb.core/store/ChunkSource.scala +++ b/core/src/main/scala/filodb.core/store/ChunkSource.scala @@ -78,16 +78,18 @@ trait ChunkSource extends RawChunkSource with StrictLogging { def scanPartitions(ref: DatasetRef, columnIDs: Seq[Types.ColumnId], partMethod: PartitionScanMethod, - chunkMethod: ChunkScanMethod = AllChunkScan): Observable[ReadablePartition] = { + chunkMethod: ChunkScanMethod = AllChunkScan, + querySession: QuerySession): Observable[ReadablePartition] = { logger.debug(s"scanPartitions dataset=$ref shard=${partMethod.shard} " + s"partMethod=$partMethod chunkMethod=$chunkMethod") - scanPartitions(ref, lookupPartitions(ref, partMethod, chunkMethod)) + scanPartitions(ref, lookupPartitions(ref, partMethod, chunkMethod, querySession), querySession) } // Internal API that needs to actually be implemented def scanPartitions(ref: DatasetRef, - lookupRes: PartLookupResult): Observable[ReadablePartition] + lookupRes: PartLookupResult, + querySession: QuerySession): Observable[ReadablePartition] // internal method to find # of groups in a dataset def groupsInDataset(ref: DatasetRef): Int @@ -108,7 +110,8 @@ trait ChunkSource extends RawChunkSource with StrictLogging { */ def lookupPartitions(ref: DatasetRef, partMethod: PartitionScanMethod, - chunkMethod: ChunkScanMethod): PartLookupResult + chunkMethod: ChunkScanMethod, + querySession: QuerySession): PartLookupResult /** * Returns a stream of RangeVectors's. Good for per-partition (or time series) processing. @@ -124,18 +127,19 @@ trait ChunkSource extends RawChunkSource with StrictLogging { lookupRes: PartLookupResult, columnIDs: Seq[Types.ColumnId], schema: Schema, - filterSchemas: Boolean): Observable[RangeVector] = { + filterSchemas: Boolean, + querySession: QuerySession): Observable[RangeVector] = { val ids = columnIDs.toArray val partCols = schema.infosFromIDs(schema.partition.columns.map(_.id)) val numGroups = groupsInDataset(ref) val filteredParts = if (filterSchemas) { - scanPartitions(ref, lookupRes) + scanPartitions(ref, lookupRes, querySession) .filter { p => p.schema.schemaHash == schema.schemaHash && p.hasChunks(lookupRes.chunkMethod) } } else { lookupRes.firstSchemaId match { case Some(reqSchemaId) => - scanPartitions(ref, lookupRes).filter { p => + scanPartitions(ref, lookupRes, querySession).filter { p => if (p.schema.schemaHash != reqSchemaId) throw SchemaMismatch(Schemas.global.schemaName(reqSchemaId), p.schema.name) p.hasChunks(lookupRes.chunkMethod) diff --git a/core/src/main/scala/filodb.core/store/package.scala b/core/src/main/scala/filodb.core/store/package.scala index 666e7f6d40..a7ab5cef7e 100644 --- a/core/src/main/scala/filodb.core/store/package.scala +++ b/core/src/main/scala/filodb.core/store/package.scala @@ -6,6 +6,7 @@ import net.jpountz.lz4.{LZ4Compressor, LZ4Factory, LZ4FastDecompressor} import filodb.core.Types._ import filodb.core.metadata.Dataset +import filodb.core.query.QuerySession import filodb.memory.format.{RowReader, UnsafeUtils} package object store { @@ -138,6 +139,8 @@ package object store { * Convenience method to scan/iterate over all rows of given selection of source data. You must iterate * through all the elements. * + * Used for testing only. + * * @param dataset the Dataset to read from * @param columnIDs the set of column IDs to read back. Order determines the order of columns read back * in each row. These are the IDs from the Column instances. @@ -145,8 +148,9 @@ package object store { def scanRows(dataset: Dataset, columnIDs: Seq[ColumnId], partMethod: PartitionScanMethod, - chunkMethod: ChunkScanMethod = AllChunkScan): Iterator[RowReader] = - source.scanPartitions(dataset.ref, columnIDs, partMethod, chunkMethod) + chunkMethod: ChunkScanMethod = AllChunkScan, + querySession: QuerySession = QuerySession.forTestingOnly): Iterator[RowReader] = + source.scanPartitions(dataset.ref, columnIDs, partMethod, chunkMethod, querySession) .toIterator() .flatMap(_.timeRangeRows(chunkMethod, columnIDs.toArray)) } diff --git a/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala b/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala index 46f70c2153..7252773c99 100644 --- a/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala +++ b/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala @@ -13,7 +13,7 @@ import org.scalatest.time.{Millis, Seconds, Span} import filodb.core._ import filodb.core.binaryrecord2.RecordBuilder import filodb.core.metadata.Schemas -import filodb.core.query.{ColumnFilter, Filter} +import filodb.core.query.{ColumnFilter, Filter, QuerySession} import filodb.core.store._ import filodb.memory.format.{UnsafeUtils, ZeroCopyUTF8String} import filodb.memory.format.vectors.LongHistogram @@ -46,6 +46,7 @@ class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter w } } + // Look mama! Real-time time series ingestion and querying across multiple partitions! it("should ingest into multiple series and be able to query across all partitions in real time") { memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf) @@ -345,7 +346,8 @@ class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter w val filter = ColumnFilter("n", Filter.Equals("2".utf8)) val range = TimeRangeChunkScan(105000L, 2000000L) - val res = memStore.lookupPartitions(dataset2.ref, FilteredPartitionScan(split, Seq(filter)), range) + val res = memStore.lookupPartitions(dataset2.ref, FilteredPartitionScan(split, Seq(filter)), range, + QuerySession.forTestingOnly) res.firstSchemaId shouldEqual Some(schema2.schemaHash) res.partsInMemory.length shouldEqual 2 // two partitions should match res.shard shouldEqual 0 @@ -451,7 +453,8 @@ class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter w memStore.getShardE(dataset1.ref, 0).partKeyIndex.indexNumEntries shouldEqual 22 val split = memStore.getScanSplits(dataset1.ref, 1).head - val parts = memStore.scanPartitions(dataset1.ref, Seq(0, 1), FilteredPartitionScan(split)) + val parts = memStore.scanPartitions(dataset1.ref, Seq(0, 1), FilteredPartitionScan(split), + querySession = QuerySession.forTestingOnly) .toListL.runAsync .futureValue .asInstanceOf[Seq[TimeSeriesPartition]] @@ -487,7 +490,8 @@ class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter w // one more part val split = memStore.getScanSplits(dataset1.ref, 1).head val filter = ColumnFilter("series", Filter.Equals("Series 0".utf8)) - val parts = memStore.scanPartitions(dataset1.ref, Seq(0, 1), FilteredPartitionScan(split, Seq(filter))) + val parts = memStore.scanPartitions(dataset1.ref, Seq(0, 1), FilteredPartitionScan(split, Seq(filter)), + querySession = QuerySession.forTestingOnly) .toListL.runAsync .futureValue .asInstanceOf[Seq[TimeSeriesPartition]] @@ -564,7 +568,8 @@ class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter w memStore.refreshIndexForTesting(dataset1.ref) // Check partitions are now 0 to 20, 21/22 did not get added val split = memStore.getScanSplits(dataset1.ref, 1).head - val parts = memStore.scanPartitions(dataset1.ref, Seq(0, 1), FilteredPartitionScan(split)) + val parts = memStore.scanPartitions(dataset1.ref, Seq(0, 1), FilteredPartitionScan(split), + querySession = QuerySession.forTestingOnly) .toListL.runAsync .futureValue .asInstanceOf[Seq[TimeSeriesPartition]] diff --git a/core/src/test/scala/filodb.core/store/ColumnStoreSpec.scala b/core/src/test/scala/filodb.core/store/ColumnStoreSpec.scala index e896aa6f5a..81133c5728 100644 --- a/core/src/test/scala/filodb.core/store/ColumnStoreSpec.scala +++ b/core/src/test/scala/filodb.core/store/ColumnStoreSpec.scala @@ -12,7 +12,7 @@ import org.scalatest.time.{Millis, Seconds, Span} import filodb.core._ import filodb.core.memstore.{FixedMaxPartitionsEvictionPolicy, TimeSeriesMemStore} import filodb.core.metadata.Schemas -import filodb.core.query.{ColumnFilter, Filter} +import filodb.core.query.{ColumnFilter, Filter, QuerySession} // TODO: figure out what to do with this.. most of the tests are really irrelevant trait ColumnStoreSpec extends FlatSpec with Matchers @@ -151,9 +151,11 @@ with BeforeAndAfter with BeforeAndAfterAll with ScalaFutures { val filter = ColumnFilter("MonthYear", Filter.Equals(197902)) val method = FilteredPartitionScan(paramSet.head, Seq(filter)) - val lookupRes = memStore.lookupPartitions(dataset2.ref, method, AllChunkScan) + val lookupRes = memStore.lookupPartitions(dataset2.ref, method, AllChunkScan, + querySession = QuerySession.forTestingOnly) val rangeVectorObs = memStore.rangeVectors(dataset2.ref, lookupRes, schema2.colIDs("NumArticles").get, - schema2, false) + schema2, false, + QuerySession.forTestingOnly) val rangeVectors = rangeVectorObs.toListL.runAsync.futureValue rangeVectors should have length (1) diff --git a/jmh/src/main/scala/filodb.jmh/HistogramQueryBenchmark.scala b/jmh/src/main/scala/filodb.jmh/HistogramQueryBenchmark.scala index 3ac8a4eace..85dd75971a 100644 --- a/jmh/src/main/scala/filodb.jmh/HistogramQueryBenchmark.scala +++ b/jmh/src/main/scala/filodb.jmh/HistogramQueryBenchmark.scala @@ -19,12 +19,11 @@ 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.QueryContext +import filodb.core.query.{QueryConfig, QueryContext, QuerySession} import filodb.core.store._ import filodb.memory.MemFactory import filodb.memory.format.SeqRowReader import filodb.prometheus.parse.Parser -import filodb.query.QueryConfig //scalastyle:off regex /** @@ -113,7 +112,8 @@ class HistogramQueryBenchmark { @OperationsPerInvocation(500) def histSchemaQuantileQuery(): Long = { val f = Observable.fromIterable(0 until numQueries).mapAsync(1) { n => - hExecPlan.execute(memStore, queryConfig)(querySched) + val querySession = QuerySession(QueryContext(), queryConfig) + hExecPlan.execute(memStore, querySession)(querySched) }.executeOn(querySched) .countL.runAsync Await.result(f, 60.seconds) @@ -125,7 +125,8 @@ class HistogramQueryBenchmark { @OperationsPerInvocation(500) def promSchemaQuantileQuery(): Long = { val f = Observable.fromIterable(0 until numQueries).mapAsync(1) { n => - pExecPlan.execute(memStore, queryConfig)(querySched) + val querySession = QuerySession(QueryContext(), queryConfig) + pExecPlan.execute(memStore, querySession)(querySched) }.executeOn(querySched) .countL.runAsync Await.result(f, 60.seconds) diff --git a/jmh/src/main/scala/filodb.jmh/QueryHiCardInMemoryBenchmark.scala b/jmh/src/main/scala/filodb.jmh/QueryHiCardInMemoryBenchmark.scala index c6c3974a20..11a1f3d839 100644 --- a/jmh/src/main/scala/filodb.jmh/QueryHiCardInMemoryBenchmark.scala +++ b/jmh/src/main/scala/filodb.jmh/QueryHiCardInMemoryBenchmark.scala @@ -18,11 +18,10 @@ import filodb.core.SpreadChange import filodb.core.binaryrecord2.RecordContainer import filodb.core.memstore.{SomeData, TimeSeriesMemStore} import filodb.core.metadata.Schemas -import filodb.core.query.QueryContext +import filodb.core.query.{QueryConfig, QueryContext, QuerySession} import filodb.core.store.StoreConfig import filodb.prometheus.ast.TimeStepParams import filodb.prometheus.parse.Parser -import filodb.query.QueryConfig import filodb.query.exec.ExecPlan import filodb.timeseries.TestTimeseriesProducer @@ -133,7 +132,8 @@ class QueryHiCardInMemoryBenchmark extends StrictLogging { @OperationsPerInvocation(100) def scanSumOfRateBenchmark(): Unit = { (0 until numQueries).foreach { _ => - Await.result(scanSumOfRate.execute(store, queryConfig).runAsync, 60.seconds) + val querySession = QuerySession(QueryContext(), queryConfig) + Await.result(scanSumOfRate.execute(store, querySession).runAsync, 60.seconds) } } @@ -144,7 +144,8 @@ class QueryHiCardInMemoryBenchmark extends StrictLogging { @OperationsPerInvocation(100) def scanSumOfSumOverTimeBenchmark(): Unit = { (0 until numQueries).foreach { _ => - Await.result(scanSumSumOverTime.execute(store, queryConfig).runAsync, 60.seconds) + val querySession = QuerySession(QueryContext(), queryConfig) + Await.result(scanSumSumOverTime.execute(store, querySession).runAsync, 60.seconds) } } diff --git a/jmh/src/main/scala/filodb.jmh/QueryInMemoryBenchmark.scala b/jmh/src/main/scala/filodb.jmh/QueryInMemoryBenchmark.scala index 84041eec5c..4a66a75793 100644 --- a/jmh/src/main/scala/filodb.jmh/QueryInMemoryBenchmark.scala +++ b/jmh/src/main/scala/filodb.jmh/QueryInMemoryBenchmark.scala @@ -19,11 +19,11 @@ import filodb.core.SpreadChange import filodb.core.binaryrecord2.RecordContainer import filodb.core.memstore.{SomeData, TimeSeriesMemStore} import filodb.core.metadata.Schemas -import filodb.core.query.QueryContext +import filodb.core.query.{QueryConfig, QueryContext, QuerySession} import filodb.core.store.StoreConfig import filodb.prometheus.ast.TimeStepParams import filodb.prometheus.parse.Parser -import filodb.query.{QueryConfig, QueryError => QError, QueryResult => QueryResult2} +import filodb.query.{QueryError => QError, QueryResult => QueryResult2} import filodb.timeseries.TestTimeseriesProducer //scalastyle:off regex @@ -186,8 +186,10 @@ class QueryInMemoryBenchmark extends StrictLogging { @OutputTimeUnit(TimeUnit.SECONDS) @OperationsPerInvocation(500) def singleThreadedRawQuery(): Long = { + val querySession = QuerySession(QueryContext(), queryConfig) + val f = Observable.fromIterable(0 until numQueries).mapAsync(1) { n => - execPlan.execute(cluster.memStore, queryConfig)(querySched) + execPlan.execute(cluster.memStore, querySession)(querySched) }.executeOn(querySched) .countL.runAsync Await.result(f, 60.seconds) @@ -203,7 +205,8 @@ class QueryInMemoryBenchmark extends StrictLogging { @OperationsPerInvocation(500) def singleThreadedMinOverTimeQuery(): Long = { val f = Observable.fromIterable(0 until numQueries).mapAsync(1) { n => - minEP.execute(cluster.memStore, queryConfig)(querySched) + val querySession = QuerySession(QueryContext(), queryConfig) + minEP.execute(cluster.memStore, querySession)(querySched) }.executeOn(querySched) .countL.runAsync Await.result(f, 60.seconds) @@ -219,7 +222,8 @@ class QueryInMemoryBenchmark extends StrictLogging { @OperationsPerInvocation(500) def singleThreadedSumRateCCQuery(): Long = { val f = Observable.fromIterable(0 until numQueries).mapAsync(1) { n => - sumRateEP.execute(cluster.memStore, queryConfig)(querySched) + val querySession = QuerySession(QueryContext(), queryConfig) + sumRateEP.execute(cluster.memStore, querySession)(querySched) }.executeOn(querySched) .countL.runAsync Await.result(f, 60.seconds) diff --git a/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala b/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala index 496a0e2d4e..1ec4cd38fe 100644 --- a/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala +++ b/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala @@ -25,7 +25,7 @@ final case class ReduceAggregateExec(queryContext: QueryContext, protected def compose(childResponses: Observable[(QueryResponse, Int)], firstSchema: Task[ResultSchema], - queryConfig: QueryConfig): Observable[RangeVector] = { + querySession: QuerySession): Observable[RangeVector] = { val results = childResponses.flatMap { case (QueryResult(_, _, result), _) => Observable.fromIterable(result) case (QueryError(_, ex), _) => throw ex @@ -55,7 +55,7 @@ final case class AggregateMapReduce(aggrOp: AggregationOperator, s"aggrOp=$aggrOp, aggrParams=$aggrParams, without=$without, by=$by" def apply(source: Observable[RangeVector], - queryConfig: QueryConfig, + querySession: QuerySession, limit: Int, sourceSchema: ResultSchema, paramResponse: Seq[Observable[ScalarRangeVector]] = Nil): Observable[RangeVector] = { @@ -71,7 +71,7 @@ final case class AggregateMapReduce(aggrOp: AggregationOperator, // IF no grouping is done AND prev transformer is Periodic (has fixed length), use optimal path if (without.isEmpty && by.isEmpty && sourceSchema.fixedVectorLen.isDefined) { - sourceSchema.fixedVectorLen.filter(_ <= queryConfig.fastReduceMaxWindows).map { numWindows => + sourceSchema.fixedVectorLen.filter(_ <= querySession.queryConfig.fastReduceMaxWindows).map { numWindows => RangeVectorAggregator.fastReduce(aggregator, false, source, numWindows) }.getOrElse { RangeVectorAggregator.mapReduce(aggregator, skipMapPhase = false, source, grouping) @@ -95,7 +95,7 @@ final case class AggregatePresenter(aggrOp: AggregationOperator, protected[exec] def args: String = s"aggrOp=$aggrOp, aggrParams=$aggrParams" def apply(source: Observable[RangeVector], - queryConfig: QueryConfig, + querySession: QuerySession, limit: Int, sourceSchema: ResultSchema, paramResponse: Seq[Observable[ScalarRangeVector]]): Observable[RangeVector] = { diff --git a/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala b/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala index b63e76240b..59277a6676 100644 --- a/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala +++ b/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala @@ -63,7 +63,7 @@ final case class BinaryJoinExec(queryContext: QueryContext, protected[exec] def compose(childResponses: Observable[(QueryResponse, Int)], firstSchema: Task[ResultSchema], - queryConfig: QueryConfig): Observable[RangeVector] = { + querySession: QuerySession): Observable[RangeVector] = { val taskOfResults = childResponses.map { case (QueryResult(_, _, result), i) => (result, i) case (QueryError(_, ex), _) => throw ex diff --git a/query/src/main/scala/filodb/query/exec/DistConcatExec.scala b/query/src/main/scala/filodb/query/exec/DistConcatExec.scala index cab9ebd719..d32169d621 100644 --- a/query/src/main/scala/filodb/query/exec/DistConcatExec.scala +++ b/query/src/main/scala/filodb/query/exec/DistConcatExec.scala @@ -18,7 +18,7 @@ final case class DistConcatExec(queryContext: QueryContext, protected def compose(childResponses: Observable[(QueryResponse, Int)], firstSchema: Task[ResultSchema], - queryConfig: QueryConfig): Observable[RangeVector] = { + querySession: QuerySession): Observable[RangeVector] = { childResponses.flatMap { case (QueryResult(_, _, result), _) => Observable.fromIterable(result) case (QueryError(_, ex), _) => throw ex diff --git a/query/src/main/scala/filodb/query/exec/EmptyResultExec.scala b/query/src/main/scala/filodb/query/exec/EmptyResultExec.scala index 44f91e8863..fe8103ed05 100644 --- a/query/src/main/scala/filodb/query/exec/EmptyResultExec.scala +++ b/query/src/main/scala/filodb/query/exec/EmptyResultExec.scala @@ -5,16 +5,16 @@ import monix.execution.Scheduler import filodb.core.DatasetRef import filodb.core.metadata.Column.ColumnType -import filodb.core.query.{ColumnInfo, QueryContext, ResultSchema} +import filodb.core.query.{ColumnInfo, QueryContext, QuerySession, ResultSchema} import filodb.core.store.ChunkSource -import filodb.query.{QueryConfig, QueryResponse, QueryResult} +import filodb.query.{QueryResponse, QueryResult} case class EmptyResultExec(queryContext: QueryContext, dataset: DatasetRef) extends LeafExecPlan { override def dispatcher: PlanDispatcher = InProcessPlanDispatcher override def execute(source: ChunkSource, - queryConfig: QueryConfig) + querySession: QuerySession) (implicit sched: Scheduler): Task[QueryResponse] = { Task(QueryResult(queryContext.queryId, new ResultSchema(Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), @@ -22,7 +22,8 @@ case class EmptyResultExec(queryContext: QueryContext, Seq.empty)) } - override def doExecute(source: ChunkSource, queryConfig: QueryConfig) + override def doExecute(source: ChunkSource, + querySession: QuerySession) (implicit sched: Scheduler): ExecResult = ??? override protected def args: String = "" diff --git a/query/src/main/scala/filodb/query/exec/ExecPlan.scala b/query/src/main/scala/filodb/query/exec/ExecPlan.scala index 4004f3a8e2..9ed7f309d8 100644 --- a/query/src/main/scala/filodb/query/exec/ExecPlan.scala +++ b/query/src/main/scala/filodb/query/exec/ExecPlan.scala @@ -92,7 +92,8 @@ trait ExecPlan extends QueryCommand { * */ // scalastyle:off method.length - def execute(source: ChunkSource, queryConfig: QueryConfig) + def execute(source: ChunkSource, + querySession: QuerySession) (implicit sched: Scheduler): Task[QueryResponse] = { val parentSpan = Kamon.currentSpan() @@ -110,7 +111,7 @@ trait ExecPlan extends QueryCommand { // 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) { - doExecute(source, queryConfig) + doExecute(source, querySession) } } @@ -132,7 +133,8 @@ 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, queryConfig, queryContext.sampleLimit, acc._2, paramRangeVector), transf.schema(acc._2)) + (transf.apply(acc._1, querySession, queryContext.sampleLimit, acc._2, + paramRangeVector), transf.schema(acc._2)) } val recSchema = SerializedRangeVector.toSchema(finalRes._2.columns, finalRes._2.brSchemas) val builder = SerializedRangeVector.newBuilder() @@ -204,7 +206,7 @@ trait ExecPlan extends QueryCommand { * Note that this should not include any operations done in the transformers. */ def doExecute(source: ChunkSource, - queryConfig: QueryConfig) + querySession: QuerySession) (implicit sched: Scheduler): ExecResult /** @@ -363,7 +365,7 @@ abstract class NonLeafExecPlan extends ExecPlan { * from the non-empty results. */ final def doExecute(source: ChunkSource, - queryConfig: QueryConfig) + querySession: QuerySession) (implicit sched: Scheduler): ExecResult = { val parentSpan = Kamon.currentSpan() parentSpan.mark("create-child-tasks") @@ -390,7 +392,7 @@ abstract class NonLeafExecPlan extends ExecPlan { case (QueryResult(_, schema, _), _) => schema }.firstOptionL.map(_.getOrElse(ResultSchema.empty)) parentSpan.mark("output-compose") - val outputRvs = compose(processedTasks, outputSchema, queryConfig) + val outputRvs = compose(processedTasks, outputSchema, querySession) parentSpan.mark("return-results") ExecResult(outputRvs, outputSchema) } @@ -422,6 +424,6 @@ abstract class NonLeafExecPlan extends ExecPlan { */ protected def compose(childResponses: Observable[(QueryResponse, Int)], firstSchema: Task[ResultSchema], - queryConfig: QueryConfig): Observable[RangeVector] + querySession: QuerySession): Observable[RangeVector] } diff --git a/query/src/main/scala/filodb/query/exec/HistogramQuantileMapper.scala b/query/src/main/scala/filodb/query/exec/HistogramQuantileMapper.scala index 424334a6d8..3386f36098 100644 --- a/query/src/main/scala/filodb/query/exec/HistogramQuantileMapper.scala +++ b/query/src/main/scala/filodb/query/exec/HistogramQuantileMapper.scala @@ -7,7 +7,6 @@ import scalaxy.loops._ import filodb.core.query._ import filodb.memory.format.{RowReader, ZeroCopyUTF8String} import filodb.memory.format.vectors.Histogram -import filodb.query.QueryConfig object HistogramQuantileMapper { import ZeroCopyUTF8String._ @@ -47,7 +46,7 @@ final case class HistogramQuantileMapper(funcParams: Seq[FuncArgs]) extends Rang * be preceded by a rate function or a sum-of-rate function. */ override def apply(source: Observable[RangeVector], - queryConfig: QueryConfig, + querySession: QuerySession, limit: Int, sourceSchema: ResultSchema, paramResponse: Seq[Observable[ScalarRangeVector]]): Observable[RangeVector] = { diff --git a/query/src/main/scala/filodb/query/exec/InProcessPlanDispatcher.scala b/query/src/main/scala/filodb/query/exec/InProcessPlanDispatcher.scala index 5443360877..f9f7c4cfe0 100644 --- a/query/src/main/scala/filodb/query/exec/InProcessPlanDispatcher.scala +++ b/query/src/main/scala/filodb/query/exec/InProcessPlanDispatcher.scala @@ -8,8 +8,9 @@ import monix.reactive.Observable import filodb.core.DatasetRef import filodb.core.memstore.PartLookupResult import filodb.core.metadata.Schemas +import filodb.core.query.{EmptyQueryConfig, QueryConfig, QuerySession} import filodb.core.store._ -import filodb.query.{EmptyQueryConfig, QueryConfig, QueryResponse} +import filodb.query.QueryResponse /** * Dispatcher which will make a No-Op style call to ExecPlan#excecute(). @@ -30,7 +31,8 @@ case object InProcessPlanDispatcher extends PlanDispatcher { // kamon uses thread-locals. Kamon.runWithSpan(Kamon.currentSpan(), false) { // translate implicit ExecutionContext to monix.Scheduler - plan.execute(source, queryConfig) + val querySession = QuerySession(plan.queryContext, queryConfig) + plan.execute(source, querySession) } } @@ -41,12 +43,14 @@ case object InProcessPlanDispatcher extends PlanDispatcher { */ case class UnsupportedChunkSource() extends ChunkSource { def scanPartitions(ref: DatasetRef, - iter: PartLookupResult): Observable[ReadablePartition] = + iter: PartLookupResult, + querySession: QuerySession): Observable[ReadablePartition] = throw new UnsupportedOperationException("This operation is not supported") def lookupPartitions(ref: DatasetRef, partMethod: PartitionScanMethod, - chunkMethod: ChunkScanMethod): PartLookupResult = + chunkMethod: ChunkScanMethod, + querySession: QuerySession): PartLookupResult = throw new UnsupportedOperationException("This operation is not supported") override def groupsInDataset(dataset: DatasetRef): Int = diff --git a/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala b/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala index 03a369449d..b5d75b4fd6 100644 --- a/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala +++ b/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala @@ -36,7 +36,7 @@ final case class PartKeysDistConcatExec(queryContext: QueryContext, */ protected def compose(childResponses: Observable[(QueryResponse, Int)], firstSchema: Task[ResultSchema], - queryConfig: QueryConfig): Observable[RangeVector] = { + querySession: QuerySession): Observable[RangeVector] = { qLogger.debug(s"NonLeafMetadataExecPlan: Concatenating results") val taskOfResults = childResponses.map { case (QueryResult(_, _, result), _) => result @@ -68,7 +68,7 @@ final case class LabelValuesDistConcatExec(queryContext: QueryContext, */ protected def compose(childResponses: Observable[(QueryResponse, Int)], firstSchema: Task[ResultSchema], - queryConfig: QueryConfig): Observable[RangeVector] = { + querySession: QuerySession): Observable[RangeVector] = { qLogger.debug(s"NonLeafMetadataExecPlan: Concatenating results") val taskOfResults = childResponses.map { case (QueryResult(_, _, result), _) => result @@ -108,7 +108,7 @@ final case class PartKeysExec(queryContext: QueryContext, override def enforceLimit: Boolean = false def doExecute(source: ChunkSource, - queryConfig: QueryConfig) + querySession: QuerySession) (implicit sched: Scheduler): ExecResult = { val rvs = source match { case memStore: MemStore => @@ -140,7 +140,7 @@ final case class LabelValuesExec(queryContext: QueryContext, override def enforceLimit: Boolean = false def doExecute(source: ChunkSource, - queryConfig: QueryConfig) + querySession: QuerySession) (implicit sched: Scheduler): ExecResult = { val parentSpan = Kamon.currentSpan() val rvs = if (source.isInstanceOf[MemStore]) { diff --git a/query/src/main/scala/filodb/query/exec/MultiSchemaPartitionsExec.scala b/query/src/main/scala/filodb/query/exec/MultiSchemaPartitionsExec.scala index 3d2228501f..a612ba5c10 100644 --- a/query/src/main/scala/filodb/query/exec/MultiSchemaPartitionsExec.scala +++ b/query/src/main/scala/filodb/query/exec/MultiSchemaPartitionsExec.scala @@ -5,10 +5,9 @@ import monix.execution.Scheduler import filodb.core.{DatasetRef, QueryTimeoutException} import filodb.core.metadata.Schemas -import filodb.core.query.{ColumnFilter, QueryContext} +import filodb.core.query.{ColumnFilter, QueryContext, QuerySession} import filodb.core.store._ import filodb.query.Query.qLogger -import filodb.query.QueryConfig final case class UnknownSchemaQueryErr(id: Int) extends Exception(s"Unknown schema ID $id during query. This likely means a schema config change happened and " + @@ -39,10 +38,11 @@ final case class MultiSchemaPartitionsExec(queryContext: QueryContext, var finalPlan: ExecPlan = _ - private def finalizePlan(source: ChunkSource): ExecPlan = { + private def finalizePlan(source: ChunkSource, + querySession: QuerySession): ExecPlan = { val partMethod = FilteredPartitionScan(ShardSplit(shard), filters) Kamon.currentSpan().mark("filtered-partition-scan") - val lookupRes = source.lookupPartitions(dataset, partMethod, chunkMethod) + val lookupRes = source.lookupPartitions(dataset, partMethod, chunkMethod, querySession) Kamon.currentSpan().mark("lookup-partitions-done") val queryTimeElapsed = System.currentTimeMillis() - queryContext.submitTime @@ -83,10 +83,10 @@ final case class MultiSchemaPartitionsExec(queryContext: QueryContext, } def doExecute(source: ChunkSource, - queryConfig: QueryConfig) + querySession: QuerySession) (implicit sched: Scheduler): ExecResult = { - finalPlan = finalizePlan(source) - finalPlan.doExecute(source, queryConfig)(sched) + finalPlan = finalizePlan(source, querySession) + finalPlan.doExecute(source, querySession)(sched) } protected def args: String = s"dataset=$dataset, shard=$shard, " + diff --git a/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala b/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala index 48eafbfdbf..cc0a86cc59 100644 --- a/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala +++ b/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala @@ -48,18 +48,18 @@ final case class PeriodicSamplesMapper(start: Long, //scalastyle:off method.length def apply(source: Observable[RangeVector], - queryConfig: QueryConfig, + querySession: QuerySession, limit: Int, sourceSchema: ResultSchema, paramResponse: Seq[Observable[ScalarRangeVector]]): Observable[RangeVector] = { // enforcement of minimum step is good since we have a high limit on number of samples - if (step < queryConfig.minStepMs) - throw new BadQueryException(s"step should be at least ${queryConfig.minStepMs/1000}s") + if (step < querySession.queryConfig.minStepMs) + throw new BadQueryException(s"step should be at least ${querySession.queryConfig.minStepMs/1000}s") val valColType = RangeVectorTransformer.valueColumnType(sourceSchema) // If a max column is present, the ExecPlan's job is to put it into column 2 val hasMaxCol = valColType == ColumnType.HistogramColumn && sourceSchema.colIDs.length > 2 && sourceSchema.columns(2).name == "max" - val rangeFuncGen = RangeFunction.generatorFor(sourceSchema, functionId, valColType, queryConfig, + val rangeFuncGen = RangeFunction.generatorFor(sourceSchema, functionId, valColType, querySession.queryConfig, funcParams, rawSource) // Generate one range function to check if it is chunked @@ -67,7 +67,7 @@ final case class PeriodicSamplesMapper(start: Long, // Really, use the stale lookback window size, not 0 which doesn't make sense // Default value for window should be queryConfig.staleSampleAfterMs + 1 for empty functionId, // so that it returns value present at time - staleSampleAfterMs - val windowLength = window.getOrElse(if (isLastFn) queryConfig.staleSampleAfterMs + 1 else 0L) + val windowLength = window.getOrElse(if (isLastFn) querySession.queryConfig.staleSampleAfterMs + 1 else 0L) val rvs = sampleRangeFunc match { case c: ChunkedRangeFunction[_] if valColType == ColumnType.HistogramColumn => @@ -75,28 +75,28 @@ final case class PeriodicSamplesMapper(start: Long, val histRow = if (hasMaxCol) new TransientHistMaxRow() else new TransientHistRow() IteratorBackedRangeVector(rv.key, new ChunkedWindowIteratorH(rv.asInstanceOf[RawDataRangeVector], startWithOffset, step, endWithOffset, - windowLength, rangeFuncGen().asChunkedH, queryConfig, queryContext, histRow)) + windowLength, rangeFuncGen().asChunkedH, querySession, histRow)) } case c: ChunkedRangeFunction[_] => source.map { rv => qLogger.trace(s"Creating ChunkedWindowIterator for rv=${rv.key}, step=$step windowLength=$windowLength") IteratorBackedRangeVector(rv.key, new ChunkedWindowIteratorD(rv.asInstanceOf[RawDataRangeVector], startWithOffset, step, endWithOffset, - windowLength, rangeFuncGen().asChunkedD, queryConfig, queryContext)) + windowLength, rangeFuncGen().asChunkedD, querySession)) } // Iterator-based: Wrap long columns to yield a double value case f: RangeFunction if valColType == ColumnType.LongColumn => source.map { rv => IteratorBackedRangeVector(rv.key, new SlidingWindowIterator(new LongToDoubleIterator(rv.rows), startWithOffset, step, endWithOffset, - window.getOrElse(0L), rangeFuncGen().asSliding, queryConfig)) + window.getOrElse(0L), rangeFuncGen().asSliding, querySession.queryConfig)) } // Otherwise just feed in the double column case f: RangeFunction => source.map { rv => IteratorBackedRangeVector(rv.key, new SlidingWindowIterator(rv.rows, startWithOffset, step, endWithOffset, window.getOrElse(0L), - rangeFuncGen().asSliding, queryConfig)) + rangeFuncGen().asSliding, querySession.queryConfig)) } } @@ -157,13 +157,12 @@ abstract class ChunkedWindowIterator[R <: MutableRowReader]( end: Long, window: Long, rangeFunction: ChunkedRangeFunction[R], - queryConfig: QueryConfig, - queryContext: QueryContext) + querySession: QuerySession) extends Iterator[R] with StrictLogging { // Lazily open the iterator and obtain the lock. This allows one thread to create the // iterator, but the lock is owned by the thread actually performing the iteration. private lazy val windowIt = { - val it = new WindowedChunkIterator(rv, start, step, end, window, queryContext) + val it = new WindowedChunkIterator(rv, start, step, end, window, querySession.qContext) // Need to hold the shared lock explicitly, because the window iterator needs to // pre-fetch chunks to determine the window. This pre-fetching can force the internal // iterator to close, which would release the lock too soon. @@ -187,7 +186,7 @@ extends Iterator[R] with StrictLogging { try { rangeFunction.addChunks(nextInfo.getTsVectorAccessor, nextInfo.getTsVectorAddr, nextInfo.getTsReader, nextInfo.getValueVectorAccessor, nextInfo.getValueVectorAddr, nextInfo.getValueReader, - wit.curWindowStart, wit.curWindowEnd, nextInfo, queryConfig) + wit.curWindowStart, wit.curWindowEnd, nextInfo, querySession.queryConfig) } catch { case e: Exception => val tsReader = LongBinaryVector(nextInfo.getTsVectorAccessor, nextInfo.getTsVectorAddr) @@ -216,20 +215,18 @@ extends Iterator[R] with StrictLogging { class ChunkedWindowIteratorD(rv: RawDataRangeVector, start: Long, step: Long, end: Long, window: Long, rangeFunction: ChunkedRangeFunction[TransientRow], - queryConfig: QueryConfig, - queryContext: QueryContext = QueryContext(), + querySession: QuerySession, // put emitter here in constructor for faster access var sampleToEmit: TransientRow = new TransientRow()) extends -ChunkedWindowIterator[TransientRow](rv, start, step, end, window, rangeFunction, queryConfig, queryContext) +ChunkedWindowIterator[TransientRow](rv, start, step, end, window, rangeFunction, querySession) class ChunkedWindowIteratorH(rv: RawDataRangeVector, start: Long, step: Long, end: Long, window: Long, rangeFunction: ChunkedRangeFunction[TransientHistRow], - queryConfig: QueryConfig, - queryContext: QueryContext = QueryContext(), + querySession: QuerySession, // put emitter here in constructor for faster access var sampleToEmit: TransientHistRow = new TransientHistRow()) extends -ChunkedWindowIterator[TransientHistRow](rv, start, step, end, window, rangeFunction, queryConfig, queryContext) +ChunkedWindowIterator[TransientHistRow](rv, start, step, end, window, rangeFunction, querySession) class QueueBasedWindow(q: IndexedArrayQueue[TransientRow]) extends Window { def size: Int = q.size diff --git a/query/src/main/scala/filodb/query/exec/PromQlExec.scala b/query/src/main/scala/filodb/query/exec/PromQlExec.scala index c42834abe0..af2a120ddd 100644 --- a/query/src/main/scala/filodb/query/exec/PromQlExec.scala +++ b/query/src/main/scala/filodb/query/exec/PromQlExec.scala @@ -37,11 +37,12 @@ case class PromQlExec(queryContext: QueryContext, * implementation of the operation represented by this exec plan * node */ - def doExecute(source: ChunkSource, queryConfig: QueryConfig) + def doExecute(source: ChunkSource, + querySession: QuerySession) (implicit sched: Scheduler): ExecResult = ??? override def execute(source: ChunkSource, - queryConfig: QueryConfig) + querySession: QuerySession) (implicit sched: Scheduler): Task[QueryResponse] = { val execPlan2Span = Kamon.spanBuilder(s"execute-${getClass.getSimpleName}") .asChildOf(Kamon.currentSpan()) diff --git a/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala b/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala index 1d1dfd41c4..e68f1ae758 100644 --- a/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala +++ b/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala @@ -11,7 +11,7 @@ import filodb.core.query.Filter.Equals import filodb.memory.format.{RowReader, ZeroCopyUTF8String} import filodb.memory.format.vectors.{HistogramBuckets, HistogramWithBuckets} import filodb.query._ -import filodb.query.{BinaryOperator, InstantFunctionId, MiscellaneousFunctionId, QueryConfig, SortFunctionId} +import filodb.query.{BinaryOperator, InstantFunctionId, MiscellaneousFunctionId, SortFunctionId} import filodb.query.InstantFunctionId.HistogramQuantile import filodb.query.MiscellaneousFunctionId.{LabelJoin, LabelReplace} import filodb.query.ScalarFunctionId.Scalar @@ -36,7 +36,7 @@ trait RangeVectorTransformer extends java.io.Serializable { def funcParams: Seq[FuncArgs] def apply(source: Observable[RangeVector], - queryConfig: QueryConfig, + querySession: QuerySession, limit: Int, sourceSchema: ResultSchema, paramsResponse: Seq[Observable[ScalarRangeVector]]): Observable[RangeVector] @@ -69,7 +69,7 @@ final case class InstantVectorFunctionMapper(function: InstantFunctionId, funcParams: Seq[FuncArgs] = Nil) extends RangeVectorTransformer { protected[exec] def args: String = s"function=$function" - def evaluate(source: Observable[RangeVector], scalarRangeVector: Seq[ScalarRangeVector], queryConfig: QueryConfig, + def evaluate(source: Observable[RangeVector], scalarRangeVector: Seq[ScalarRangeVector], querySession: QuerySession, limit: Int, sourceSchema: ResultSchema) : Observable[RangeVector] = { RangeVectorTransformer.valueColumnType(sourceSchema) match { case ColumnType.HistogramColumn => @@ -91,7 +91,7 @@ final case class InstantVectorFunctionMapper(function: InstantFunctionId, if (function == HistogramQuantile) { // Special mapper to pull all buckets together from different Prom-schema time series val mapper = HistogramQuantileMapper(funcParams) - mapper.apply(source, queryConfig, limit, sourceSchema, Nil) + mapper.apply(source, querySession, limit, sourceSchema, Nil) } else { val instantFunction = InstantFunction.double(function) source.map { rv => @@ -105,21 +105,21 @@ final case class InstantVectorFunctionMapper(function: InstantFunctionId, } def apply(source: Observable[RangeVector], - queryConfig: QueryConfig, + querySession: QuerySession, limit: Int, sourceSchema: ResultSchema, paramResponse: Seq[Observable[ScalarRangeVector]]): Observable[RangeVector] = { if (funcParams.isEmpty) { - evaluate(source, Nil, queryConfig, limit, sourceSchema) + evaluate(source, Nil, querySession, limit, sourceSchema) } else { // Multiple ExecPlanFunArgs not supported yet funcParams.head match { case s: StaticFuncArgs => evaluate(source, funcParams.map(x => x.asInstanceOf[StaticFuncArgs]). - map(x => ScalarFixedDouble(x.timeStepParams, x.scalar)), queryConfig, limit, + map(x => ScalarFixedDouble(x.timeStepParams, x.scalar)), querySession, limit, sourceSchema) case t: TimeFuncArgs => evaluate(source, funcParams.map(x => x.asInstanceOf[TimeFuncArgs]). - map(x => TimeScalar(x.timeStepParams)), queryConfig, limit, sourceSchema) + map(x => TimeScalar(x.timeStepParams)), querySession, limit, sourceSchema) case e: ExecPlanFuncArgs => paramResponse.head.map { param => - evaluate(source, Seq(param), queryConfig, limit, sourceSchema) + evaluate(source, Seq(param), querySession, limit, sourceSchema) }.flatten case _ => throw new IllegalArgumentException(s"Invalid function param") } @@ -198,7 +198,7 @@ final case class ScalarOperationMapper(operator: BinaryOperator, val operatorFunction = BinaryOperatorFunction.factoryMethod(operator) def apply(source: Observable[RangeVector], - queryConfig: QueryConfig, + querySession: QuerySession, limit: Int, sourceSchema: ResultSchema, paramResponse: Seq[Observable[ScalarRangeVector]] = Nil): Observable[RangeVector] = { @@ -250,7 +250,7 @@ final case class MiscellaneousFunctionMapper(function: MiscellaneousFunctionId, } def apply(source: Observable[RangeVector], - queryConfig: QueryConfig, + querySession: QuerySession, limit: Int, sourceSchema: ResultSchema, paramResponse: Seq[Observable[ScalarRangeVector]]): Observable[RangeVector] = { @@ -262,7 +262,7 @@ final case class SortFunctionMapper(function: SortFunctionId) extends RangeVecto protected[exec] def args: String = s"function=$function" def apply(source: Observable[RangeVector], - queryConfig: QueryConfig, + querySession: QuerySession, limit: Int, sourceSchema: ResultSchema, paramResponse: Seq[Observable[ScalarRangeVector]]): Observable[RangeVector] = { @@ -309,7 +309,7 @@ final case class ScalarFunctionMapper(function: ScalarFunctionId, Observable.fromTask(resultRv).flatten } def apply(source: Observable[RangeVector], - queryConfig: QueryConfig, + querySession: QuerySession, limit: Int, sourceSchema: ResultSchema, paramResponse: Seq[Observable[ScalarRangeVector]]): Observable[RangeVector] = { @@ -327,7 +327,7 @@ final case class VectorFunctionMapper() extends RangeVectorTransformer { protected[exec] def args: String = s"funcParams=$funcParams" def apply(source: Observable[RangeVector], - queryConfig: QueryConfig, + querySession: QuerySession, limit: Int, sourceSchema: ResultSchema, paramResponse: Seq[Observable[ScalarRangeVector]]): Observable[RangeVector] = { @@ -356,7 +356,7 @@ final case class AbsentFunctionMapper(columnFilter: Seq[ColumnFilter], rangePara } def apply(source: Observable[RangeVector], - queryConfig: QueryConfig, + querySession: QuerySession, limit: Int, sourceSchema: ResultSchema, paramResponse: Seq[Observable[ScalarRangeVector]]): Observable[RangeVector] = { @@ -414,7 +414,7 @@ final case class HistToPromSeriesMapper(sch: PartitionSchema) extends RangeVecto // NOTE: apply() is only called for explicit instantiation of conversion function. So this will error out if // the data source is not histogram. def apply(source: Observable[RangeVector], - queryConfig: QueryConfig, + querySession: QuerySession, limit: Int, sourceSchema: ResultSchema, paramResponse: Seq[Observable[ScalarRangeVector]]): Observable[RangeVector] = { diff --git a/query/src/main/scala/filodb/query/exec/ScalarBinaryOperationExec.scala b/query/src/main/scala/filodb/query/exec/ScalarBinaryOperationExec.scala index 8a0e590297..012f2c92d1 100644 --- a/query/src/main/scala/filodb/query/exec/ScalarBinaryOperationExec.scala +++ b/query/src/main/scala/filodb/query/exec/ScalarBinaryOperationExec.scala @@ -8,7 +8,7 @@ import filodb.core.DatasetRef import filodb.core.metadata.Column.ColumnType import filodb.core.query._ import filodb.core.store.ChunkSource -import filodb.query.{BinaryOperator, QueryConfig, QueryResponse, QueryResult} +import filodb.query.{BinaryOperator, QueryResponse, QueryResult} import filodb.query.exec.binaryOp.BinaryOperatorFunction /** @@ -47,14 +47,15 @@ case class ScalarBinaryOperationExec(queryContext: QueryContext, * implementation of the operation represented by this exec plan * node */ - override def doExecute(source: ChunkSource, queryConfig: QueryConfig) + override def doExecute(source: ChunkSource, + querySession: QuerySession) (implicit sched: Scheduler): ExecResult = { throw new IllegalStateException("doExecute should not be called for ScalarBinaryOperationExec since it represents" + "a static value") } override def execute(source: ChunkSource, - queryConfig: QueryConfig) + querySession: QuerySession) (implicit sched: Scheduler): Task[QueryResponse] = { val execPlan2Span = Kamon.spanBuilder(s"execute-${getClass.getSimpleName}") .asChildOf(Kamon.currentSpan()) diff --git a/query/src/main/scala/filodb/query/exec/ScalarFixedDoubleExec.scala b/query/src/main/scala/filodb/query/exec/ScalarFixedDoubleExec.scala index 4c18481d30..a6dd0fa3c9 100644 --- a/query/src/main/scala/filodb/query/exec/ScalarFixedDoubleExec.scala +++ b/query/src/main/scala/filodb/query/exec/ScalarFixedDoubleExec.scala @@ -9,7 +9,7 @@ import filodb.core.DatasetRef import filodb.core.metadata.Column.ColumnType import filodb.core.query._ import filodb.core.store.ChunkSource -import filodb.query.{QueryConfig, QueryResponse, QueryResult} +import filodb.query.{QueryResponse, QueryResult} /** @@ -29,7 +29,8 @@ case class ScalarFixedDoubleExec(queryContext: QueryContext, * implementation of the operation represented by this exec plan * node */ - override def doExecute(source: ChunkSource, queryConfig: QueryConfig) + override def doExecute(source: ChunkSource, + querySession: QuerySession) (implicit sched: Scheduler): ExecResult = { throw new IllegalStateException("doExecute should not be called for ScalarFixedDoubleExec since it represents a " + "readily available static value") @@ -43,7 +44,7 @@ case class ScalarFixedDoubleExec(queryContext: QueryContext, override def execute(source: ChunkSource, - queryConfig: QueryConfig) + querySession: QuerySession) (implicit sched: Scheduler): Task[QueryResponse] = { val execPlan2Span = Kamon.spanBuilder(s"execute-${getClass.getSimpleName}") .asChildOf(Kamon.currentSpan()) @@ -62,7 +63,8 @@ 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, queryConfig, queryContext.sampleLimit, acc._2, paramRangeVector), transf.schema(acc._2)) + (transf.apply(acc._1, querySession, queryContext.sampleLimit, acc._2, + paramRangeVector), transf.schema(acc._2)) }._1.toListL.map({ span.finish() QueryResult(queryContext.queryId, resultSchema, _) diff --git a/query/src/main/scala/filodb/query/exec/SelectChunkInfosExec.scala b/query/src/main/scala/filodb/query/exec/SelectChunkInfosExec.scala index 1b067954e9..b449ffbefb 100644 --- a/query/src/main/scala/filodb/query/exec/SelectChunkInfosExec.scala +++ b/query/src/main/scala/filodb/query/exec/SelectChunkInfosExec.scala @@ -9,7 +9,6 @@ import filodb.core.memstore.TimeSeriesShard import filodb.core.metadata.Column import filodb.core.query._ import filodb.core.store._ -import filodb.query.QueryConfig object SelectChunkInfosExec { import Column.ColumnType._ @@ -42,10 +41,10 @@ final case class SelectChunkInfosExec(queryContext: QueryContext, import SelectChunkInfosExec._ def doExecute(source: ChunkSource, - queryConfig: QueryConfig) + querySession: QuerySession) (implicit sched: Scheduler): ExecResult = { val partMethod = FilteredPartitionScan(ShardSplit(shard), filters) - val lookupRes = source.lookupPartitions(dataset, partMethod, chunkMethod) + val lookupRes = source.lookupPartitions(dataset, partMethod, chunkMethod, querySession) val schemas = source.schemas(dataset).get val dataSchema = schema.map { s => schemas.schemas(s) } @@ -55,7 +54,7 @@ final case class SelectChunkInfosExec(queryContext: QueryContext, val partCols = dataSchema.partitionInfos val numGroups = source.groupsInDataset(dataset) Kamon.currentSpan().mark("creating-scanpartitions") - val rvs = source.scanPartitions(dataset, lookupRes) + val rvs = source.scanPartitions(dataset, lookupRes, querySession) .filter(_.hasChunks(chunkMethod)) .map { partition => source.stats.incrReadPartitions(1) diff --git a/query/src/main/scala/filodb/query/exec/SelectRawPartitionsExec.scala b/query/src/main/scala/filodb/query/exec/SelectRawPartitionsExec.scala index 6afbc59c05..a18a668baa 100644 --- a/query/src/main/scala/filodb/query/exec/SelectRawPartitionsExec.scala +++ b/query/src/main/scala/filodb/query/exec/SelectRawPartitionsExec.scala @@ -8,9 +8,9 @@ import monix.reactive.Observable import filodb.core.{DatasetRef, Types} import filodb.core.memstore.PartLookupResult import filodb.core.metadata.{Column, Schema, Schemas} -import filodb.core.query.{QueryContext, ResultSchema} +import filodb.core.query.{QueryContext, QuerySession, ResultSchema} import filodb.core.store._ -import filodb.query.{Query, QueryConfig} +import filodb.query.Query import filodb.query.Query.qLogger import filodb.query.exec.rangefn.RangeFunction @@ -122,7 +122,7 @@ final case class SelectRawPartitionsExec(queryContext: QueryContext, } def doExecute(source: ChunkSource, - queryConfig: QueryConfig) + querySession: QuerySession) (implicit sched: Scheduler): ExecResult = { val span = Kamon.spanBuilder(s"execute-${getClass.getSimpleName}") .asChildOf(Kamon.currentSpan()) @@ -131,7 +131,7 @@ final case class SelectRawPartitionsExec(queryContext: QueryContext, 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) + 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/SetOperatorExec.scala b/query/src/main/scala/filodb/query/exec/SetOperatorExec.scala index 3f72cf7067..9735d60cc6 100644 --- a/query/src/main/scala/filodb/query/exec/SetOperatorExec.scala +++ b/query/src/main/scala/filodb/query/exec/SetOperatorExec.scala @@ -51,7 +51,7 @@ final case class SetOperatorExec(queryContext: QueryContext, protected[exec] def compose(childResponses: Observable[(QueryResponse, Int)], firstSchema: Task[ResultSchema], - queryConfig: QueryConfig): Observable[RangeVector] = { + querySession: QuerySession): Observable[RangeVector] = { val taskOfResults = childResponses.map { case (QueryResult(_, _, result), i) => (result, i) case (QueryError(_, ex), _) => throw ex diff --git a/query/src/main/scala/filodb/query/exec/StitchRvsExec.scala b/query/src/main/scala/filodb/query/exec/StitchRvsExec.scala index 7cad5555b5..f68912e911 100644 --- a/query/src/main/scala/filodb/query/exec/StitchRvsExec.scala +++ b/query/src/main/scala/filodb/query/exec/StitchRvsExec.scala @@ -66,7 +66,7 @@ final case class StitchRvsExec(queryContext: QueryContext, protected def compose(childResponses: Observable[(QueryResponse, Int)], firstSchema: Task[ResultSchema], - queryConfig: QueryConfig): Observable[RangeVector] = { + querySession: QuerySession): Observable[RangeVector] = { qLogger.debug(s"StitchRvsExec: Stitching results:") val stitched = childResponses.map { case (QueryResult(_, _, result), _) => result @@ -103,7 +103,7 @@ final case class StitchRvsExec(queryContext: QueryContext, final case class StitchRvsMapper() extends RangeVectorTransformer { def apply(source: Observable[RangeVector], - queryConfig: QueryConfig, + querySession: QuerySession, limit: Int, sourceSchema: ResultSchema, paramResponse: Seq[Observable[ScalarRangeVector]]): Observable[RangeVector] = { qLogger.debug(s"StitchRvsMapper: Stitching results:") diff --git a/query/src/main/scala/filodb/query/exec/TimeScalarGeneratorExec.scala b/query/src/main/scala/filodb/query/exec/TimeScalarGeneratorExec.scala index f9c344de55..49d9212385 100644 --- a/query/src/main/scala/filodb/query/exec/TimeScalarGeneratorExec.scala +++ b/query/src/main/scala/filodb/query/exec/TimeScalarGeneratorExec.scala @@ -9,7 +9,7 @@ import filodb.core.DatasetRef import filodb.core.metadata.Column.ColumnType import filodb.core.query._ import filodb.core.store.ChunkSource -import filodb.query.{BadQueryException, QueryConfig, QueryResponse, QueryResult, ScalarFunctionId} +import filodb.query.{BadQueryException, QueryResponse, QueryResult, ScalarFunctionId} import filodb.query.ScalarFunctionId.{DayOfMonth, DayOfWeek, DaysInMonth, Hour, Minute, Month, Time, Year} /** @@ -28,7 +28,8 @@ case class TimeScalarGeneratorExec(queryContext: QueryContext, * implementation of the operation represented by this exec plan * node */ - override def doExecute(source: ChunkSource, queryConfig: QueryConfig) + override def doExecute(source: ChunkSource, + querySession: QuerySession) (implicit sched: Scheduler): ExecResult = { throw new IllegalStateException("doExecute should not be called for TimeScalarGeneratorExec since it represents" + "a readily available static value") @@ -41,7 +42,7 @@ case class TimeScalarGeneratorExec(queryContext: QueryContext, override protected def args: String = s"params = $params, function = $function" override def execute(source: ChunkSource, - queryConfig: QueryConfig) + querySession: QuerySession) (implicit sched: Scheduler): Task[QueryResponse] = { val execPlan2Span = Kamon.spanBuilder(s"execute-${getClass.getSimpleName}") .asChildOf(Kamon.currentSpan()) @@ -71,7 +72,8 @@ 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, queryConfig, queryContext.sampleLimit, acc._2, paramRangeVector), transf.schema(acc._2)) + (transf.apply(acc._1, querySession, queryContext.sampleLimit, acc._2, + paramRangeVector), transf.schema(acc._2)) }._1.toListL.map({ span.finish() QueryResult(queryContext.queryId, resultSchema, _) diff --git a/query/src/main/scala/filodb/query/exec/rangefn/AggrOverTimeFunctions.scala b/query/src/main/scala/filodb/query/exec/rangefn/AggrOverTimeFunctions.scala index 994d3fbcc8..434d05efad 100644 --- a/query/src/main/scala/filodb/query/exec/rangefn/AggrOverTimeFunctions.scala +++ b/query/src/main/scala/filodb/query/exec/rangefn/AggrOverTimeFunctions.scala @@ -5,12 +5,11 @@ import java.lang.{Double => JLDouble} import debox.Buffer import java.util -import filodb.core.query.{TransientHistMaxRow, TransientHistRow, TransientRow} +import filodb.core.query.{QueryConfig, TransientHistMaxRow, TransientHistRow, TransientRow} import filodb.core.store.ChunkSetInfoReader import filodb.memory.format.{BinaryVector, MemoryReader, VectorDataReader} import filodb.memory.format.{vectors => bv} import filodb.memory.format.vectors.DoubleIterator -import filodb.query.QueryConfig import filodb.query.exec.{FuncArgs, StaticFuncArgs} class MinMaxOverTimeFunction(ord: Ordering[Double]) extends RangeFunction { 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 cf4b59cf47..356c03195c 100644 --- a/query/src/main/scala/filodb/query/exec/rangefn/RangeFunction.scala +++ b/query/src/main/scala/filodb/query/exec/rangefn/RangeFunction.scala @@ -2,11 +2,10 @@ package filodb.query.exec.rangefn import filodb.core.metadata.Column.ColumnType import filodb.core.metadata.Schema -import filodb.core.query.{MutableRowReader, ResultSchema, TransientHistMaxRow, TransientHistRow, TransientRow} +import filodb.core.query._ import filodb.core.store.ChunkSetInfoReader import filodb.memory.format.{vectors => bv, _} import filodb.memory.format.BinaryVector.BinaryVectorPtr -import filodb.query.QueryConfig import filodb.query.exec._ /** diff --git a/query/src/main/scala/filodb/query/exec/rangefn/RangeInstantFunctions.scala b/query/src/main/scala/filodb/query/exec/rangefn/RangeInstantFunctions.scala index e4453904df..3675029130 100644 --- a/query/src/main/scala/filodb/query/exec/rangefn/RangeInstantFunctions.scala +++ b/query/src/main/scala/filodb/query/exec/rangefn/RangeInstantFunctions.scala @@ -1,7 +1,6 @@ package filodb.query.exec.rangefn -import filodb.core.query.TransientRow -import filodb.query.QueryConfig +import filodb.core.query.{QueryConfig, TransientRow} object RangeInstantFunctions { def derivFunction(window: Window): Double = { diff --git a/query/src/main/scala/filodb/query/exec/rangefn/RateFunctions.scala b/query/src/main/scala/filodb/query/exec/rangefn/RateFunctions.scala index fd60a19361..417f653158 100644 --- a/query/src/main/scala/filodb/query/exec/rangefn/RateFunctions.scala +++ b/query/src/main/scala/filodb/query/exec/rangefn/RateFunctions.scala @@ -2,11 +2,10 @@ package filodb.query.exec.rangefn import scalaxy.loops._ -import filodb.core.query.{TransientHistRow, TransientRow} +import filodb.core.query.{QueryConfig, TransientHistRow, TransientRow} import filodb.memory.format.{CounterVectorReader, MemoryReader} import filodb.memory.format.{vectors => bv} import filodb.memory.format.BinaryVector.BinaryVectorPtr -import filodb.query.QueryConfig object RateFunctions { diff --git a/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala b/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala index 9a3525eb71..54be905e64 100644 --- a/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala +++ b/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala @@ -308,8 +308,8 @@ class AggrOverRangeVectorsSpec extends RawDataWindowingSpec with ScalaFutures { val agg = RowAggregator(AggregationOperator.Avg, Nil, tvSchema) val aggMR = AggregateMapReduce(AggregationOperator.Avg, Nil, Nil, Nil) - val mapped1 = aggMR(Observable.fromIterable(Seq(toRv(s1))), queryConfig, 1000, tvSchema) - val mapped2 = aggMR(Observable.fromIterable(Seq(toRv(s2))), queryConfig, 1000, tvSchema) + val mapped1 = aggMR(Observable.fromIterable(Seq(toRv(s1))), querySession, 1000, tvSchema) + val mapped2 = aggMR(Observable.fromIterable(Seq(toRv(s2))), querySession, 1000, tvSchema) val resultObs4 = RangeVectorAggregator.mapReduce(agg, true, mapped1 ++ mapped2, rv=>rv.key) val result4 = resultObs4.toListL.runAsync.futureValue diff --git a/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala b/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala index aa4a067b78..6c6dec9928 100644 --- a/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala @@ -21,6 +21,7 @@ class BinaryJoinExecSpec extends FunSpec with Matchers with ScalaFutures { val config = ConfigFactory.load("application_test.conf").getConfig("filodb") val queryConfig = new QueryConfig(config.getConfig("query")) + val querySession = QuerySession(QueryContext(), queryConfig) val rand = new Random() val error = 0.00000001d @@ -94,7 +95,7 @@ class BinaryJoinExecSpec extends FunSpec with Matchers with ScalaFutures { val rhs = QueryResult("someId", null, samplesRhs2.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on // note below that order of lhs and rhs is reversed, but index is right. Join should take that into account - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), tvSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), tvSchemaTask, querySession) .toListL.runAsync.futureValue result.foreach { rv => @@ -123,7 +124,7 @@ class BinaryJoinExecSpec extends FunSpec with Matchers with ScalaFutures { val lhs = QueryResult("someId", null, samplesLhs.map(rv => SerializedRangeVector(rv, schema))) val rhs = QueryResult("someId", null, samplesRhs2.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val result = execPlan.compose(Observable.fromIterable(Seq((lhs, 0), (rhs, 1))), tvSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((lhs, 0), (rhs, 1))), tvSchemaTask, querySession) .toListL.runAsync.futureValue result.foreach { rv => @@ -160,7 +161,7 @@ class BinaryJoinExecSpec extends FunSpec with Matchers with ScalaFutures { val rhs = QueryResult("someId", null, samplesRhs2.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val fut = execPlan.compose(Observable.fromIterable(Seq((lhs, 0), (rhs, 1))), tvSchemaTask, queryConfig) + val fut = execPlan.compose(Observable.fromIterable(Seq((lhs, 0), (rhs, 1))), tvSchemaTask, querySession) .toListL.runAsync ScalaFutures.whenReady(fut.failed) { e => e shouldBe a[BadQueryException] @@ -191,7 +192,7 @@ class BinaryJoinExecSpec extends FunSpec with Matchers with ScalaFutures { val rhs = QueryResult("someId", null, samplesRhs.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val fut = execPlan.compose(Observable.fromIterable(Seq((lhs, 0), (rhs, 1))), tvSchemaTask, queryConfig) + val fut = execPlan.compose(Observable.fromIterable(Seq((lhs, 0), (rhs, 1))), tvSchemaTask, querySession) .toListL.runAsync ScalaFutures.whenReady(fut.failed) { e => e.printStackTrace() @@ -213,7 +214,7 @@ class BinaryJoinExecSpec extends FunSpec with Matchers with ScalaFutures { val rhs = QueryResult("someId", null, samplesRhsGrouping.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on // note below that order of lhs and rhs is reversed, but index is right. Join should take that into account - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), tvSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), tvSchemaTask, querySession) .toListL.runAsync.futureValue result.foreach { rv => @@ -241,7 +242,7 @@ class BinaryJoinExecSpec extends FunSpec with Matchers with ScalaFutures { val rhs = QueryResult("someId", null, samplesRhsGrouping.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on // note below that order of lhs and rhs is reversed, but index is right. Join should take that into account - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), tvSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), tvSchemaTask, querySession) .toListL.runAsync.futureValue result.foreach { rv => @@ -289,7 +290,7 @@ class BinaryJoinExecSpec extends FunSpec with Matchers with ScalaFutures { val rhs = QueryResult("someId", null, samplesRhs2.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on // note below that order of lhs and rhs is reversed, but index is right. Join should take that into account - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), tvSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), tvSchemaTask, querySession) .toListL.runAsync.futureValue result.foreach { rv => @@ -337,7 +338,7 @@ class BinaryJoinExecSpec extends FunSpec with Matchers with ScalaFutures { val rhs = QueryResult("someId", null, samplesRhs.map(rv => SerializedRangeVector (rv, schema))) // scalastyle:on // note below that order of lhs and rhs is reversed, but index is right. Join should take that into account - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), tvSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), tvSchemaTask, querySession) .toListL.runAsync.futureValue result.foreach { rv => diff --git a/query/src/test/scala/filodb/query/exec/BinaryJoinGroupingSpec.scala b/query/src/test/scala/filodb/query/exec/BinaryJoinGroupingSpec.scala index d88f1a1109..e64dc08436 100644 --- a/query/src/test/scala/filodb/query/exec/BinaryJoinGroupingSpec.scala +++ b/query/src/test/scala/filodb/query/exec/BinaryJoinGroupingSpec.scala @@ -1,6 +1,7 @@ package filodb.query.exec import scala.util.Random + import com.typesafe.config.ConfigFactory import monix.eval.Task import monix.execution.Scheduler @@ -22,6 +23,8 @@ class BinaryJoinGroupingSpec extends FunSpec with Matchers with ScalaFutures { val config = ConfigFactory.load("application_test.conf").getConfig("filodb") val queryConfig = new QueryConfig(config.getConfig("query")) + val querySession = QuerySession(QueryContext(), queryConfig) + val tvSchema = ResultSchema(Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), ColumnInfo("value", ColumnType.DoubleColumn)), 1) val schema = Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), @@ -123,7 +126,7 @@ class BinaryJoinGroupingSpec extends FunSpec with Matchers with ScalaFutures { val lhs = QueryResult("someId", null, sampleNodeCpu.map(rv => SerializedRangeVector(rv, schema))) val rhs = QueryResult("someId", null, samplesRhs2.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), tvSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), tvSchemaTask, querySession) .toListL.runAsync.futureValue val expectedLabels = List(Map(ZeroCopyUTF8String("instance") -> ZeroCopyUTF8String("abc"), @@ -152,13 +155,13 @@ class BinaryJoinGroupingSpec extends FunSpec with Matchers with ScalaFutures { new Array[ExecPlan](1), BinaryOperator.MUL, Cardinality.ManyToOne, - Nil, Seq("role", "mode"), Seq("role"), "__name__") + Nil, Seq("role", "mode"), Seq("role"), "__name__") // scalastyle:off val lhs = QueryResult("someId", null, sampleNodeCpu.map(rv => SerializedRangeVector(rv, schema))) val rhs = QueryResult("someId", null, samplesRhs2.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), tvSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), tvSchemaTask, querySession) .toListL.runAsync.futureValue val expectedLabels = List(Map(ZeroCopyUTF8String("instance") -> ZeroCopyUTF8String("abc"), @@ -182,7 +185,7 @@ class BinaryJoinGroupingSpec extends FunSpec with Matchers with ScalaFutures { val agg = RowAggregator(AggregationOperator.Sum, Nil, tvSchema) val aggMR = AggregateMapReduce(AggregationOperator.Sum, Nil, Nil, Seq("instance", "job")) - val mapped = aggMR(Observable.fromIterable(sampleNodeCpu), queryConfig, 1000, tvSchema) + val mapped = aggMR(Observable.fromIterable(sampleNodeCpu), querySession, 1000, tvSchema) val resultObs4 = RangeVectorAggregator.mapReduce(agg, true, mapped, rv=>rv.key) val samplesRhs = resultObs4.toListL.runAsync.futureValue @@ -198,7 +201,7 @@ class BinaryJoinGroupingSpec extends FunSpec with Matchers with ScalaFutures { val lhs = QueryResult("someId", null, sampleNodeCpu.map(rv => SerializedRangeVector(rv, schema))) val rhs = QueryResult("someId", null, samplesRhs.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), tvSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), tvSchemaTask, querySession) .toListL.runAsync.futureValue val expectedLabels = List(Map(ZeroCopyUTF8String("instance") -> ZeroCopyUTF8String("abc"), @@ -243,7 +246,7 @@ class BinaryJoinGroupingSpec extends FunSpec with Matchers with ScalaFutures { val lhs = QueryResult("someId", null, sampleNodeRole.map(rv => SerializedRangeVector(rv, schema))) val rhs = QueryResult("someId", null, samplesRhs2.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), tvSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), tvSchemaTask, querySession) .toListL.runAsync.futureValue val expectedLabels = List(Map(ZeroCopyUTF8String("instance") -> ZeroCopyUTF8String("abc"), @@ -261,7 +264,7 @@ class BinaryJoinGroupingSpec extends FunSpec with Matchers with ScalaFutures { val agg = RowAggregator(AggregationOperator.Sum, Nil, tvSchema) val aggMR = AggregateMapReduce(AggregationOperator.Sum, Nil, Nil, Seq("instance", "job")) - val mapped = aggMR(Observable.fromIterable(sampleNodeCpu), queryConfig, 1000, tvSchema) + val mapped = aggMR(Observable.fromIterable(sampleNodeCpu), querySession, 1000, tvSchema) val resultObs4 = RangeVectorAggregator.mapReduce(agg, true, mapped, rv=>rv.key) val samplesRhs = resultObs4.toListL.runAsync.futureValue @@ -277,7 +280,7 @@ class BinaryJoinGroupingSpec extends FunSpec with Matchers with ScalaFutures { val lhs = QueryResult("someId", null, sampleNodeCpu.map(rv => SerializedRangeVector(rv, schema))) val rhs = QueryResult("someId", null, samplesRhs.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), tvSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), tvSchemaTask, querySession) .toListL.runAsync.futureValue val expectedLabels = List(Map(ZeroCopyUTF8String("instance") -> ZeroCopyUTF8String("abc"), @@ -357,7 +360,7 @@ class BinaryJoinGroupingSpec extends FunSpec with Matchers with ScalaFutures { val lhs = QueryResult("someId", null, sampleLhs.map(rv => SerializedRangeVector(rv, schema))) val rhs = QueryResult("someId", null, sampleRhs.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), tvSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), tvSchemaTask, querySession) .toListL.runAsync.futureValue val expectedLabels = List(Map(ZeroCopyUTF8String("instance") -> ZeroCopyUTF8String("abc"), diff --git a/query/src/test/scala/filodb/query/exec/BinaryJoinSetOperatorSpec.scala b/query/src/test/scala/filodb/query/exec/BinaryJoinSetOperatorSpec.scala index 07d47254a5..83019579a0 100644 --- a/query/src/test/scala/filodb/query/exec/BinaryJoinSetOperatorSpec.scala +++ b/query/src/test/scala/filodb/query/exec/BinaryJoinSetOperatorSpec.scala @@ -24,6 +24,8 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures val config = ConfigFactory.load("application_test.conf").getConfig("filodb") val queryConfig = new QueryConfig(config.getConfig("query")) + val querySession = QuerySession(QueryContext(), queryConfig) + val tvSchema = ResultSchema(Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), ColumnInfo("value", ColumnType.DoubleColumn)), 1) val schema = Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), @@ -186,7 +188,7 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures val lhs = QueryResult("someId", null, sampleCanary.map(rv => SerializedRangeVector(rv, schema))) val rhs = QueryResult("someId", null, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue @@ -217,13 +219,13 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures BinaryOperator.LAND, Nil, Nil, "__name__") - val canaryPlusOne = scalarOpMapper(Observable.fromIterable(sampleCanary), queryConfig, 1000, resultSchema). + val canaryPlusOne = scalarOpMapper(Observable.fromIterable(sampleCanary), querySession, 1000, resultSchema). toListL.runAsync.futureValue // scalastyle:off val lhs = QueryResult("someId", null, canaryPlusOne.map(rv => SerializedRangeVector(rv, schema))) val rhs = QueryResult("someId", null, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue val expectedLabels = List(Map(ZeroCopyUTF8String("__name__") -> ZeroCopyUTF8String("http_requests"), @@ -254,13 +256,13 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures BinaryOperator.LAND, Seq("instance", "job"), Nil, "__name__") - val canaryPlusOne = scalarOpMapper(Observable.fromIterable(sampleCanary), queryConfig, 1000, resultSchema). + val canaryPlusOne = scalarOpMapper(Observable.fromIterable(sampleCanary), querySession, 1000, resultSchema). toListL.runAsync.futureValue // scalastyle:off val lhs = QueryResult("someId", null, canaryPlusOne.map(rv => SerializedRangeVector(rv, schema))) val rhs = QueryResult("someId", null, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue val expectedLabels = List(Map(ZeroCopyUTF8String("__name__") -> ZeroCopyUTF8String("http_requests"), @@ -291,13 +293,13 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures BinaryOperator.LAND, Seq("instance"), Nil, "__name__") - val canaryPlusOne = scalarOpMapper(Observable.fromIterable(sampleCanary), queryConfig, 1000, resultSchema). + val canaryPlusOne = scalarOpMapper(Observable.fromIterable(sampleCanary), querySession, 1000, resultSchema). toListL.runAsync.futureValue // scalastyle:off val lhs = QueryResult("someId", null, canaryPlusOne.map(rv => SerializedRangeVector(rv, schema))) val rhs = QueryResult("someId", null, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue val expectedLabels = List(Map(ZeroCopyUTF8String("__name__") -> ZeroCopyUTF8String("http_requests"), @@ -327,13 +329,13 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures BinaryOperator.LAND, Nil, Seq("group"), "__name__") - val canaryPlusOne = scalarOpMapper(Observable.fromIterable(sampleCanary), queryConfig, 1000, resultSchema). + val canaryPlusOne = scalarOpMapper(Observable.fromIterable(sampleCanary), querySession, 1000, resultSchema). toListL.runAsync.futureValue // scalastyle:off val lhs = QueryResult("someId", null, canaryPlusOne.map(rv => SerializedRangeVector(rv, schema))) val rhs = QueryResult("someId", null, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue val expectedLabels = List(Map(ZeroCopyUTF8String("__name__") -> ZeroCopyUTF8String("http_requests"), @@ -362,13 +364,13 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures BinaryOperator.LAND, Nil, Seq("group", "job"), "__name__") - val canaryPlusOne = scalarOpMapper(Observable.fromIterable(sampleCanary), queryConfig, 1000, resultSchema). + val canaryPlusOne = scalarOpMapper(Observable.fromIterable(sampleCanary), querySession, 1000, resultSchema). toListL.runAsync.futureValue // scalastyle:off val lhs = QueryResult("someId", null, canaryPlusOne.map(rv => SerializedRangeVector(rv, schema))) val rhs = QueryResult("someId", null, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue val expectedLabels = List(Map(ZeroCopyUTF8String("__name__") -> ZeroCopyUTF8String("http_requests"), @@ -401,7 +403,7 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures val lhs = QueryResult("someId", null, sampleHttpRequests.map(rv => SerializedRangeVector(rv, schema))) val rhs = QueryResult("someId", null, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue result.size shouldEqual 8 @@ -423,7 +425,7 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures val lhs = QueryResult("someId", null, sampleHttpRequests.map(rv => SerializedRangeVector(rv, schema))) val rhs = QueryResult("someId", null, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue result.size shouldEqual 8 @@ -445,7 +447,7 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures val lhs = QueryResult("someId", null, sampleCanary.map(rv => SerializedRangeVector(rv, schema))) val rhs = QueryResult("someId", null, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue result.size shouldEqual 8 @@ -469,13 +471,13 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures BinaryOperator.LOR, Nil, Nil, "__name__") - val canaryPlusOne = scalarOpMapper(Observable.fromIterable(sampleCanary), queryConfig, 1000, resultSchema). + val canaryPlusOne = scalarOpMapper(Observable.fromIterable(sampleCanary), querySession, 1000, resultSchema). toListL.runAsync.futureValue // scalastyle:off val lhs = QueryResult("someId", null, canaryPlusOne.map(rv => SerializedRangeVector(rv, schema))) val rhs = QueryResult("someId", null, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue val expectedResult = (canaryPlusOne.toArray ++ sampleInstance1).distinct @@ -529,13 +531,13 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures BinaryOperator.LOR, Nil, Nil, "__name__") - val canaryPlusOne = scalarOpMapper(Observable.fromIterable(sampleCanary), queryConfig, 1000, resultSchema). + val canaryPlusOne = scalarOpMapper(Observable.fromIterable(sampleCanary), querySession, 1000, resultSchema). toListL.runAsync.futureValue // scalastyle:off val lhs1 = QueryResult("someId", null, sampleHttpRequests.map(rv => SerializedRangeVector(rv, schema))) val rhs1 = QueryResult("someId", null, sampleVectorMatching.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val result1 = execPlan1.compose(Observable.fromIterable(Seq((rhs1, 1), (lhs1, 0))), resSchemaTask, queryConfig) + val result1 = execPlan1.compose(Observable.fromIterable(Seq((rhs1, 1), (lhs1, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue val execPlan2 = SetOperatorExec(QueryContext(), dummyDispatcher, @@ -548,7 +550,7 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures val lhs2 = QueryResult("someId", null, canaryPlusOne.map(rv => SerializedRangeVector(rv, schema))) val rhs2 = QueryResult("someId", null, result1.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val result2 = execPlan2.compose(Observable.fromIterable(Seq((rhs2, 1), (lhs2, 0))), resSchemaTask, queryConfig) + val result2 = execPlan2.compose(Observable.fromIterable(Seq((rhs2, 1), (lhs2, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue val expectedLabels = List(Map(ZeroCopyUTF8String("__name__") -> ZeroCopyUTF8String("http_requests"), @@ -600,13 +602,13 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures BinaryOperator.LOR, Nil, Nil, "__name__") - val canaryPlusOne = scalarOpMapper(Observable.fromIterable(sampleCanary), queryConfig, 1000, resultSchema). + val canaryPlusOne = scalarOpMapper(Observable.fromIterable(sampleCanary), querySession, 1000, resultSchema). toListL.runAsync.futureValue // scalastyle:off val lhs1 = QueryResult("someId", null, sampleHttpRequests.map(rv => SerializedRangeVector(rv, schema))) val rhs1 = QueryResult("someId", null, sampleVectorMatching.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val result1 = execPlan1.compose(Observable.fromIterable(Seq((rhs1, 1), (lhs1, 0))), resSchemaTask, queryConfig) + val result1 = execPlan1.compose(Observable.fromIterable(Seq((rhs1, 1), (lhs1, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue val execPlan2 = SetOperatorExec(QueryContext(), dummyDispatcher, @@ -619,7 +621,7 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures val lhs2 = QueryResult("someId", null, canaryPlusOne.map(rv => SerializedRangeVector(rv, schema))) val rhs2 = QueryResult("someId", null, result1.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val result2 = execPlan2.compose(Observable.fromIterable(Seq((rhs2, 1), (lhs2, 0))), resSchemaTask, queryConfig) + val result2 = execPlan2.compose(Observable.fromIterable(Seq((rhs2, 1), (lhs2, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue val expectedLabels = List(Map(ZeroCopyUTF8String("__name__") -> ZeroCopyUTF8String("http_requests"), @@ -674,7 +676,7 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures val lhs = QueryResult("someId", null, sampleCanary.map(rv => SerializedRangeVector(rv, schema))) val rhs = QueryResult("someId", null, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue val expectedLabels = List(Map(ZeroCopyUTF8String("__name__") -> ZeroCopyUTF8String("http_requests"), @@ -708,7 +710,7 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures val lhs = QueryResult("someId", null, sampleCanary.map(rv => SerializedRangeVector(rv, schema))) val rhs = QueryResult("someId", null, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue val expectedLabels = List(Map(ZeroCopyUTF8String("__name__") -> ZeroCopyUTF8String("http_requests"), @@ -739,7 +741,7 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures val lhs = QueryResult("someId", null, sampleCanary.map(rv => SerializedRangeVector(rv, schema))) val rhs = QueryResult("someId", null, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue val expectedLabels = List(Map(ZeroCopyUTF8String("__name__") -> ZeroCopyUTF8String("http_requests"), @@ -774,7 +776,7 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures val lhs = QueryResult("someId", null, sampleCanary.map(rv => SerializedRangeVector(rv, schema))) val rhs = QueryResult("someId", null, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue val expectedLabels = List(Map(ZeroCopyUTF8String("__name__") -> ZeroCopyUTF8String("http_requests"), @@ -806,7 +808,7 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures val lhs = QueryResult("someId", null, sampleCanary.map(rv => SerializedRangeVector(rv, schema))) val rhs = QueryResult("someId", null, sampleRhsShuffled.map(rv => SerializedRangeVector(rv, schema))) // scalastyle:on - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, queryConfig) + val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), resSchemaTask, querySession) .toListL.runAsync.futureValue val expectedLabels = List(Map(ZeroCopyUTF8String("__name__") -> ZeroCopyUTF8String("http_requests"), diff --git a/query/src/test/scala/filodb/query/exec/HistToPromSeriesMapperSpec.scala b/query/src/test/scala/filodb/query/exec/HistToPromSeriesMapperSpec.scala index 6ea08e747a..0b9fcaca99 100644 --- a/query/src/test/scala/filodb/query/exec/HistToPromSeriesMapperSpec.scala +++ b/query/src/test/scala/filodb/query/exec/HistToPromSeriesMapperSpec.scala @@ -10,11 +10,11 @@ import filodb.core.metadata.Column.ColumnType import filodb.core.query._ import filodb.memory.format.{ZeroCopyUTF8String => ZCUTF8} import filodb.memory.format.vectors.HistogramWithBuckets -import filodb.query.QueryConfig class HistToPromSeriesMapperSpec extends FunSpec with Matchers with ScalaFutures { val config = ConfigFactory.load("application_test.conf").getConfig("filodb") val queryConfig = new QueryConfig(config.getConfig("query")) + val querySession = QuerySession(QueryContext(), queryConfig) import monix.execution.Scheduler.Implicits.global @@ -36,7 +36,7 @@ class HistToPromSeriesMapperSpec extends FunSpec with Matchers with ScalaFutures mapper.schema(sourceSchema).columns shouldEqual Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), ColumnInfo("value", ColumnType.DoubleColumn)) - val destObs = mapper.apply(sourceObs, queryConfig, 1000, sourceSchema, Nil) + val destObs = mapper.apply(sourceObs, querySession, 1000, sourceSchema, Nil) val destRvs = destObs.toListL.runAsync.futureValue // Should be 8 time series since there are 8 buckets @@ -71,7 +71,7 @@ class HistToPromSeriesMapperSpec extends FunSpec with Matchers with ScalaFutures mapper.schema(sourceSchema).columns shouldEqual Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), ColumnInfo("value", ColumnType.DoubleColumn)) - val destObs = mapper.apply(sourceObs, queryConfig, 1000, sourceSchema, Nil) + val destObs = mapper.apply(sourceObs, querySession, 1000, sourceSchema, Nil) val destRvs = destObs.toListL.runAsync.futureValue // Should be 10 time series since there are up to 10 buckets diff --git a/query/src/test/scala/filodb/query/exec/HistogramQuantileMapperSpec.scala b/query/src/test/scala/filodb/query/exec/HistogramQuantileMapperSpec.scala index 74a52b4ed1..52810a192a 100644 --- a/query/src/test/scala/filodb/query/exec/HistogramQuantileMapperSpec.scala +++ b/query/src/test/scala/filodb/query/exec/HistogramQuantileMapperSpec.scala @@ -11,12 +11,13 @@ import org.scalatest.concurrent.ScalaFutures import filodb.core.metadata.Column.ColumnType import filodb.core.query._ import filodb.memory.format.ZeroCopyUTF8String -import filodb.query.QueryConfig class HistogramQuantileMapperSpec extends FunSpec with Matchers with ScalaFutures { val config = ConfigFactory.load("application_test.conf").getConfig("filodb") val queryConfig = new QueryConfig(config.getConfig("query")) + val querySession = QuerySession(QueryContext(), queryConfig) + val rangeParams = RangeParams(100, 20, 200) import HistogramQuantileMapper._ import ZeroCopyUTF8String._ @@ -57,7 +58,7 @@ class HistogramQuantileMapperSpec extends FunSpec with Matchers with ScalaFuture expectedResult: Seq[(Map[ZeroCopyUTF8String, ZeroCopyUTF8String], Seq[(Int, Double)])]): Unit = { val hqMapper = HistogramQuantileMapper(Seq(StaticFuncArgs(q, rangeParams))) - val result = hqMapper.apply(Observable.fromIterable(histRvs), queryConfig, 10, + val result = hqMapper.apply(Observable.fromIterable(histRvs), querySession, 10, new ResultSchema(Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), ColumnInfo("value", ColumnType.DoubleColumn)), 1), Nil) .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 cab2054b0d..c82695a2f4 100644 --- a/query/src/test/scala/filodb/query/exec/InProcessPlanDispatcherSpec.scala +++ b/query/src/test/scala/filodb/query/exec/InProcessPlanDispatcherSpec.scala @@ -5,18 +5,20 @@ import java.util.concurrent.{Executors, TimeUnit} import scala.collection.immutable import scala.concurrent.ExecutionContext import scala.concurrent.duration.FiniteDuration + import com.typesafe.config.{Config, ConfigFactory} import monix.eval.Task import monix.execution.Scheduler import org.scalatest.{BeforeAndAfterAll, FunSpec, Matchers} import org.scalatest.concurrent.ScalaFutures import org.scalatest.time.{Millis, Seconds, Span} + import filodb.core.MetricsTestData.{builder, timeseriesDataset, timeseriesSchema} import filodb.core.TestData import filodb.core.binaryrecord2.{RecordBuilder, RecordContainer} import filodb.core.memstore.{FixedMaxPartitionsEvictionPolicy, SomeData, TimeSeriesMemStore} import filodb.core.metadata.{Column, Dataset, Schemas} -import filodb.core.query.{ColumnFilter, Filter, QueryContext} +import filodb.core.query.{ColumnFilter, Filter, QueryConfig, QueryContext, QuerySession} import filodb.core.store.{AllChunkScan, InMemoryMetaStore, NullColumnStore} import filodb.memory.MemFactory import filodb.memory.format.{SeqRowReader, ZeroCopyUTF8String} @@ -59,6 +61,7 @@ class InProcessPlanDispatcherSpec extends FunSpec with Matchers with ScalaFuture val config: Config = ConfigFactory.load("application_test.conf").getConfig("filodb") val queryConfig = new QueryConfig(config.getConfig("query")) + val querySession = QuerySession(QueryContext(), queryConfig) val policy = new FixedMaxPartitionsEvictionPolicy(20) val memStore = new TimeSeriesMemStore(config, new NullColumnStore, new InMemoryMetaStore(), Some(policy)) @@ -91,7 +94,7 @@ class InProcessPlanDispatcherSpec extends FunSpec with Matchers with ScalaFuture val dispatcher: PlanDispatcher = InProcessPlanDispatcher - val dummyDispatcher = DummyDispatcher(memStore, queryConfig) + val dummyDispatcher = DummyDispatcher(memStore, querySession) val execPlan1 = MultiSchemaPartitionsExec(QueryContext(), dummyDispatcher, timeseriesDataset.ref, 0, filters, AllChunkScan) @@ -119,7 +122,7 @@ class InProcessPlanDispatcherSpec extends FunSpec with Matchers with ScalaFuture val dispatcher: PlanDispatcher = InProcessPlanDispatcher - val dummyDispatcher = DummyDispatcher(memStore, queryConfig) + val dummyDispatcher = DummyDispatcher(memStore, querySession) val execPlan1 = MultiSchemaPartitionsExec(QueryContext(), dummyDispatcher, timeseriesDataset.ref, 0, filters, AllChunkScan) @@ -162,10 +165,10 @@ class InProcessPlanDispatcherSpec extends FunSpec with Matchers with ScalaFuture } } -case class DummyDispatcher(memStore: TimeSeriesMemStore, queryConfig: QueryConfig) extends PlanDispatcher { +case class DummyDispatcher(memStore: TimeSeriesMemStore, querySession: QuerySession) extends PlanDispatcher { // run locally withing any check. override def dispatch(plan: ExecPlan) (implicit sched: Scheduler): Task[QueryResponse] = { - plan.execute(memStore, queryConfig) + plan.execute(memStore, querySession) } } \ No newline at end of file diff --git a/query/src/test/scala/filodb/query/exec/LastSampleFunctionSpec.scala b/query/src/test/scala/filodb/query/exec/LastSampleFunctionSpec.scala index ca66feeafa..38f53f6ab0 100644 --- a/query/src/test/scala/filodb/query/exec/LastSampleFunctionSpec.scala +++ b/query/src/test/scala/filodb/query/exec/LastSampleFunctionSpec.scala @@ -28,7 +28,7 @@ class LastSampleFunctionSpec extends RawDataWindowingSpec { val lastSamplesIter = new SlidingWindowIterator(rv.rows, start, step, end, 0, LastSampleFunction, queryConfig) validateLastSamples(samples, lastSamplesIter, start, end, step) - val chunkedIter = new ChunkedWindowIteratorD(rv, start, step, end, w, chunkedLSFunc, queryConfig) + val chunkedIter = new ChunkedWindowIteratorD(rv, start, step, end, w, chunkedLSFunc, querySession) validateLastSamples(samples, chunkedIter, start, end, step) } } @@ -40,7 +40,7 @@ class LastSampleFunctionSpec extends RawDataWindowingSpec { val lastSamplesIter = new SlidingWindowIterator(rv.rows, start, step, end, 0, LastSampleFunction, queryConfig) validateLastSamples(samples, lastSamplesIter, start, end, step) - val chunkedIter = new ChunkedWindowIteratorD(rv, start, step, end, w, chunkedLSFunc, queryConfig) + val chunkedIter = new ChunkedWindowIteratorD(rv, start, step, end, w, chunkedLSFunc, querySession) validateLastSamples(samples, chunkedIter, start, end, step) } } @@ -53,7 +53,7 @@ class LastSampleFunctionSpec extends RawDataWindowingSpec { val lastSamplesIter = new SlidingWindowIterator(rv.rows, start, step, end, 0, LastSampleFunction, queryConfig) validateLastSamples(samples, lastSamplesIter, start, end, step) - val chunkedIter = new ChunkedWindowIteratorD(rv, start, step, end, w, chunkedLSFunc, queryConfig) + val chunkedIter = new ChunkedWindowIteratorD(rv, start, step, end, w, chunkedLSFunc, querySession) validateLastSamples(samples, chunkedIter, start, end, step) } } @@ -70,7 +70,7 @@ class LastSampleFunctionSpec extends RawDataWindowingSpec { start, step, end, 0, LastSampleFunction, queryConfig) validateLastSamples(samplesWithLongGap, lastSamplesIter, start, end, step) - val chunkedIter = new ChunkedWindowIteratorD(rvWithLongGap, start, step, end, w, chunkedLSFunc, queryConfig) + val chunkedIter = new ChunkedWindowIteratorD(rvWithLongGap, start, step, end, w, chunkedLSFunc, querySession) validateLastSamples(samplesWithLongGap, chunkedIter, start, end, step) } @@ -85,7 +85,7 @@ class LastSampleFunctionSpec extends RawDataWindowingSpec { start, step, end, 0, LastSampleFunction, queryConfig) validateLastSamples(samplesWithLongGap, lastSamplesIter, start, end, step) - val chunkedIter = new ChunkedWindowIteratorD(rvWithLongGap, start, step, end, w, chunkedLSFunc, queryConfig) + val chunkedIter = new ChunkedWindowIteratorD(rvWithLongGap, start, step, end, w, chunkedLSFunc, querySession) validateLastSamples(samplesWithLongGap, chunkedIter, start, end, step) } @@ -100,7 +100,7 @@ class LastSampleFunctionSpec extends RawDataWindowingSpec { start, step, end, 0, LastSampleFunction, queryConfig) validateLastSamples(samplesWithLongGap, lastSamplesIter, start, end, step) - val chunkedIter = new ChunkedWindowIteratorD(rvWithLongGap, start, step, end, w, chunkedLSFunc, queryConfig) + val chunkedIter = new ChunkedWindowIteratorD(rvWithLongGap, start, step, end, w, chunkedLSFunc, querySession) validateLastSamples(samplesWithLongGap, chunkedIter, start, end, step) } diff --git a/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala b/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala index 26e7920e46..a360807b1e 100644 --- a/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala @@ -16,7 +16,7 @@ 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, QueryContext, SerializedRangeVector} +import filodb.core.query.{ColumnFilter, Filter, QueryConfig, QueryContext, QuerySession, SerializedRangeVector} import filodb.core.store.{InMemoryMetaStore, NullColumnStore} import filodb.memory.format.{SeqRowReader, ZeroCopyUTF8String} import filodb.query._ @@ -28,6 +28,8 @@ class MetadataExecSpec extends FunSpec with Matchers with ScalaFutures with Befo val config = ConfigFactory.load("application_test.conf").getConfig("filodb") val queryConfig = new QueryConfig(config.getConfig("query")) + val querySession = QuerySession(QueryContext(), queryConfig) + val policy = new FixedMaxPartitionsEvictionPolicy(20) val memStore = new TimeSeriesMemStore(config, new NullColumnStore, new InMemoryMetaStore(), Some(policy)) @@ -86,7 +88,7 @@ class MetadataExecSpec extends FunSpec with Matchers with ScalaFutures with Befo val execPlan = LabelValuesExec(QueryContext(), dummyDispatcher, timeseriesDataset.ref, 0, filters, Seq("job"), 10) - val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue + val resp = execPlan.execute(memStore, querySession).runAsync.futureValue val result = resp match { case QueryResult(id, _, response) => { val rv = response(0) @@ -98,7 +100,6 @@ class MetadataExecSpec extends FunSpec with Matchers with ScalaFutures with Befo result shouldEqual jobQueryResult1 } - it ("should not return any rows for wrong column filters") { import ZeroCopyUTF8String._ val filters = Seq (ColumnFilter("__name__", Filter.Equals("http_req_total1".utf8)), @@ -107,7 +108,7 @@ class MetadataExecSpec extends FunSpec with Matchers with ScalaFutures with Befo val execPlan = PartKeysExec(QueryContext(), dummyDispatcher, timeseriesDataset.ref, 0, Schemas.promCounter.partition, filters, false, now-5000, now) - val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue + val resp = execPlan.execute(memStore, querySession).runAsync.futureValue resp match { case QueryResult(_, _, results) => results.size shouldEqual 1 results(0).rows.size shouldEqual 0 @@ -121,7 +122,7 @@ class MetadataExecSpec extends FunSpec with Matchers with ScalaFutures with Befo val execPlan = PartKeysExec(QueryContext(), dummyDispatcher, timeseriesDataset.ref, 0, Schemas.promCounter.partition, filters, false, now-5000, now) - val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue + val resp = execPlan.execute(memStore, querySession).runAsync.futureValue val result = resp match { case QueryResult(id, _, response) => response.size shouldEqual 1 @@ -142,7 +143,7 @@ class MetadataExecSpec extends FunSpec with Matchers with ScalaFutures with Befo val execPlan = PartKeysExec(QueryContext(sampleLimit = limit-1), dummyDispatcher, timeseriesDataset.ref, 0, Schemas.promCounter.partition, filters, false, now-5000, now) - val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue + val resp = execPlan.execute(memStore, querySession).runAsync.futureValue val result = resp match { case QueryResult(id, _, response) => { response.size shouldEqual 1 diff --git a/query/src/test/scala/filodb/query/exec/MultiSchemaPartitionsExecSpec.scala b/query/src/test/scala/filodb/query/exec/MultiSchemaPartitionsExecSpec.scala index bc31a90f90..9a712af2e3 100644 --- a/query/src/test/scala/filodb/query/exec/MultiSchemaPartitionsExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/MultiSchemaPartitionsExecSpec.scala @@ -1,24 +1,26 @@ package filodb.query.exec import scala.concurrent.duration._ + import com.typesafe.config.ConfigFactory import monix.eval.Task +import monix.execution.Scheduler import monix.execution.Scheduler.Implicits.global import org.scalatest.{BeforeAndAfterAll, FunSpec, Matchers} import org.scalatest.concurrent.ScalaFutures +import org.scalatest.exceptions.TestFailedException import org.scalatest.time.{Millis, Seconds, Span} + import filodb.core.{DatasetRef, QueryTimeoutException, TestData, Types} import filodb.core.binaryrecord2.RecordBuilder import filodb.core.memstore.{FixedMaxPartitionsEvictionPolicy, SchemaMismatch, SomeData, TimeSeriesMemStore} -import filodb.core.metadata.Schemas import filodb.core.metadata.Column.ColumnType.{DoubleColumn, HistogramColumn, LongColumn, TimestampColumn} +import filodb.core.metadata.Schemas import filodb.core.query._ import filodb.core.store.{AllChunkScan, InMemoryMetaStore, NullColumnStore, TimeRangeChunkScan} import filodb.memory.MemFactory import filodb.memory.format.{SeqRowReader, ZeroCopyUTF8String} import filodb.query._ -import monix.execution.Scheduler -import org.scalatest.exceptions.TestFailedException object MultiSchemaPartitionsExecSpec { val dummyDispatcher = new PlanDispatcher { @@ -42,6 +44,7 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu val config = ConfigFactory.load("application_test.conf").getConfig("filodb") val queryConfig = new QueryConfig(config.getConfig("query")) + val querySession = QuerySession(QueryContext(), queryConfig) val policy = new FixedMaxPartitionsEvictionPolicy(20) val memStore = new TimeSeriesMemStore(config, new NullColumnStore, new InMemoryMetaStore(), Some(policy)) @@ -101,7 +104,7 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu val execPlan = MultiSchemaPartitionsExec(QueryContext(), dummyDispatcher, dsRef, 0, filters, AllChunkScan) - val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue + val resp = execPlan.execute(memStore, querySession).runAsync.futureValue val result = resp.asInstanceOf[QueryResult] result.resultSchema.columns.map(_.colType) shouldEqual Seq(TimestampColumn, DoubleColumn) result.result.size shouldEqual 1 @@ -122,7 +125,7 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu val execPlan = MultiSchemaPartitionsExec(QueryContext(), dummyDispatcher, dsRef, 0, filters, TimeRangeChunkScan(startTime, endTime)) - val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue + val resp = execPlan.execute(memStore, querySession).runAsync.futureValue val result = resp.asInstanceOf[QueryResult] result.result.size shouldEqual 1 val dataRead = result.result(0).rows.map(r=>(r.getLong(0), r.getDouble(1))).toList @@ -138,7 +141,7 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu val execPlan = MultiSchemaPartitionsExec(QueryContext(), dummyDispatcher, dsRef, 0, filters, AllChunkScan) - val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue + val resp = execPlan.execute(memStore, querySession).runAsync.futureValue val result = resp.asInstanceOf[QueryResult] result.resultSchema.columns.isEmpty shouldEqual true result.result.size shouldEqual 0 @@ -152,7 +155,7 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu val execPlan = MultiSchemaPartitionsExec(QueryContext(), dummyDispatcher, MMD.dataset1.ref, 0, filters, TimeRangeChunkScan(100000L, 150000L), colName = Some("count")) - val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue + val resp = execPlan.execute(memStore, querySession).runAsync.futureValue val result = resp.asInstanceOf[QueryResult] result.resultSchema.columns.map(_.colType) shouldEqual Seq(TimestampColumn, LongColumn) result.result.size shouldEqual 1 @@ -168,7 +171,7 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu val execPlan = MultiSchemaPartitionsExec(QueryContext(), dummyDispatcher, dsRef, 0, filters, TimeRangeChunkScan(100000L, 150000L), colName=Some("h")) - val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue + val resp = execPlan.execute(memStore, querySession).runAsync.futureValue val result = resp.asInstanceOf[QueryResult] result.resultSchema.columns.map(_.colType) shouldEqual Seq(TimestampColumn, HistogramColumn) result.result.size shouldEqual 1 @@ -188,7 +191,7 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu val end = now - (numRawSamples-100) * reportingInterval execPlan.addRangeVectorTransformer(new PeriodicSamplesMapper(start, step, end, None, None, QueryContext())) - val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue + val resp = execPlan.execute(memStore, querySession).runAsync.futureValue val result = resp.asInstanceOf[QueryResult] result.resultSchema.columns.map(_.colType) shouldEqual Seq(TimestampColumn, DoubleColumn) // PSM should rename the double column to value always @@ -224,7 +227,7 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu val end = 185000L execPlan.addRangeVectorTransformer(new PeriodicSamplesMapper(start, step, end, None, None, QueryContext())) - val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue + val resp = execPlan.execute(memStore, querySession).runAsync.futureValue val result = resp.asInstanceOf[QueryResult] result.resultSchema.columns.map(_.colType) shouldEqual Seq(TimestampColumn, DoubleColumn) result.result.size shouldEqual 1 @@ -245,7 +248,7 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu val end = 185000L execPlan.addRangeVectorTransformer(new PeriodicSamplesMapper(start, step, end, None, None, QueryContext())) - val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue + val resp = execPlan.execute(memStore, querySession).runAsync.futureValue val result = resp.asInstanceOf[QueryResult] result.resultSchema.columns.map(_.colType) shouldEqual Seq(TimestampColumn, HistogramColumn) result.result.size shouldEqual 1 @@ -272,7 +275,7 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu execPlan.addRangeVectorTransformer(new PeriodicSamplesMapper(start, step, end, Some(300 * 1000), // [5m] Some(InternalRangeFunction.Rate), QueryContext(), rawSource = false)) - val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue + val resp = execPlan.execute(memStore, querySession).runAsync.futureValue val result = resp.asInstanceOf[QueryResult] result.resultSchema.columns.map(_.colType) shouldEqual Seq(TimestampColumn, DoubleColumn) result.result.size shouldEqual 1 @@ -289,7 +292,7 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu val execPlan = MultiSchemaPartitionsExec(QueryContext(), dummyDispatcher, dsRef, 0, Nil, AllChunkScan) - val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue + val resp = execPlan.execute(memStore, querySession).runAsync.futureValue val result = resp.asInstanceOf[QueryError] result.t.getClass shouldEqual classOf[SchemaMismatch] } @@ -298,7 +301,7 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu val execPlan = MultiSchemaPartitionsExec(QueryContext(), dummyDispatcher, dsRef, 0, Nil, AllChunkScan, schema = Some("prom-counter")) - val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue + val resp = execPlan.execute(memStore, querySession).runAsync.futureValue val result = resp.asInstanceOf[QueryResult] result.resultSchema.columns.map(_.colType) shouldEqual Seq(TimestampColumn, DoubleColumn) result.result.size shouldEqual 1 @@ -319,7 +322,7 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu Some(InternalRangeFunction.SumOverTime), QueryContext())) execPlan.addRangeVectorTransformer(AggregateMapReduce(AggregationOperator.Sum, Nil, Nil, Nil)) - val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue + val resp = execPlan.execute(memStore, querySession).runAsync.futureValue info(execPlan.printTree()) // Check that the "inner" SelectRawPartitionsExec has the right schema/columnIDs execPlan.finalPlan shouldBe a[SelectRawPartitionsExec] @@ -342,7 +345,7 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu // Add the histogram_max_quantile function to ExecPlan and make sure results are OK execPlan.addRangeVectorTransformer( exec.InstantVectorFunctionMapper(InstantFunctionId.HistogramMaxQuantile, Seq(StaticFuncArgs(0.99, RangeParams(0,0,0))))) - val resp2 = execPlan.execute(memStore, queryConfig).runAsync.futureValue + val resp2 = execPlan.execute(memStore, querySession).runAsync.futureValue val result2 = resp2.asInstanceOf[QueryResult] result2.resultSchema.columns.map(_.colType) shouldEqual Seq(TimestampColumn, DoubleColumn) result2.result.size shouldEqual 1 @@ -363,7 +366,7 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu val end = 185000L execPlan.addRangeVectorTransformer(new PeriodicSamplesMapper(start, step, end, None, None, QueryContext())) - val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue + val resp = execPlan.execute(memStore, querySession).runAsync.futureValue val result = resp.asInstanceOf[QueryResult] result.resultSchema.columns.map(_.colType) shouldEqual Seq(TimestampColumn, HistogramColumn, DoubleColumn) result.result.size shouldEqual 1 @@ -387,7 +390,7 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu // TODO: SelectChunkInfos should not require a raw schema val execPlan = SelectChunkInfosExec(QueryContext(), dummyDispatcher, dsRef, 0, filters, AllChunkScan, colName = Some("timestamp")) - val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue + val resp = execPlan.execute(memStore, querySession).runAsync.futureValue info(s"resp = $resp") val result = resp.asInstanceOf[QueryResult] result.result.size shouldEqual 1 @@ -419,7 +422,7 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu val execPlan = MultiSchemaPartitionsExec(QueryContext(sampleLimit = 999), dummyDispatcher, dsRef, 0, filters, AllChunkScan) - val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue + val resp = execPlan.execute(memStore, querySession).runAsync.futureValue val result = resp.asInstanceOf[QueryError] result.t.getClass shouldEqual classOf[BadQueryException] } @@ -432,7 +435,7 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu dummyDispatcher, dsRef, 0, filters, AllChunkScan) val thrown = intercept[TestFailedException] { - val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue + val resp = execPlan.execute(memStore, querySession).runAsync.futureValue val result = resp.asInstanceOf[QueryResult] } thrown.getCause.getClass shouldEqual classOf[QueryTimeoutException] diff --git a/query/src/test/scala/filodb/query/exec/PeriodicSamplesMapperSpec.scala b/query/src/test/scala/filodb/query/exec/PeriodicSamplesMapperSpec.scala index c61166c22d..c9b1ee4921 100644 --- a/query/src/test/scala/filodb/query/exec/PeriodicSamplesMapperSpec.scala +++ b/query/src/test/scala/filodb/query/exec/PeriodicSamplesMapperSpec.scala @@ -31,7 +31,8 @@ class PeriodicSamplesMapperSpec extends FunSpec with Matchers with ScalaFutures 500000L -> 200d ) val periodicSamplesVectorFnMapper = exec.PeriodicSamplesMapper(100000L, 100000, 600000L, None, None, QueryContext()) - val resultObs = periodicSamplesVectorFnMapper(Observable.fromIterable(Seq(rv)), queryConfig, 1000, resultSchema, Nil) + val resultObs = periodicSamplesVectorFnMapper(Observable.fromIterable(Seq(rv)), + querySession, 1000, resultSchema, Nil) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map (r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN)) @@ -54,7 +55,8 @@ class PeriodicSamplesMapperSpec extends FunSpec with Matchers with ScalaFutures val periodicSamplesVectorFnMapper = exec.PeriodicSamplesMapper(100100L, 100000, 600100L, None, None, QueryContext(), Nil, Some(100)) - val resultObs = periodicSamplesVectorFnMapper(Observable.fromIterable(Seq(rv)), queryConfig, 1000, resultSchema, Nil) + val resultObs = periodicSamplesVectorFnMapper(Observable.fromIterable(Seq(rv)), querySession, + 1000, resultSchema, Nil) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map (r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN)) diff --git a/query/src/test/scala/filodb/query/exec/TimestampFunctionSpec.scala b/query/src/test/scala/filodb/query/exec/TimestampFunctionSpec.scala index 86934d50ef..e2bac263ae 100644 --- a/query/src/test/scala/filodb/query/exec/TimestampFunctionSpec.scala +++ b/query/src/test/scala/filodb/query/exec/TimestampFunctionSpec.scala @@ -13,7 +13,7 @@ class TimestampFunctionSpec extends RawDataWindowingSpec { val list = rv.rows.map(x => (x.getLong(0), x.getDouble(1))).toList val chunkedIt = new ChunkedWindowIteratorD(rv, 100000, 5000, 120000, w, - new TimestampChunkedFunction, queryConfig) + new TimestampChunkedFunction, querySession) val aggregated = chunkedIt.map(x => (x.getLong(0), x.getDouble(1))).toList val expectedResult = List((100000, 100), (105000, 100), (110000, 110), (115000, 110), (120000, 120)) @@ -27,7 +27,7 @@ class TimestampFunctionSpec extends RawDataWindowingSpec { val list = rv.rows.map(x => (x.getLong(0), x.getDouble(1))).toList val chunkedIt = new ChunkedWindowIteratorD(rv, 95000, 50000, 450000, w, - new TimestampChunkedFunction, queryConfig) + new TimestampChunkedFunction, querySession) val aggregated = chunkedIt.map(x => (x.getLong(0), x.getDouble(1))).toList val expectedResult = List((95000, Double.NaN), (145000, 120d), (195000, 120d), (245000, 120d), (295000, 120d), (345000, 120d), (395000, 120d), (445000, Double.NaN)) diff --git a/query/src/test/scala/filodb/query/exec/WindowIteratorSpec.scala b/query/src/test/scala/filodb/query/exec/WindowIteratorSpec.scala index 3ec4a0e3e1..cd3b1f809a 100644 --- a/query/src/test/scala/filodb/query/exec/WindowIteratorSpec.scala +++ b/query/src/test/scala/filodb/query/exec/WindowIteratorSpec.scala @@ -210,7 +210,7 @@ class WindowIteratorSpec extends RawDataWindowingSpec { val chunkedIt = new ChunkedWindowIteratorD(rv, 50000L, 100000, 1100000L, 100000, RangeFunction(tsResSchema, Some(InternalRangeFunction.SumOverTime), ColumnType.DoubleColumn, queryConfig, - useChunked = true).asChunkedD, queryConfig) + useChunked = true).asChunkedD, querySession) chunkedIt.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual windowResults } @@ -245,7 +245,7 @@ class WindowIteratorSpec extends RawDataWindowingSpec { val rv = timeValueRV(samples) val chunkedIt = new ChunkedWindowIteratorD(rv, 1548191496000L, 15000, 1548191796000L, 300000, RangeFunction(tsResSchema, - Some(Rate), ColumnType.DoubleColumn, queryConfig, useChunked = true).asChunkedD, queryConfig) + Some(Rate), ColumnType.DoubleColumn, queryConfig, useChunked = true).asChunkedD, querySession) chunkedIt.foreach { v => windowResults.find(a => a._1 == v.timestamp).foreach(b => v.value shouldEqual b._2 +- 0.0000000001) } @@ -292,7 +292,7 @@ class WindowIteratorSpec extends RawDataWindowingSpec { val chunkedWinIt = new ChunkedWindowIteratorD(rv, 1540845090000L, 15000, 1540855905000L, 180000, RangeFunction(tsResSchema, - Some(Last), ColumnType.DoubleColumn, queryConfig, useChunked = true).asChunkedD, queryConfig) + Some(Last), ColumnType.DoubleColumn, queryConfig, useChunked = true).asChunkedD, querySession) chunkedWinIt.map(r => (r.getLong(0), r.getDouble(1))).toList.filter(!_._2.isNaN) shouldEqual windowResults } @@ -359,7 +359,7 @@ class WindowIteratorSpec extends RawDataWindowingSpec { val chunkedWinIt = new ChunkedWindowIteratorD(rv, 1540845090000L, 15000, 1540855905000L, queryConfig.staleSampleAfterMs, RangeFunction(tsResSchema, - None, ColumnType.DoubleColumn, queryConfig, useChunked = true).asChunkedD, queryConfig) + None, ColumnType.DoubleColumn, queryConfig, useChunked = true).asChunkedD, querySession) chunkedWinIt.map(r => (r.getLong(0), r.getDouble(1))).toList.filter(!_._2.isNaN) shouldEqual windowResults } @@ -389,7 +389,7 @@ class WindowIteratorSpec extends RawDataWindowingSpec { val chunkedWinIt = new ChunkedWindowIteratorD(rv, 100000L, 100000, 600000L, queryConfig.staleSampleAfterMs + 1, RangeFunction(tsResSchema, - None, ColumnType.DoubleColumn, queryConfig, useChunked = true).asChunkedD, queryConfig) + None, ColumnType.DoubleColumn, queryConfig, useChunked = true).asChunkedD, querySession) chunkedWinIt.map(r => (r.getLong(0), r.getDouble(1))).toList.filter(!_._2.isNaN) shouldEqual windowResults } @@ -425,7 +425,7 @@ class WindowIteratorSpec extends RawDataWindowingSpec { val chunkedIt = new ChunkedWindowIteratorD(rv, 50000L, 100000, 700000L, 100000, RangeFunction(tsResSchema, Some(InternalRangeFunction.AvgOverTime), ColumnType.DoubleColumn, queryConfig, - useChunked = true).asChunkedD, queryConfig) + useChunked = true).asChunkedD, querySession) chunkedIt.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual windowResults } @@ -461,7 +461,7 @@ class WindowIteratorSpec extends RawDataWindowingSpec { val chunkedIt = new ChunkedWindowIteratorD(rv, 50000L, 100000, 700000L, 100000, RangeFunction(tsResSchema, Some(InternalRangeFunction.CountOverTime), ColumnType.DoubleColumn, queryConfig, - useChunked = true).asChunkedD, queryConfig) + useChunked = true).asChunkedD, querySession) chunkedIt.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual windowResults } @@ -500,7 +500,7 @@ class WindowIteratorSpec extends RawDataWindowingSpec { val chunkedItAvg = new ChunkedWindowIteratorD(rvAvg, 50000L, 100000, 750000L, 100000, RangeFunction(dsResSchema, rangeFunc, ColumnType.DoubleColumn, queryConfig, - useChunked = true).asChunkedD, queryConfig) + useChunked = true).asChunkedD, querySession) chunkedItAvg.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual avgWindowResults val rvCnt = timeValueRvDownsample(samples, Array(0, 4)) @@ -515,7 +515,7 @@ class WindowIteratorSpec extends RawDataWindowingSpec { val cntFunc = RangeFunction.downsampleRangeFunction(Some(InternalRangeFunction.CountOverTime)) val chunkedItCnt = new ChunkedWindowIteratorD(rvCnt, 50000L, 100000, 750000L, 100000, RangeFunction(dsResSchema, cntFunc, - ColumnType.DoubleColumn, queryConfig, useChunked = true).asChunkedD, queryConfig) + ColumnType.DoubleColumn, queryConfig, useChunked = true).asChunkedD, querySession) chunkedItCnt.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual countWindowResults } @@ -550,7 +550,7 @@ class WindowIteratorSpec extends RawDataWindowingSpec { val chunkedIt = new ChunkedWindowIteratorD(rv, 50000L, 100000, 700000L, 100000, RangeFunction(tsResSchema, Some(InternalRangeFunction.MinOverTime), ColumnType.DoubleColumn, queryConfig, - useChunked = true).asChunkedD, queryConfig) + useChunked = true).asChunkedD, querySession) chunkedIt.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual windowResults } @@ -586,7 +586,7 @@ class WindowIteratorSpec extends RawDataWindowingSpec { val chunkedIt = new ChunkedWindowIteratorD(rv, 50000L, 100000, 700000L, 100000, RangeFunction(tsResSchema, Some(InternalRangeFunction.MaxOverTime), ColumnType.DoubleColumn, queryConfig, - useChunked = true).asChunkedD, queryConfig) + useChunked = true).asChunkedD, querySession) chunkedIt.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual windowResults } } 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 ce871f1462..7a43ae816a 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/AbsentFunctionSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/AbsentFunctionSpec.scala @@ -2,21 +2,23 @@ package filodb.query.exec.rangefn import com.typesafe.config.{Config, ConfigFactory} -import filodb.core.MetricsTestData -import filodb.core.query.Filter.{Equals, NotEqualsRegex} -import filodb.core.query._ -import filodb.memory.format.{RowReader, ZeroCopyUTF8String} -import filodb.query.{QueryConfig, exec} import monix.execution.Scheduler.Implicits.global import monix.reactive.Observable -import org.scalatest.concurrent.ScalaFutures import org.scalatest.{FunSpec, Matchers} +import org.scalatest.concurrent.ScalaFutures + +import filodb.core.MetricsTestData +import filodb.core.query._ +import filodb.core.query.Filter.{Equals, NotEqualsRegex} +import filodb.memory.format.{RowReader, ZeroCopyUTF8String} +import filodb.query.exec class AbsentFunctionSpec extends FunSpec with Matchers with ScalaFutures { val config: Config = ConfigFactory.load("application_test.conf").getConfig("filodb") val resultSchema = ResultSchema(MetricsTestData.timeseriesSchema.infosFromIDs(0 to 1), 1) val queryConfig = new QueryConfig(config.getConfig("query")) - + val querySession = QuerySession(QueryContext(), queryConfig) + val testKey1 = CustomRangeVectorKey( Map(ZeroCopyUTF8String("metric") -> ZeroCopyUTF8String("test1"), ZeroCopyUTF8String("src") -> ZeroCopyUTF8String("source-value"), @@ -67,7 +69,7 @@ class AbsentFunctionSpec extends FunSpec with Matchers with ScalaFutures { ZeroCopyUTF8String("instance") -> ZeroCopyUTF8String("instance1")) val expectedRows = List(1.0, 1.0, 1.0, 1.0, 1.0, 1.0) val absentFunctionMapper = exec.AbsentFunctionMapper(columnFilter, RangeParams(1, 2, 11), "metric") - val resultObs = absentFunctionMapper(Observable.fromIterable(emptySample), queryConfig, 1000, resultSchema, Nil) + val resultObs = absentFunctionMapper(Observable.fromIterable(emptySample), querySession, 1000, resultSchema, Nil) val result = resultObs.toListL.runAsync.futureValue result.size shouldEqual (1) val keys = result.map(_.key.labelValues) @@ -79,7 +81,7 @@ class AbsentFunctionSpec extends FunSpec with Matchers with ScalaFutures { it("should not generate range vector when sample is present") { val columnFilter = Seq(ColumnFilter("host", Equals("host1")), ColumnFilter("instance", Equals("instance1"))) val absentFunctionMapper = exec.AbsentFunctionMapper(columnFilter, RangeParams(1, 20, 1), "metric") - val resultObs = absentFunctionMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema, Nil) + val resultObs = absentFunctionMapper(Observable.fromIterable(testSample), querySession, 1000, resultSchema, Nil) val result = resultObs.toListL.runAsync.futureValue val keys = result.map(_.key.labelValues) val rows = result.flatMap(_.rows.map(_.getDouble(1)).toList) @@ -90,7 +92,7 @@ class AbsentFunctionSpec extends FunSpec with Matchers with ScalaFutures { val columnFilter = Seq(ColumnFilter("host", NotEqualsRegex("host1"))) val expectedRows = List(1.0, 1.0, 1.0, 1.0, 1.0, 1.0) val absentFunctionMapper = exec.AbsentFunctionMapper(columnFilter, RangeParams(1, 2, 11), "metric") - val resultObs = absentFunctionMapper(Observable.fromIterable(emptySample), queryConfig, 1000, resultSchema, Nil) + val resultObs = absentFunctionMapper(Observable.fromIterable(emptySample), querySession, 1000, resultSchema, Nil) val result = resultObs.toListL.runAsync.futureValue result.size shouldEqual (1) val keys = result.map(_.key.labelValues) @@ -105,7 +107,7 @@ class AbsentFunctionSpec extends FunSpec with Matchers with ScalaFutures { ZeroCopyUTF8String("instance") -> ZeroCopyUTF8String("instance1")) val expectedRows = List(1.0, 1.0, 1.0, 1.0, 1.0, 1.0) val absentFunctionMapper = exec.AbsentFunctionMapper(columnFilter, RangeParams(1, 2, 11), "metric") - val resultObs = absentFunctionMapper(Observable.fromIterable(emptySample), queryConfig, 1000, resultSchema, Nil) + val resultObs = absentFunctionMapper(Observable.fromIterable(emptySample), querySession, 1000, resultSchema, Nil) val result = resultObs.toListL.runAsync.futureValue result.size shouldEqual (1) val keys = result.map(_.key.labelValues) @@ -119,7 +121,7 @@ class AbsentFunctionSpec extends FunSpec with Matchers with ScalaFutures { ZeroCopyUTF8String("instance") -> ZeroCopyUTF8String("instance1")) val expectedRows = List((3000, 1.0)) val absentFunctionMapper = exec.AbsentFunctionMapper(columnFilter, RangeParams(1, 1, 3), "metric") - val resultObs = absentFunctionMapper(Observable.fromIterable(testSampleNan), queryConfig, 1000, resultSchema, Nil) + val resultObs = absentFunctionMapper(Observable.fromIterable(testSampleNan), querySession, 1000, resultSchema, Nil) val result = resultObs.toListL.runAsync.futureValue result.size shouldEqual (1) val keys = result.map(_.key.labelValues) 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 5fd6809a4d..84407eeaed 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala @@ -1,16 +1,17 @@ package filodb.query.exec.rangefn -import scala.util.Random import scala.collection.mutable.ArrayBuffer +import scala.util.Random + import com.typesafe.config.ConfigFactory import org.scalatest.{BeforeAndAfterAll, FunSpec, Matchers} + +import filodb.core.{MetricsTestData, QueryTimeoutException, TestData, MachineMetricsData => MMD} import filodb.core.memstore.{TimeSeriesPartition, TimeSeriesPartitionSpec, WriteBufferPool} -import filodb.core.query.{QueryContext, RangeParams, RawDataRangeVector, TransientHistMaxRow, TransientHistRow, TransientRow} +import filodb.core.query._ import filodb.core.store.AllChunkScan -import filodb.core.{MetricsTestData, QueryTimeoutException, TestData, MachineMetricsData => MMD} import filodb.memory._ import filodb.memory.format.{TupleRowReader, vectors => bv} -import filodb.query.QueryConfig import filodb.query.exec._ /** @@ -133,6 +134,7 @@ trait RawDataWindowingSpec extends FunSpec with Matchers with BeforeAndAfterAll val config = ConfigFactory.load("application_test.conf").getConfig("filodb") val queryConfig = new QueryConfig(config.getConfig("query")) + val querySession = QuerySession(QueryContext(), queryConfig) // windowSize and step are in number of elements of the data def numWindows(data: Seq[Any], windowSize: Int, step: Int): Int = data.sliding(windowSize, step).length @@ -193,7 +195,7 @@ trait RawDataWindowingSpec extends FunSpec with Matchers with BeforeAndAfterAll val windowStartTS = defaultStartTS + windowTime val stepTimeMillis = step.toLong * pubFreq val windowEndTS = windowStartTS + (numWindows(data, windowSize, step) - 1) * stepTimeMillis - new ChunkedWindowIteratorD(rv, windowStartTS, stepTimeMillis, windowEndTS, windowTime, func, queryConfig) + new ChunkedWindowIteratorD(rv, windowStartTS, stepTimeMillis, windowEndTS, windowTime, func, querySession) } def chunkedWindowItHist[R <: TransientHistRow](data: Seq[Seq[Any]], @@ -207,7 +209,7 @@ trait RawDataWindowingSpec extends FunSpec with Matchers with BeforeAndAfterAll val stepTimeMillis = step.toLong * pubFreq val windowEndTS = windowStartTS + (numWindows(data, windowSize, step) - 1) * stepTimeMillis new ChunkedWindowIteratorH(rv, windowStartTS, stepTimeMillis, windowEndTS, windowTime, - func.asInstanceOf[ChunkedRangeFunction[TransientHistRow]], queryConfig, QueryContext(), row) + func.asInstanceOf[ChunkedRangeFunction[TransientHistRow]], querySession, row) } def chunkedWindowItHist(data: Seq[Seq[Any]], @@ -396,7 +398,7 @@ class AggrOverTimeFunctionsSpec extends RawDataWindowingSpec { val step = 20 val chunkedIt = new ChunkedWindowIteratorD(rv, 100000, 20000, 150000, 30000, - new ChangesChunkedFunctionD(), queryConfig) + new ChangesChunkedFunctionD(), querySession) val aggregated = chunkedIt.map(x => (x.getLong(0), x.getDouble(1))).toList aggregated shouldEqual List((100000, 0.0), (120000, 2.0), (140000, 2.0)) } @@ -416,26 +418,26 @@ class AggrOverTimeFunctionsSpec extends RawDataWindowingSpec { for (i <- 0 until n) { var rv = timeValueRV(twoSampleData) val chunkedItTwoSample = new ChunkedWindowIteratorD(rv, 110000, 120000, 150000, 30000, - new QuantileOverTimeChunkedFunctionD(Seq(quantiles(i))), queryConfig) + new QuantileOverTimeChunkedFunctionD(Seq(quantiles(i))), querySession) val aggregated2 = chunkedItTwoSample.map(_.getDouble(1)).toBuffer aggregated2(0) shouldEqual twoSampleDataResponses(i) +- 0.0000000001 rv = timeValueRV(threeSampleData) val chunkedItThreeSample = new ChunkedWindowIteratorD(rv, 120000, 20000, 130000, 50000, - new QuantileOverTimeChunkedFunctionD(Seq(quantiles(i))), queryConfig) + new QuantileOverTimeChunkedFunctionD(Seq(quantiles(i))), querySession) val aggregated3 = chunkedItThreeSample.map(_.getDouble(1)).toBuffer aggregated3(0) shouldEqual threeSampleDataResponses(i) +- 0.0000000001 rv = timeValueRV(unevenSampleData) val chunkedItUnevenSample = new ChunkedWindowIteratorD(rv, 120000, 20000, 130000, 30000, - new QuantileOverTimeChunkedFunctionD(Seq(quantiles(i))), queryConfig) + new QuantileOverTimeChunkedFunctionD(Seq(quantiles(i))), querySession) val aggregatedUneven = chunkedItUnevenSample.map(_.getDouble(1)).toBuffer aggregatedUneven(0) shouldEqual unevenSampleDataResponses(i) +- 0.0000000001 } val emptyData = Seq() var rv = timeValueRV(emptyData) val chunkedItNoSample = new ChunkedWindowIteratorD(rv, 110000, 120000, 150000, 30000, - new QuantileOverTimeChunkedFunctionD(Seq(StaticFuncArgs(0.5, rangeParams))), queryConfig) + new QuantileOverTimeChunkedFunctionD(Seq(StaticFuncArgs(0.5, rangeParams))), querySession) val aggregatedEmpty = chunkedItNoSample.map(_.getDouble(1)).toBuffer aggregatedEmpty(0) isNaN @@ -519,25 +521,25 @@ class AggrOverTimeFunctionsSpec extends RawDataWindowingSpec { var rv = timeValueRV(positiveTrendData2) val chunkedIt2 = new ChunkedWindowIteratorD(rv, 160000, 100000, 180000, 100000, - new HoltWintersChunkedFunctionD(params), queryConfig) + new HoltWintersChunkedFunctionD(params), querySession) val aggregated2 = chunkedIt2.map(_.getDouble(1)).toBuffer aggregated2(0) shouldEqual holt_winters(positiveTrendData2) rv = timeValueRV(positiveTrendData3) val chunkedIt3 = new ChunkedWindowIteratorD(rv, 160000, 100000, 180000, 100000, - new HoltWintersChunkedFunctionD(params), queryConfig) + new HoltWintersChunkedFunctionD(params), querySession) val aggregated3 = chunkedIt3.map(_.getDouble(1)).toBuffer aggregated3(0) shouldEqual holt_winters(positiveTrendData3) rv = timeValueRV(positiveTrendData4) val chunkedIt4 = new ChunkedWindowIteratorD(rv, 160000, 100000, 180000, 100000, - new HoltWintersChunkedFunctionD(params), queryConfig) + new HoltWintersChunkedFunctionD(params), querySession) val aggregated4 = chunkedIt4.map(_.getDouble(1)).toBuffer aggregated4(0) shouldEqual holt_winters(positiveTrendData4) rv = timeValueRV(negativeTrendData2) val chunkedItNeg2 = new ChunkedWindowIteratorD(rv, 160000, 100000, 180000, 100000, - new HoltWintersChunkedFunctionD(params), queryConfig) + new HoltWintersChunkedFunctionD(params), querySession) val aggregatedNeg2 = chunkedItNeg2.map(_.getDouble(1)).toBuffer aggregatedNeg2(0) shouldEqual holt_winters(negativeTrendData2) @@ -627,7 +629,8 @@ class AggrOverTimeFunctionsSpec extends RawDataWindowingSpec { } } - it("it should correctly calculate sum_over_time, avg_over_time, stddev_over_time & zscore when the sequence contains NaNs or is empty") { + it("it should correctly calculate sum_over_time, avg_over_time, stddev_over_time & " + + "zscore when the sequence contains NaNs or is empty") { val test_data = Seq( Seq(15900.0, 15920.0, 15940.0, 15960.0, 15980.0, 16000.0, 16020.0), Seq(-15900.0, -15920.0, -15940.0, -15960.0, -15980.0, -16000.0), @@ -644,43 +647,44 @@ class AggrOverTimeFunctionsSpec extends RawDataWindowingSpec { val rv = timeValueRV(data) // sum_over_time - val chunkedItSumOverTime = new ChunkedWindowIteratorD(rv, 160000, 100000, 180000, 100000, new SumOverTimeChunkedFunctionD(), queryConfig) + val chunkedItSumOverTime = new ChunkedWindowIteratorD(rv, 160000, 100000, 180000, 100000, new SumOverTimeChunkedFunctionD(), querySession) val aggregatedSumOverTime = chunkedItSumOverTime.map(_.getDouble(1)).toBuffer - if (aggregatedSumOverTime(0).isNaN) aggregatedSumOverTime(0).isNaN shouldBe true else aggregatedSumOverTime(0) shouldBe sumWithNaN(data) + if (aggregatedSumOverTime(0).isNaN) aggregatedSumOverTime(0).isNaN shouldBe true + else aggregatedSumOverTime(0) shouldBe sumWithNaN(data) // avg_over_time - val chunkedItAvgOverTime = new ChunkedWindowIteratorD(rv, 160000, 100000, 180000, 100000, new AvgOverTimeChunkedFunctionD(), queryConfig) + val chunkedItAvgOverTime = new ChunkedWindowIteratorD(rv, 160000, 100000, 180000, 100000, new AvgOverTimeChunkedFunctionD(), querySession) val aggregatedAvgOverTime = chunkedItAvgOverTime.map(_.getDouble(1)).toBuffer - if (aggregatedAvgOverTime(0).isNaN) aggregatedAvgOverTime(0).isNaN shouldBe true else aggregatedAvgOverTime(0) shouldBe avgWithNaN(data) + if (aggregatedAvgOverTime(0).isNaN) aggregatedAvgOverTime(0).isNaN shouldBe true + else aggregatedAvgOverTime(0) shouldBe avgWithNaN(data) // stdvar_over_time - val chunkedItStdVarOverTime = new ChunkedWindowIteratorD(rv, 160000, 100000, 180000, 100000, new StdVarOverTimeChunkedFunctionD(), queryConfig) + val chunkedItStdVarOverTime = new ChunkedWindowIteratorD(rv, 160000, 100000, 180000, 100000, new StdVarOverTimeChunkedFunctionD(), querySession) val aggregatedStdVarOverTime = chunkedItStdVarOverTime.map(_.getDouble(1)).toBuffer - if (aggregatedStdVarOverTime(0).isNaN) aggregatedStdVarOverTime(0).isNaN shouldBe true else aggregatedStdVarOverTime(0) shouldBe stdVarWithNaN(data) + if (aggregatedStdVarOverTime(0).isNaN) aggregatedStdVarOverTime(0).isNaN shouldBe true + else aggregatedStdVarOverTime(0) shouldBe stdVarWithNaN(data) // stddev_over_time - val chunkedItStdDevOverTime = new ChunkedWindowIteratorD(rv, 160000, 100000, 180000, 100000, new StdDevOverTimeChunkedFunctionD(), queryConfig) + val chunkedItStdDevOverTime = new ChunkedWindowIteratorD(rv, 160000, 100000, 180000, 100000, new StdDevOverTimeChunkedFunctionD(), querySession) val aggregatedStdDevOverTime = chunkedItStdDevOverTime.map(_.getDouble(1)).toBuffer - if (aggregatedStdDevOverTime(0).isNaN) aggregatedStdDevOverTime(0).isNaN shouldBe true else aggregatedStdDevOverTime(0) shouldBe Math.sqrt(stdVarWithNaN(data)) + if (aggregatedStdDevOverTime(0).isNaN) aggregatedStdDevOverTime(0).isNaN shouldBe true + else aggregatedStdDevOverTime(0) shouldBe Math.sqrt(stdVarWithNaN(data)) // zscore - val chunkedItZscore = new ChunkedWindowIteratorD(rv, 160000, 100000, 180000, 100000, new ZScoreChunkedFunctionD(), queryConfig) + val chunkedItZscore = new ChunkedWindowIteratorD(rv, 160000, 100000, 180000, 100000, new ZScoreChunkedFunctionD(), querySession) val aggregatedZscore = chunkedItZscore.map(_.getDouble(1)).toBuffer - if (aggregatedZscore(0).isNaN) aggregatedZscore(0).isNaN shouldBe true else aggregatedZscore(0) shouldBe z_score(data) + if (aggregatedZscore(0).isNaN) aggregatedZscore(0).isNaN shouldBe true + else aggregatedZscore(0) shouldBe z_score(data) } } it("should throw QueryTimeoutException when query processing time is greater than timeout") { the[QueryTimeoutException] thrownBy { - var data = Seq(1.5, 2.5, 3.5, 4.5, 5.5) + val data = Seq(1.5, 2.5, 3.5, 4.5, 5.5) val rv = timeValueRV(data) - val list = rv.rows.map(x => (x.getLong(0), x.getDouble(1))).toList - - val windowSize = 100 - val step = 20 - val chunkedIt = new ChunkedWindowIteratorD(rv, 100000, 20000, 150000, 30000, - new ChangesChunkedFunctionD(), queryConfig, QueryContext(submitTime = System.currentTimeMillis() - 180000)) + new ChangesChunkedFunctionD(), + QuerySession(QueryContext(submitTime = System.currentTimeMillis() - 180000), queryConfig)) val aggregated = chunkedIt.map(x => (x.getLong(0), x.getDouble(1))).toList aggregated shouldEqual List((100000, 0.0), (120000, 2.0), (140000, 2.0)) } should have message "Query timeout in filodb.core.store.WindowedChunkIterator after 180 seconds" diff --git a/query/src/test/scala/filodb/query/exec/rangefn/BinaryOperatorSpec.scala b/query/src/test/scala/filodb/query/exec/rangefn/BinaryOperatorSpec.scala index 7d1fbfe905..dffffc4b47 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/BinaryOperatorSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/BinaryOperatorSpec.scala @@ -7,7 +7,7 @@ import monix.reactive.Observable import org.scalatest.{FunSpec, Matchers} import org.scalatest.concurrent.ScalaFutures import filodb.core.MetricsTestData -import filodb.core.query.{CustomRangeVectorKey, RangeParams, RangeVector, RangeVectorKey, ResultSchema, TransientRow} +import filodb.core.query._ import filodb.memory.format.{RowReader, ZeroCopyUTF8String} import filodb.query._ import filodb.query.exec.{StaticFuncArgs, TimeFuncArgs} @@ -32,6 +32,8 @@ class BinaryOperatorSpec extends FunSpec with Matchers with ScalaFutures { new TransientRow(4L, 94935.1523d)).iterator }) val queryConfig = new QueryConfig(config.getConfig("query")) + val querySession = QuerySession(QueryContext(), queryConfig) + val rand = new Random() val error = 0.00000001d val scalar = 5.0 @@ -233,7 +235,7 @@ class BinaryOperatorSpec extends FunSpec with Matchers with ScalaFutures { // ceil val expectedVal = sampleBase.map(_.rows.map(v => scala.math.floor(v.getDouble(1)))) val binaryOpMapper = exec.ScalarOperationMapper(BinaryOperator.ADD, true, Seq(StaticFuncArgs(scalar, RangeParams(0,0,0)))) - val resultObs = binaryOpMapper(Observable.fromIterable(sampleBase), queryConfig, 1000, resultSchema) + val resultObs = binaryOpMapper(Observable.fromIterable(sampleBase), querySession, 1000, resultSchema) val result = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) expectedVal.zip(result).foreach { case (ex, res) => { @@ -269,7 +271,7 @@ class BinaryOperatorSpec extends FunSpec with Matchers with ScalaFutures { ) val expectedVal = samples.map(_.rows.map(v => v.getDouble(1) * 2)) val binaryOpMapper = exec.ScalarOperationMapper(BinaryOperator.ADD, true, Seq(TimeFuncArgs(RangeParams(1,1,4)))) - val resultObs = binaryOpMapper(Observable.fromIterable(samples), queryConfig, 1000, resultSchema) + val resultObs = binaryOpMapper(Observable.fromIterable(samples), querySession, 1000, resultSchema) val result = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) result.foreach(x=> println(x.toList)) expectedVal.zip(result).foreach { @@ -293,7 +295,7 @@ class BinaryOperatorSpec extends FunSpec with Matchers with ScalaFutures { ) val expectedVal = samples.map(_.rows.map(v => scala.math.floor(v.getDouble(1)))) val binaryOpMapper = exec.ScalarOperationMapper(BinaryOperator.ADD, true, Seq(StaticFuncArgs(1571267260, RangeParams(0,0,0)))) - val resultObs = binaryOpMapper(Observable.fromIterable(samples), queryConfig, 1000, resultSchema) + val resultObs = binaryOpMapper(Observable.fromIterable(samples), querySession, 1000, resultSchema) val result = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) expectedVal.zip(result).foreach { case (ex, res) => { @@ -308,7 +310,7 @@ class BinaryOperatorSpec extends FunSpec with Matchers with ScalaFutures { private def applyBinaryOperationAndAssertResult(samples: Array[RangeVector], expectedVal: Array[Iterator[Double]], binOp: BinaryOperator, scalar: Double, scalarOnLhs: Boolean): Unit = { val scalarOpMapper = exec.ScalarOperationMapper(binOp, scalarOnLhs, Seq(StaticFuncArgs(scalar, RangeParams(0,0,0)))) - val resultObs = scalarOpMapper(Observable.fromIterable(samples), queryConfig, 1000, resultSchema) + val resultObs = scalarOpMapper(Observable.fromIterable(samples), querySession, 1000, resultSchema) val result = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) expectedVal.zip(result).foreach { diff --git a/query/src/test/scala/filodb/query/exec/rangefn/InstantFunctionSpec.scala b/query/src/test/scala/filodb/query/exec/rangefn/InstantFunctionSpec.scala index e24b601eb9..be21ac981e 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/InstantFunctionSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/InstantFunctionSpec.scala @@ -160,28 +160,28 @@ class InstantFunctionSpec extends RawDataWindowingSpec with ScalaFutures { // clamp_max the[IllegalArgumentException] thrownBy { val instantVectorFnMapper1 = exec.InstantVectorFunctionMapper(InstantFunctionId.ClampMax) - val resultObs = instantVectorFnMapper1(Observable.fromIterable(sampleBase), queryConfig, 1000, resultSchema, Nil) + val resultObs = instantVectorFnMapper1(Observable.fromIterable(sampleBase), querySession, 1000, resultSchema, Nil) val result = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1)).toList) } should have message "requirement failed: Cannot use ClampMax without providing a upper limit of max." // clamp_min the[IllegalArgumentException] thrownBy { val instantVectorFnMapper3 = exec.InstantVectorFunctionMapper(InstantFunctionId.ClampMin) - val resultObs = instantVectorFnMapper3(Observable.fromIterable(sampleBase), queryConfig, 1000, resultSchema, Nil) + val resultObs = instantVectorFnMapper3(Observable.fromIterable(sampleBase), querySession, 1000, resultSchema, Nil) resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1)).toList) } should have message "requirement failed: Cannot use ClampMin without providing a lower limit of min." the[IllegalArgumentException] thrownBy { val instantVectorFnMapper5 = exec.InstantVectorFunctionMapper(InstantFunctionId.Sqrt, Seq(StaticFuncArgs(1, rangeParams))) - val resultObs = instantVectorFnMapper5(Observable.fromIterable(sampleBase), queryConfig, 1000, resultSchema, Nil) + val resultObs = instantVectorFnMapper5(Observable.fromIterable(sampleBase), querySession, 1000, resultSchema, Nil) resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1)).toList) } should have message "requirement failed: No additional parameters required for the instant function." the[IllegalArgumentException] thrownBy { val instantVectorFnMapper5 = exec.InstantVectorFunctionMapper(InstantFunctionId.Round, Seq(StaticFuncArgs(1, rangeParams), StaticFuncArgs(2, rangeParams))) - val resultObs = instantVectorFnMapper5(Observable.fromIterable(sampleBase), queryConfig, 1000, resultSchema, Nil) + val resultObs = instantVectorFnMapper5(Observable.fromIterable(sampleBase), querySession, 1000, resultSchema, Nil) resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1)).toList) } should have message "requirement failed: Only one optional parameters allowed for Round." @@ -189,7 +189,7 @@ class InstantFunctionSpec extends RawDataWindowingSpec with ScalaFutures { the[IllegalArgumentException] thrownBy { val (data, histRV) = histogramRV(numSamples = 10) val ivMapper = exec.InstantVectorFunctionMapper(InstantFunctionId.HistogramQuantile) - val resultObs = ivMapper(Observable.fromIterable(Array(histRV)), queryConfig, 1000, histSchema, Nil) + val resultObs = ivMapper(Observable.fromIterable(Array(histRV)), querySession, 1000, histSchema, Nil) resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1)).toList) } should have message "requirement failed: Quantile (between 0 and 1) required for histogram quantile" @@ -197,7 +197,7 @@ class InstantFunctionSpec extends RawDataWindowingSpec with ScalaFutures { the[IllegalArgumentException] thrownBy { val (data, histRV) = histogramRV(numSamples = 10) val ivMapper = exec.InstantVectorFunctionMapper(InstantFunctionId.HistogramBucket) - val resultObs = ivMapper(Observable.fromIterable(Array(histRV)), queryConfig, 1000, histSchema, Nil) + val resultObs = ivMapper(Observable.fromIterable(Array(histRV)), querySession, 1000, histSchema, Nil) resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1)).toList) } should have message "requirement failed: Bucket/le required for histogram bucket" } @@ -206,7 +206,7 @@ class InstantFunctionSpec extends RawDataWindowingSpec with ScalaFutures { // ceil val expectedVal = sampleBase.map(_.rows.map(v => scala.math.floor(v.getDouble(1)))) val instantVectorFnMapper = exec.InstantVectorFunctionMapper(InstantFunctionId.Ceil) - val resultObs = instantVectorFnMapper(Observable.fromIterable(sampleBase), queryConfig, 1000, resultSchema, Nil) + val resultObs = instantVectorFnMapper(Observable.fromIterable(sampleBase), querySession, 1000, resultSchema, Nil) val result = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) expectedVal.zip(result).foreach { case (ex, res) => { @@ -314,7 +314,7 @@ class InstantFunctionSpec extends RawDataWindowingSpec with ScalaFutures { instantFunctionId: InstantFunctionId, funcParams: Seq[Double] = Nil, schema: ResultSchema = resultSchema): Unit = { val instantVectorFnMapper = exec.InstantVectorFunctionMapper(instantFunctionId, funcParams.map(x => StaticFuncArgs(x, RangeParams(100,10,200)))) - val resultObs = instantVectorFnMapper(Observable.fromIterable(samples), queryConfig, 1000, schema, Nil) + val resultObs = instantVectorFnMapper(Observable.fromIterable(samples), querySession, 1000, schema, Nil) val result = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) expectedVal.zip(result).foreach { case (ex, res) => { diff --git a/query/src/test/scala/filodb/query/exec/rangefn/LabelReplaceSpec.scala b/query/src/test/scala/filodb/query/exec/rangefn/LabelReplaceSpec.scala index ed24592067..db0c662aad 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/LabelReplaceSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/LabelReplaceSpec.scala @@ -5,8 +5,9 @@ import monix.execution.Scheduler.Implicits.global import monix.reactive.Observable import org.scalatest.{FunSpec, Matchers} import org.scalatest.concurrent.ScalaFutures + import filodb.core.MetricsTestData -import filodb.core.query.{CustomRangeVectorKey, RangeVector, RangeVectorKey, ResultSchema, TransientRow} +import filodb.core.query.{CustomRangeVectorKey, QueryConfig, QueryContext, QuerySession, RangeVector, RangeVectorKey, ResultSchema, TransientRow} import filodb.memory.format.{RowReader, ZeroCopyUTF8String} import filodb.query._ @@ -42,6 +43,7 @@ class LabelReplaceSpec extends FunSpec with Matchers with ScalaFutures { }) val queryConfig = new QueryConfig(config.getConfig("query")) + val querySession = QuerySession(QueryContext(), queryConfig) it("should replace label only when match is found in label replace") { val sampleKey1 = CustomRangeVectorKey( @@ -72,7 +74,7 @@ class LabelReplaceSpec extends FunSpec with Matchers with ScalaFutures { val funcParams = Seq("instance", "$1 new Label Value $2", "instance", "(.*):90(.*)") val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, funcParams) - val resultObs = labelVectorFnMapper(Observable.fromIterable(sampleWithKey), queryConfig, 1000, resultSchema, Nil) + val resultObs = labelVectorFnMapper(Observable.fromIterable(sampleWithKey), querySession, 1000, resultSchema, Nil) val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) @@ -118,7 +120,7 @@ class LabelReplaceSpec extends FunSpec with Matchers with ScalaFutures { val funcParams = Seq("instanceNew", "$1-$1", "instance", "(.*)\\d") val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, funcParams) - val resultObs = labelVectorFnMapper(Observable.fromIterable(sampleWithKey), queryConfig, 1000, resultSchema, Nil) + val resultObs = labelVectorFnMapper(Observable.fromIterable(sampleWithKey), querySession, 1000, resultSchema, Nil) val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) @@ -159,7 +161,7 @@ class LabelReplaceSpec extends FunSpec with Matchers with ScalaFutures { val funcParams = Seq("instance", "$1", "instance", "(.*)9") val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, funcParams) - val resultObs = labelVectorFnMapper(Observable.fromIterable(sampleWithKey), queryConfig, 1000, resultSchema, Nil) + val resultObs = labelVectorFnMapper(Observable.fromIterable(sampleWithKey), querySession, 1000, resultSchema, Nil) val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) @@ -182,13 +184,13 @@ class LabelReplaceSpec extends FunSpec with Matchers with ScalaFutures { the[IllegalArgumentException] thrownBy { val miscellaneousFunctionMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, funcParams) - miscellaneousFunctionMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema, Nil) + miscellaneousFunctionMapper(Observable.fromIterable(testSample), querySession, 1000, resultSchema, Nil) } should have message "Invalid Regular Expression for label_replace" the[IllegalArgumentException] thrownBy { val miscellaneousFunctionMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, Seq("instance", "$1")) - miscellaneousFunctionMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema, Nil) + miscellaneousFunctionMapper(Observable.fromIterable(testSample), querySession, 1000, resultSchema, Nil) } should have message "requirement failed: " + "Cannot use LabelReplace without function parameters: " + "instant-vector, dst_label string, replacement string, src_label string, regex string" @@ -196,7 +198,7 @@ class LabelReplaceSpec extends FunSpec with Matchers with ScalaFutures { the[IllegalArgumentException] thrownBy { val miscellaneousFunctionMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, Seq("$instance", "$1", "instance", "(.*)9(")) - miscellaneousFunctionMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema, Nil) + miscellaneousFunctionMapper(Observable.fromIterable(testSample), querySession, 1000, resultSchema, Nil) } should have message "requirement failed: Invalid destination label name" } @@ -210,7 +212,7 @@ class LabelReplaceSpec extends FunSpec with Matchers with ScalaFutures { val funcParams = Seq("dst", "destination-value-$1", "src", "source-value-(.*)") val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, funcParams) - val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema, Nil) + val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), querySession, 1000, resultSchema, Nil) val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) @@ -236,7 +238,7 @@ class LabelReplaceSpec extends FunSpec with Matchers with ScalaFutures { val funcParams = Seq("dst", "destination-value-$1", "src", "value-(.*)") val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, funcParams) - val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema, Nil) + val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), querySession, 1000, resultSchema, Nil) val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) @@ -262,7 +264,7 @@ class LabelReplaceSpec extends FunSpec with Matchers with ScalaFutures { val funcParams = Seq("dst", "$1-value-$2 $3$67", "src", "(.*)-value-(.*)") val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, funcParams) - val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema, Nil) + val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), querySession, 1000, resultSchema, Nil) val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) @@ -288,7 +290,7 @@ class LabelReplaceSpec extends FunSpec with Matchers with ScalaFutures { val funcParams = Seq("dst", "value-$1", "nonexistent-src", "source-value-(.*)") val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, funcParams) - val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema, Nil) + val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), querySession, 1000, resultSchema, Nil) val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) @@ -314,7 +316,7 @@ class LabelReplaceSpec extends FunSpec with Matchers with ScalaFutures { val funcParams = Seq("dst", "value-$1", "nonexistent-src", ".*") val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, funcParams) - val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema, Nil) + val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), querySession, 1000, resultSchema, Nil) val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) @@ -340,7 +342,7 @@ class LabelReplaceSpec extends FunSpec with Matchers with ScalaFutures { val funcParams = Seq("dst", "value-$1", "src", "dummy-regex") val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, funcParams) - val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema, Nil) + val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), querySession, 1000, resultSchema, Nil) val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) @@ -364,7 +366,7 @@ class LabelReplaceSpec extends FunSpec with Matchers with ScalaFutures { val funcParams = Seq("dst", "", "dst", ".*") val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, funcParams) - val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema, Nil) + val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), querySession, 1000, resultSchema, Nil) val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) @@ -388,7 +390,7 @@ class LabelReplaceSpec extends FunSpec with Matchers with ScalaFutures { val funcParams = Seq("src", "", "", "") val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, funcParams) - val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema, Nil) + val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), querySession, 1000, resultSchema, Nil) val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) diff --git a/query/src/test/scala/filodb/query/exec/rangefn/LableJoinSpec.scala b/query/src/test/scala/filodb/query/exec/rangefn/LableJoinSpec.scala index e0254b1b4d..0429c7039c 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/LableJoinSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/LableJoinSpec.scala @@ -5,8 +5,9 @@ import monix.execution.Scheduler.Implicits.global import monix.reactive.Observable import org.scalatest.{FunSpec, Matchers} import org.scalatest.concurrent.ScalaFutures + import filodb.core.MetricsTestData -import filodb.core.query.{CustomRangeVectorKey, RangeVector, RangeVectorKey, ResultSchema, TransientRow} +import filodb.core.query._ import filodb.memory.format.{RowReader, ZeroCopyUTF8String} import filodb.query._ @@ -83,6 +84,7 @@ class LableJoinSpec extends FunSpec with Matchers with ScalaFutures { }) val queryConfig = new QueryConfig(config.getConfig("query")) + val querySession = QuerySession(QueryContext(), queryConfig) it("label_join joins all src values in order") { @@ -99,7 +101,7 @@ class LableJoinSpec extends FunSpec with Matchers with ScalaFutures { val funcParams = Seq("dst", "-", "src", "src1", "src2") val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelJoin, funcParams) - val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema, Nil) + val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), querySession, 1000, resultSchema, Nil) val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) @@ -130,7 +132,7 @@ class LableJoinSpec extends FunSpec with Matchers with ScalaFutures { val funcParams = Seq("dst", "-", "src", "src3", "src1") val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelJoin, funcParams) - val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema, Nil) + val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), querySession, 1000, resultSchema, Nil) val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) @@ -160,7 +162,7 @@ class LableJoinSpec extends FunSpec with Matchers with ScalaFutures { val funcParams = Seq("dst", "", "emptysrc", "emptysrc1", "emptysrc2") val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelJoin, funcParams) - val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema, Nil) + val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), querySession, 1000, resultSchema, Nil) val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) @@ -191,7 +193,7 @@ class LableJoinSpec extends FunSpec with Matchers with ScalaFutures { val funcParams = Seq("dst", "-", "src", "src1", "src2") val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelJoin, funcParams) - val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema, Nil) + val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), querySession, 1000, resultSchema, Nil) val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) @@ -221,7 +223,7 @@ class LableJoinSpec extends FunSpec with Matchers with ScalaFutures { val funcParams = Seq("dst", "-") val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelJoin, funcParams) - val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema, Nil) + val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), querySession, 1000, resultSchema, Nil) val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) @@ -246,19 +248,19 @@ class LableJoinSpec extends FunSpec with Matchers with ScalaFutures { the[IllegalArgumentException] thrownBy { val miscellaneousFunctionMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelJoin, funcParams1) - miscellaneousFunctionMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema, Nil) + miscellaneousFunctionMapper(Observable.fromIterable(testSample), querySession, 1000, resultSchema, Nil) } should have message "requirement failed: Invalid source label name in label_join()" the[IllegalArgumentException] thrownBy { val miscellaneousFunctionMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelJoin, funcParams2) - miscellaneousFunctionMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema, Nil) + miscellaneousFunctionMapper(Observable.fromIterable(testSample), querySession, 1000, resultSchema, Nil) } should have message "requirement failed: Invalid destination label name in label_join()" the[IllegalArgumentException] thrownBy { val miscellaneousFunctionMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelJoin, Seq("dst")) - miscellaneousFunctionMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema, Nil) + miscellaneousFunctionMapper(Observable.fromIterable(testSample), querySession, 1000, resultSchema, Nil) } should have message "requirement failed: expected at least 3 argument(s) in call to label_join" } } 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 abb100917f..c6efe1f78b 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/RateFunctionsSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/RateFunctionsSpec.scala @@ -64,7 +64,7 @@ class RateFunctionsSpec extends RawDataWindowingSpec { // One window, start=end=endTS val it = new ChunkedWindowIteratorD(counterRV, endTs, 10000, endTs, endTs - startTs, - new ChunkedRateFunction, queryConfig) + new ChunkedRateFunction, querySession) it.next.getDouble(1) shouldEqual expected +- errorOk } @@ -86,7 +86,7 @@ class RateFunctionsSpec extends RawDataWindowingSpec { // One window, start=end=endTS val it = new ChunkedWindowIteratorD(rv, endTs, 10000, endTs, endTs - startTs, - new ChunkedRateFunction, queryConfig) + new ChunkedRateFunction, querySession) it.next.getDouble(1) shouldEqual expected +- errorOk } @@ -123,13 +123,13 @@ class RateFunctionsSpec extends RawDataWindowingSpec { // One window, start=end=endTS val it = new ChunkedWindowIteratorD(rv, endTs, 10000, endTs, endTs - startTs, - new ChunkedRateFunction, queryConfig) + new ChunkedRateFunction, querySession) it.next.getDouble(1) shouldEqual expected +- errorOk // Two drops in one chunk val rv2 = timeValueRV(resetChunk1 ++ resetChunk2) val it2 = new ChunkedWindowIteratorD(rv2, endTs, 10000, endTs, endTs - startTs, - new ChunkedRateFunction, queryConfig) + new ChunkedRateFunction, querySession) it2.next.getDouble(1) shouldEqual expected +- errorOk } @@ -138,7 +138,7 @@ class RateFunctionsSpec extends RawDataWindowingSpec { val endTs = 8103215L val it = new ChunkedWindowIteratorD(counterRV, endTs, 10000, endTs, endTs - startTs, - new ChunkedRateFunction, queryConfig) + new ChunkedRateFunction, querySession) it.next.getDouble(1).isNaN shouldEqual true } @@ -150,7 +150,7 @@ class RateFunctionsSpec extends RawDataWindowingSpec { // One window, start=end=endTS val it = new ChunkedWindowIteratorD(flatRV, endTs, 10000, endTs, endTs - startTs, - new ChunkedRateFunction, queryConfig) + new ChunkedRateFunction, querySession) it.next.getDouble(1) shouldEqual 0.0 } @@ -206,7 +206,7 @@ class RateFunctionsSpec extends RawDataWindowingSpec { // One window, start=end=endTS val it = new ChunkedWindowIteratorH(rv, endTs, 100000, endTs, endTs - startTs, - new HistRateFunction, queryConfig) + new HistRateFunction, querySession) // Scheme should have remained the same val answer = it.next.getHistogram(1) answer.numBuckets shouldEqual expected.numBuckets @@ -245,7 +245,7 @@ class RateFunctionsSpec extends RawDataWindowingSpec { // One window, start=end=endTS val it = new ChunkedWindowIteratorH(rv, endTs, 110000, endTs, endTs - startTs, - new HistRateFunction, queryConfig) + new HistRateFunction, querySession) // Scheme should have remained the same val answer = it.next.getHistogram(1) answer.numBuckets shouldEqual expected.numBuckets @@ -374,12 +374,12 @@ class RateFunctionsSpec extends RawDataWindowingSpec { val endTs = 8163070L val expected = (q.last.value - q.head.value) / (q.last.timestamp - q.head.timestamp) * (endTs - startTs) val toEmit = new TransientRow - IncreaseFunction.apply(startTs,endTs, counterWindow, toEmit, queryConfig) + IncreaseFunction.apply(startTs, endTs, counterWindow, toEmit, queryConfig) toEmit.value shouldEqual expected +- errorOk // One window, start=end=endTS val it = new ChunkedWindowIteratorD(counterRV, endTs, 10000, endTs, endTs - startTs, - new ChunkedIncreaseFunction, queryConfig) + new ChunkedIncreaseFunction, querySession) it.next.getDouble(1) shouldEqual expected +- errorOk } @@ -388,13 +388,13 @@ class RateFunctionsSpec extends RawDataWindowingSpec { val endTs = 8163070L val expected = (q2.last.value - q2.head.value) / (q2.last.timestamp - q2.head.timestamp) * (endTs - startTs) val toEmit = new TransientRow - DeltaFunction.apply(startTs,endTs, gaugeWindow, toEmit, queryConfig) + DeltaFunction.apply(startTs, endTs, gaugeWindow, toEmit, queryConfig) toEmit.value shouldEqual expected +- errorOk // One window, start=end=endTS val gaugeRV = timeValueRV(gaugeSamples) val it = new ChunkedWindowIteratorD(gaugeRV, endTs, 10000, endTs, endTs - startTs, - new ChunkedDeltaFunction, queryConfig) + new ChunkedDeltaFunction, querySession) it.next.getDouble(1) shouldEqual expected +- errorOk } diff --git a/query/src/test/scala/filodb/query/exec/rangefn/ScalarFunctionSpec.scala b/query/src/test/scala/filodb/query/exec/rangefn/ScalarFunctionSpec.scala index 244f08f46a..aa62e4e599 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/ScalarFunctionSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/ScalarFunctionSpec.scala @@ -10,7 +10,7 @@ import filodb.core.query._ import filodb.core.store.{InMemoryMetaStore, NullColumnStore} import filodb.memory.format.{RowReader, ZeroCopyUTF8String} import filodb.query.exec.TimeScalarGeneratorExec -import filodb.query.{QueryConfig, QueryResult, ScalarFunctionId, exec} +import filodb.query.{QueryResult, ScalarFunctionId, exec} import monix.execution.Scheduler.Implicits.global import monix.reactive.Observable import org.scalatest.concurrent.ScalaFutures @@ -26,6 +26,7 @@ class ScalarFunctionSpec extends FunSpec with Matchers with ScalaFutures { val config: Config = ConfigFactory.load("application_test.conf").getConfig("filodb") val queryConfig = new QueryConfig(config.getConfig("query")) + val querySession = QuerySession(QueryContext(), queryConfig) val policy = new FixedMaxPartitionsEvictionPolicy(20) val memStore = new TimeSeriesMemStore(config, new NullColumnStore, new InMemoryMetaStore(), Some(policy)) val resultSchema = ResultSchema(MetricsTestData.timeseriesSchema.infosFromIDs(0 to 1), 1) @@ -100,7 +101,7 @@ class ScalarFunctionSpec extends FunSpec with Matchers with ScalaFutures { it("should generate scalar") { val scalarFunctionMapper = exec.ScalarFunctionMapper(ScalarFunctionId.Scalar, RangeParams(1,1,1)) - val resultObs = scalarFunctionMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema, Nil) + val resultObs = scalarFunctionMapper(Observable.fromIterable(testSample), querySession, 1000, resultSchema, Nil) val resultRangeVectors = resultObs.toListL.runAsync.futureValue resultRangeVectors.forall(x => x.isInstanceOf[ScalarFixedDouble]) shouldEqual (true) val resultRows = resultRangeVectors.flatMap(_.rows.map(_.getDouble(1)).toList) @@ -110,7 +111,7 @@ class ScalarFunctionSpec extends FunSpec with Matchers with ScalaFutures { it("should generate scalar values when there is one range vector") { val scalarFunctionMapper = exec.ScalarFunctionMapper(ScalarFunctionId.Scalar, RangeParams(1,1,1)) - val resultObs = scalarFunctionMapper(Observable.fromIterable(oneSample), queryConfig, 1000, resultSchema, Nil) + val resultObs = scalarFunctionMapper(Observable.fromIterable(oneSample), querySession, 1000, resultSchema, Nil) val resultRangeVectors = resultObs.toListL.runAsync.futureValue resultRangeVectors.forall(x => x.isInstanceOf[ScalarVaryingDouble]) shouldEqual (true) val resultRows = resultRangeVectors.flatMap(_.rows.map(_.getDouble(1)).toList) @@ -121,7 +122,7 @@ class ScalarFunctionSpec extends FunSpec with Matchers with ScalaFutures { val execPlan = TimeScalarGeneratorExec(QueryContext(), timeseriesDataset.ref, RangeParams(10, 10, 100), ScalarFunctionId.Time) implicit val timeout: FiniteDuration = FiniteDuration(5, TimeUnit.SECONDS) import monix.execution.Scheduler.Implicits.global - val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue + val resp = execPlan.execute(memStore, querySession).runAsync.futureValue val result = resp match { case QueryResult(id, _, response) => { val rv = response(0) @@ -136,7 +137,7 @@ class ScalarFunctionSpec extends FunSpec with Matchers with ScalaFutures { val execPlan = TimeScalarGeneratorExec(QueryContext(), timeseriesDataset.ref, RangeParams(1565627710, 10, 1565627790), ScalarFunctionId.Hour) implicit val timeout: FiniteDuration = FiniteDuration(5, TimeUnit.SECONDS) import monix.execution.Scheduler.Implicits.global - val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue + val resp = execPlan.execute(memStore, querySession).runAsync.futureValue val result = resp match { case QueryResult(id, _, response) => { val rv = response(0) @@ -152,7 +153,7 @@ class ScalarFunctionSpec extends FunSpec with Matchers with ScalaFutures { val execPlan = TimeScalarGeneratorExec(QueryContext(), timeseriesDataset.ref, RangeParams(1583682900, 100, 1583683400), ScalarFunctionId.DayOfWeek) implicit val timeout: FiniteDuration = FiniteDuration(5, TimeUnit.SECONDS) import monix.execution.Scheduler.Implicits.global - val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue + val resp = execPlan.execute(memStore, querySession).runAsync.futureValue val result = resp match { case QueryResult(id, _, response) => { val rv = response(0) diff --git a/query/src/test/scala/filodb/query/exec/rangefn/SortFunctionSpec.scala b/query/src/test/scala/filodb/query/exec/rangefn/SortFunctionSpec.scala index ec5d9eeb86..660d04707b 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/SortFunctionSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/SortFunctionSpec.scala @@ -4,11 +4,11 @@ import com.typesafe.config.{Config, ConfigFactory} import filodb.core.MetricsTestData import filodb.core.metadata.Column.ColumnType -import filodb.core.query.{ColumnInfo, CustomRangeVectorKey, RangeVector, RangeVectorKey, ResultSchema, TransientRow} +import filodb.core.query._ import filodb.memory.format.{RowReader, ZeroCopyUTF8String} import filodb.query.exec.RangeVectorAggregator import filodb.query.exec.aggregator.RowAggregator -import filodb.query.{exec, AggregationOperator, QueryConfig, SortFunctionId} +import filodb.query.{exec, AggregationOperator, SortFunctionId} import monix.execution.Scheduler.Implicits.global import monix.reactive.Observable @@ -19,6 +19,7 @@ class SortFunctionSpec extends FunSpec with Matchers with ScalaFutures { val config: Config = ConfigFactory.load("application_test.conf").getConfig("filodb") val resultSchema = ResultSchema(MetricsTestData.timeseriesSchema.infosFromIDs(0 to 1), 1) val queryConfig = new QueryConfig(config.getConfig("query")) + val querySession = QuerySession(QueryContext(), queryConfig) val ignoreKey = CustomRangeVectorKey( Map(ZeroCopyUTF8String("ignore") -> ZeroCopyUTF8String("ignore"))) @@ -85,21 +86,21 @@ class SortFunctionSpec extends FunSpec with Matchers with ScalaFutures { it("should sort instant vectors in ascending order") { val sortFunctionMapper = exec.SortFunctionMapper(SortFunctionId.Sort) - val resultObs = sortFunctionMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema, Nil) + val resultObs = sortFunctionMapper(Observable.fromIterable(testSample), querySession, 1000, resultSchema, Nil) val resultRows = resultObs.toListL.runAsync.futureValue.flatMap(_.rows.map(_.getDouble(1)).toList) resultRows.shouldEqual(List(0.0, 1.0, 2.0, 3.0, 4.0, 5.0, 6.0)) } it("should sort instant vectors in descending order") { val sortFunctionMapper = exec.SortFunctionMapper(SortFunctionId.SortDesc) - val resultObs = sortFunctionMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema, Nil) + val resultObs = sortFunctionMapper(Observable.fromIterable(testSample), querySession, 1000, resultSchema, Nil) val resultRows = resultObs.toListL.runAsync.futureValue.flatMap(_.rows.map(_.getDouble(1)).toList) resultRows.shouldEqual(List(6.0, 5.0, 4.0, 3.0, 2.0, 1.0, 0.0)) } it("should return empty rangeVector when sorting empty sample") { val sortFunctionMapper = exec.SortFunctionMapper(SortFunctionId.Sort) - val resultObs = sortFunctionMapper(Observable.fromIterable(emptySample), queryConfig, 1000, resultSchema, Nil) + val resultObs = sortFunctionMapper(Observable.fromIterable(emptySample), querySession, 1000, resultSchema, Nil) val resultRows = resultObs.toListL.runAsync.futureValue.flatMap(_.rows.map(_.getDouble(1)).toList) resultRows.isEmpty shouldEqual(true) } @@ -138,7 +139,7 @@ class SortFunctionSpec extends FunSpec with Matchers with ScalaFutures { resultAgg.flatMap(_.rows.map(_.getDouble(1)).toList) shouldEqual(List(5.0, 1.0)) val sortFunctionMapper = exec.SortFunctionMapper(SortFunctionId.Sort) - val resultObs = sortFunctionMapper(resultObs2, queryConfig, 1000, resultSchema, Nil) + val resultObs = sortFunctionMapper(resultObs2, querySession, 1000, resultSchema, Nil) val resultRows = resultObs.toListL.runAsync.futureValue.flatMap(_.rows.map(_.getDouble(1)).toList) resultRows.shouldEqual(List(1.0, 5.0)) } diff --git a/spark-jobs/src/test/scala/filodb/downsampler/DownsamplerMainSpec.scala b/spark-jobs/src/test/scala/filodb/downsampler/DownsamplerMainSpec.scala index eb81798f80..61a44936a5 100644 --- a/spark-jobs/src/test/scala/filodb/downsampler/DownsamplerMainSpec.scala +++ b/spark-jobs/src/test/scala/filodb/downsampler/DownsamplerMainSpec.scala @@ -22,7 +22,7 @@ import filodb.core.downsample.DownsampledTimeSeriesStore import filodb.core.memstore.{PagedReadablePartition, TimeSeriesPartition} import filodb.core.memstore.FiloSchedulers.QuerySchedName import filodb.core.metadata.{Dataset, Schemas} -import filodb.core.query.{ColumnFilter, CustomRangeVectorKey, QueryContext, RawDataRangeVector} +import filodb.core.query._ import filodb.core.query.Filter.Equals import filodb.core.store.{AllChunkScan, PartKeyRecord, SinglePartitionScan, StoreConfig} import filodb.downsampler.chunk.{BatchDownsampler, Downsampler, DownsamplerSettings, OffHeapMemory} @@ -30,7 +30,7 @@ import filodb.downsampler.index.{DSIndexJobSettings, IndexJobDriver} import filodb.memory.format.{PrimitiveVectorReader, UnsafeUtils} import filodb.memory.format.ZeroCopyUTF8String._ import filodb.memory.format.vectors.{CustomBuckets, LongHistogram} -import filodb.query.{QueryConfig, QueryResult} +import filodb.query.QueryResult import filodb.query.exec.{InProcessPlanDispatcher, MultiSchemaPartitionsExec} /** @@ -44,6 +44,7 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w val conf = ConfigFactory.parseFile(new File("conf/timeseries-filodb-server.conf")) val settings = new DownsamplerSettings(conf) + val queryConfig = new QueryConfig(settings.filodbConfig.getConfig("query")) val dsIndexJobSettings = new DSIndexJobSettings(settings) val batchDownsampler = new BatchDownsampler(settings) @@ -652,9 +653,9 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w val exec = MultiSchemaPartitionsExec(QueryContext(sampleLimit = 1000), InProcessPlanDispatcher, batchDownsampler.rawDatasetRef, 0, queryFilters, AllChunkScan) - val queryConfig = new QueryConfig(settings.filodbConfig.getConfig("query")) + val querySession = QuerySession(QueryContext(), queryConfig) val queryScheduler = Scheduler.fixedPool(s"$QuerySchedName", 3) - val res = exec.execute(downsampleTSStore, queryConfig)(queryScheduler) + val res = exec.execute(downsampleTSStore, querySession)(queryScheduler) .runAsync(queryScheduler).futureValue.asInstanceOf[QueryResult] queryScheduler.shutdown() @@ -680,9 +681,9 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w val exec = MultiSchemaPartitionsExec(QueryContext(sampleLimit = 1000), InProcessPlanDispatcher, batchDownsampler.rawDatasetRef, 0, queryFilters, AllChunkScan) - val queryConfig = new QueryConfig(settings.filodbConfig.getConfig("query")) + val querySession = QuerySession(QueryContext(), queryConfig) val queryScheduler = Scheduler.fixedPool(s"$QuerySchedName", 3) - val res = exec.execute(downsampleTSStore, queryConfig)(queryScheduler) + val res = exec.execute(downsampleTSStore, querySession)(queryScheduler) .runAsync(queryScheduler).futureValue.asInstanceOf[QueryResult] queryScheduler.shutdown() @@ -702,9 +703,9 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w val exec = MultiSchemaPartitionsExec(QueryContext(sampleLimit = 1000), InProcessPlanDispatcher, batchDownsampler.rawDatasetRef, 0, queryFilters, AllChunkScan, colName = Option("sum")) - val queryConfig = new QueryConfig(settings.filodbConfig.getConfig("query")) + val querySession = QuerySession(QueryContext(), queryConfig) val queryScheduler = Scheduler.fixedPool(s"$QuerySchedName", 3) - val res = exec.execute(downsampleTSStore, queryConfig)(queryScheduler) + val res = exec.execute(downsampleTSStore, querySession)(queryScheduler) .runAsync(queryScheduler).futureValue.asInstanceOf[QueryResult] queryScheduler.shutdown() res.result.size shouldEqual 1 From 5f5a8a63648f9fc30ae43b55550ddd44759836b5 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Fri, 8 May 2020 09:55:50 -0700 Subject: [PATCH 03/36] bug(core): Reclaim shared lock for pinning blocks during queries (#744) Preparing to guard block memory reclaims with shard level locks. This PR just introduces shared/read locks. Subsequent PRs will introduce write locks. --- .../memstore/TimeSeriesShard.scala | 99 +++++++++++-------- .../filodb.core/query/QueryContext.scala | 12 ++- .../exec/MultiSchemaPartitionsExec.scala | 14 +++ 3 files changed, 81 insertions(+), 44 deletions(-) diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala index 6d21b651cf..137430983d 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala @@ -306,6 +306,12 @@ class TimeSeriesShard(val ref: DatasetRef, // occurs in the common case, when there isn't any contention reading from partSet. private[memstore] final val partSetLock = new StampedLock + /** + * Lock that protects chunks from being reclaimed from Memstore. + * This is needed to prevent races between ODP queries and reclaims. + */ + private[memstore] final val reclaimLock = new StampedLock + // The off-heap block store used for encoded chunks private val shardTags = Map("dataset" -> ref.dataset, "shard" -> shardNum.toString) private val blockStore = new PageAlignedBlockManager(blockMemorySize, shardStats.memoryStats, reclaimListener, @@ -1406,49 +1412,56 @@ class TimeSeriesShard(val ref: DatasetRef, */ def lookupPartitions(partMethod: PartitionScanMethod, chunkMethod: ChunkScanMethod, - querySession: QuerySession): PartLookupResult = partMethod match { - case SinglePartitionScan(partition, _) => - val partIds = debox.Buffer.empty[Int] - getPartition(partition).foreach(p => partIds += p.partID) - PartLookupResult(shardNum, chunkMethod, partIds, Some(RecordSchema.schemaID(partition))) - case MultiPartitionScan(partKeys, _) => - val partIds = debox.Buffer.empty[Int] - partKeys.flatMap(getPartition).foreach(p => partIds += p.partID) - PartLookupResult(shardNum, chunkMethod, partIds, partKeys.headOption.map(RecordSchema.schemaID)) - case FilteredPartitionScan(split, filters) => - // No matter if there are filters or not, need to run things through Lucene so we can discover potential - // TSPartitions to read back from disk - val matches = partKeyIndex.partIdsFromFilters(filters, chunkMethod.startTime, chunkMethod.endTime) - shardStats.queryTimeRangeMins.record((chunkMethod.endTime - chunkMethod.startTime) / 60000 ) - - Kamon.currentSpan().tag(s"num-partitions-from-index-$shardNum", matches.length) - if (matches.length > storeConfig.maxQueryMatches) - throw new IllegalArgumentException(s"Seeing ${matches.length} matching time series per shard. Try " + - s"to narrow your query by adding more filters so there is less than ${storeConfig.maxQueryMatches} matches " + - s"or request that number of shards for the metric be increased") - - // first find out which partitions are being queried for data not in memory - val firstPartId = if (matches.isEmpty) None else Some(matches(0)) - val _schema = firstPartId.map(schemaIDFromPartID) - val it1 = InMemPartitionIterator2(matches) - val partIdsToPage = it1.filter(_.earliestTime > chunkMethod.startTime).map(_.partID) - val partIdsNotInMem = it1.skippedPartIDs - Kamon.currentSpan().tag(s"num-partitions-not-in-memory-$shardNum", partIdsNotInMem.length) - val startTimes = if (partIdsToPage.nonEmpty) { - val st = partKeyIndex.startTimeFromPartIds(partIdsToPage) - logger.debug(s"Some partitions have earliestTime > queryStartTime(${chunkMethod.startTime}); " + - s"startTime lookup for query in dataset=$ref shard=$shardNum " + - s"resulted in startTimes=$st") - st - } - else { - logger.debug(s"StartTime lookup was not needed. All partition's data for query in dataset=$ref " + - s"shard=$shardNum are in memory") - debox.Map.empty[Int, Long] - } - // now provide an iterator that additionally supplies the startTimes for - // those partitions that may need to be paged - PartLookupResult(shardNum, chunkMethod, matches, _schema, startTimes, partIdsNotInMem) + querySession: QuerySession): PartLookupResult = { + val reclaimReadLock = reclaimLock.asReadLock() + querySession.lock = Some(reclaimReadLock) + reclaimReadLock.lock() + // any exceptions thrown here should be caught by a wrapped Task. + // At the end, MultiSchemaPartitionsExec.execute releases the lock when the task is complete + partMethod match { + case SinglePartitionScan(partition, _) => + val partIds = debox.Buffer.empty[Int] + getPartition(partition).foreach(p => partIds += p.partID) + PartLookupResult(shardNum, chunkMethod, partIds, Some(RecordSchema.schemaID(partition))) + case MultiPartitionScan(partKeys, _) => + val partIds = debox.Buffer.empty[Int] + partKeys.flatMap(getPartition).foreach(p => partIds += p.partID) + PartLookupResult(shardNum, chunkMethod, partIds, partKeys.headOption.map(RecordSchema.schemaID)) + case FilteredPartitionScan(split, filters) => + // No matter if there are filters or not, need to run things through Lucene so we can discover potential + // TSPartitions to read back from disk + val matches = partKeyIndex.partIdsFromFilters(filters, chunkMethod.startTime, chunkMethod.endTime) + shardStats.queryTimeRangeMins.record((chunkMethod.endTime - chunkMethod.startTime) / 60000 ) + + Kamon.currentSpan().tag(s"num-partitions-from-index-$shardNum", matches.length) + if (matches.length > storeConfig.maxQueryMatches) + throw new IllegalArgumentException(s"Seeing ${matches.length} matching time series per shard. Try " + + s"to narrow your query by adding more filters so there is less than " + + s"${storeConfig.maxQueryMatches} matches or request that number of shards for the metric be increased") + + // first find out which partitions are being queried for data not in memory + val firstPartId = if (matches.isEmpty) None else Some(matches(0)) + val _schema = firstPartId.map(schemaIDFromPartID) + val it1 = InMemPartitionIterator2(matches) + val partIdsToPage = it1.filter(_.earliestTime > chunkMethod.startTime).map(_.partID) + val partIdsNotInMem = it1.skippedPartIDs + Kamon.currentSpan().tag(s"num-partitions-not-in-memory-$shardNum", partIdsNotInMem.length) + val startTimes = if (partIdsToPage.nonEmpty) { + val st = partKeyIndex.startTimeFromPartIds(partIdsToPage) + logger.debug(s"Some partitions have earliestTime > queryStartTime(${chunkMethod.startTime}); " + + s"startTime lookup for query in dataset=$ref shard=$shardNum " + + s"resulted in startTimes=$st") + st + } + else { + logger.debug(s"StartTime lookup was not needed. All partition's data for query in dataset=$ref " + + s"shard=$shardNum are in memory") + debox.Map.empty[Int, Long] + } + // now provide an iterator that additionally supplies the startTimes for + // those partitions that may need to be paged + PartLookupResult(shardNum, chunkMethod, matches, _schema, startTimes, partIdsNotInMem) + } } def scanPartitions(iterResult: PartLookupResult, diff --git a/core/src/main/scala/filodb.core/query/QueryContext.scala b/core/src/main/scala/filodb.core/query/QueryContext.scala index 8b3e1247b8..e0a0192e4a 100644 --- a/core/src/main/scala/filodb.core/query/QueryContext.scala +++ b/core/src/main/scala/filodb.core/query/QueryContext.scala @@ -1,6 +1,7 @@ package filodb.core.query import java.util.UUID +import java.util.concurrent.locks.Lock import com.typesafe.config.Config @@ -58,8 +59,17 @@ object QueryContext { } } +/** + * Placeholder for query related information. Typically passed along query execution path. + */ case class QuerySession(qContext: QueryContext, - queryConfig: QueryConfig) + queryConfig: QueryConfig, + var lock: Option[Lock] = None) { + def close(): Unit = { + lock.foreach(_.unlock()) + lock = None + } +} object QuerySession { def forTestingOnly: QuerySession = QuerySession(QueryContext(), EmptyQueryConfig) diff --git a/query/src/main/scala/filodb/query/exec/MultiSchemaPartitionsExec.scala b/query/src/main/scala/filodb/query/exec/MultiSchemaPartitionsExec.scala index a612ba5c10..c7f6ecade3 100644 --- a/query/src/main/scala/filodb/query/exec/MultiSchemaPartitionsExec.scala +++ b/query/src/main/scala/filodb/query/exec/MultiSchemaPartitionsExec.scala @@ -1,6 +1,7 @@ package filodb.query.exec import kamon.Kamon +import monix.eval.Task import monix.execution.Scheduler import filodb.core.{DatasetRef, QueryTimeoutException} @@ -8,6 +9,7 @@ import filodb.core.metadata.Schemas import filodb.core.query.{ColumnFilter, QueryContext, QuerySession} import filodb.core.store._ import filodb.query.Query.qLogger +import filodb.query.QueryResponse final case class UnknownSchemaQueryErr(id: Int) extends Exception(s"Unknown schema ID $id during query. This likely means a schema config change happened and " + @@ -82,6 +84,18 @@ final case class MultiSchemaPartitionsExec(queryContext: QueryContext, } } + /** + * Overridden to close the session and release locks after query executes + */ + override def execute(source: ChunkSource, + querySession: QuerySession) + (implicit sched: Scheduler): Task[QueryResponse] = { + val child = super.execute(source, querySession) + // doOnFinish handles the success and exception case. It does not handle the canceled case + child.doOnCancel(Task.now(querySession.close())) + child.doOnFinish(_ => Task.now(querySession.close())) + } + def doExecute(source: ChunkSource, querySession: QuerySession) (implicit sched: Scheduler): ExecResult = { From bb2895accd60d3106bb83ac0aea240ea9cffa4f0 Mon Sep 17 00:00:00 2001 From: "Brian S. O'Neill" Date: Fri, 8 May 2020 10:20:07 -0700 Subject: [PATCH 04/36] feat(memory): Add method to ensure free space. (#745) --- .../scala/filodb.memory/BlockManager.scala | 38 +++++++++++++++++++ .../PageAlignedBlockManagerSpec.scala | 34 +++++++++++++++++ 2 files changed, 72 insertions(+) diff --git a/memory/src/main/scala/filodb.memory/BlockManager.scala b/memory/src/main/scala/filodb.memory/BlockManager.scala index 548ce2fc59..94cf4c84ab 100644 --- a/memory/src/main/scala/filodb.memory/BlockManager.scala +++ b/memory/src/main/scala/filodb.memory/BlockManager.scala @@ -45,6 +45,34 @@ trait BlockManager { */ def requestBlock(bucketTime: Option[Long], owner: Option[BlockMemFactory] = None): Option[Block] + /** + * Attempts to reclaim as many blocks as necessary to ensure that enough free blocks are + * available. + */ + def ensureFreeBlocks(num: Int): Unit + + /** + * Attempts to reclaim as many blocks as necessary to ensure that enough free bytes are + * available. The actual amount reclaimed might be higher than requested. + */ + def ensureFreeBytes(amt: Long): Unit = { + val blocks = (amt + blockSizeInBytes - 1) / blockSizeInBytes + ensureFreeBlocks(Math.min(Integer.MAX_VALUE, blocks).toInt) + } + + /** + * Attempts to reclaim as many blocks as necessary to ensure that enough free bytes are + * available as a percentage of total size. The actual amount reclaimed might be higher than + * requested. + * + * @param pct percentage: 0.0 to 100.0 + */ + def ensureFreePercent(pct: Double): Unit = { + ensureFreeBytes((totalMemorySizeInBytes * pct * 0.01).toLong) + } + + def totalMemorySizeInBytes: Long + /** * Releases all blocks allocated by this store. */ @@ -165,6 +193,16 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, } } + override def ensureFreeBlocks(num: Int): Unit = { + lock.lock() + try { + val require = num - freeBlocks.size + if (require > 0) tryReclaim(require) + } finally { + lock.unlock() + } + } + protected def allocate(): util.LinkedList[Block] = { val numBlocks: Int = Math.floor(totalMemorySizeInBytes / blockSizeInBytes).toInt val blocks = new util.LinkedList[Block]() diff --git a/memory/src/test/scala/filodb.memory/PageAlignedBlockManagerSpec.scala b/memory/src/test/scala/filodb.memory/PageAlignedBlockManagerSpec.scala index 4e62faa440..c15c1a4ab3 100644 --- a/memory/src/test/scala/filodb.memory/PageAlignedBlockManagerSpec.scala +++ b/memory/src/test/scala/filodb.memory/PageAlignedBlockManagerSpec.scala @@ -204,4 +204,38 @@ class PageAlignedBlockManagerSpec extends FlatSpec with Matchers with BeforeAndA factory.currentBlock.owner shouldEqual None // new requestor did not have owner } + + it should "ensure free space" in { + val stats = new MemoryStats(Map("test5" -> "test5")) + // This block manager has 5 blocks capacity + val blockManager = new PageAlignedBlockManager(5 * pageSize, stats, testReclaimer, 1) + + blockManager.numFreeBlocks shouldEqual 5 + blockManager.ensureFreePercent(50) + blockManager.numFreeBlocks shouldEqual 5 + + blockManager.requestBlock(None).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.numFreeBlocks shouldEqual 3 + blockManager.ensureFreePercent(50) + blockManager.numFreeBlocks shouldEqual 3 + + blockManager.requestBlock(None).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.numFreeBlocks shouldEqual 2 + blockManager.ensureFreePercent(90) + // Should reclaim multiple blocks. + blockManager.numFreeBlocks shouldEqual 5 + + blockManager.releaseBlocks() + } } From 64fe20df029af85494b667dc0d6a7e4afab53d72 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Fri, 8 May 2020 11:39:22 -0700 Subject: [PATCH 05/36] bug(queryPlanning): Long Lookback cases in downsample queries (#743) Since raw data retention is finite (say 3 days), long lookback queries ( > 3d) cannot be handled by raw cluster. They end up being delegated to downsample cluster. However, there is a delay in population of downsample data. Most recent data would not be available immediately. Hence window aggregations for latest time instants may not be complete. This PR omits instants with incomplete window aggregations from query results. We will consider performant window aggregations across raw/downsample clusters for the future. The thinking here is that WoW and MoM comparisons can wait for 6-12 hours. Instant queries can apply offset of 6 or 12 hours depending on delay. --- .../queryplanner/CompositePlanner.scala | 3 +- .../queryplanner/LongTimeRangePlanner.scala | 20 +++++++- .../LongTimeRangePlannerSpec.scala | 49 ++++++++++++++++--- 3 files changed, 63 insertions(+), 9 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/CompositePlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/CompositePlanner.scala index 1ca64fe039..78850b68a5 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/CompositePlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/CompositePlanner.scala @@ -21,6 +21,7 @@ class CompositePlanner(dsRef: DatasetRef, failureProvider: FailureProvider, earliestRawTimestampFn: => Long, earliestDownsampleTimestampFn: => Long, + latestDownsampleTimestampFn: => Long, queryConfig: QueryConfig, spreadProvider: SpreadProvider = StaticSpreadProvider(), stitchDispatcher: => PlanDispatcher = { InProcessPlanDispatcher }) extends QueryPlanner @@ -31,7 +32,7 @@ class CompositePlanner(dsRef: DatasetRef, val downsampleClusterPlanner = new SingleClusterPlanner(dsRef, schemas, downsampleMapperFunc, earliestDownsampleTimestampFn, queryConfig, spreadProvider) val longTimeRangePlanner = new LongTimeRangePlanner(rawClusterPlanner, downsampleClusterPlanner, - earliestRawTimestampFn, stitchDispatcher) + earliestRawTimestampFn, latestDownsampleTimestampFn, stitchDispatcher) val haPlanner = new HighAvailabilityPlanner(dsRef, longTimeRangePlanner, failureProvider, queryConfig) //val multiPodPlanner = new MultiClusterPlanner(podLocalityProvider, haPlanner) diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LongTimeRangePlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LongTimeRangePlanner.scala index 1795fbd3b3..9e389d621c 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LongTimeRangePlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LongTimeRangePlanner.scala @@ -16,11 +16,18 @@ import filodb.query.exec.{ExecPlan, PlanDispatcher, StitchRvsExec} * abstracts planning for downsample cluster data * @param earliestRawTimestampFn the function that will provide millis timestamp of earliest sample that * would be available in the raw cluster + * @param latestDownsampleTimestampFn the function that will provide millis timestamp of newest sample + * that would be available in the downsample cluster. This typically + * is not "now" because of the delay in population of downsampled data + * via spark job. If job is run every 6 hours, + * `(System.currentTimeMillis - 12.hours.toMillis)` + * may a function that could be passed. 12 hours to account for failures/reruns etc. * @param stitchDispatcher function to get the dispatcher for the stitch exec plan node */ class LongTimeRangePlanner(rawClusterPlanner: QueryPlanner, downsampleClusterPlanner: QueryPlanner, earliestRawTimestampFn: => Long, + latestDownsampleTimestampFn: => Long, stitchDispatcher: => PlanDispatcher) extends QueryPlanner { def materialize(logicalPlan: LogicalPlan, qContext: QueryContext): ExecPlan = { @@ -35,10 +42,19 @@ class LongTimeRangePlanner(rawClusterPlanner: QueryPlanner, else if (endWithOffsetMs < earliestRawTime) downsampleClusterPlanner.materialize(logicalPlan, qContext) else if (startWithOffsetMs - lookbackMs >= earliestRawTime) rawClusterPlanner.materialize(logicalPlan, qContext) - else { + else if (endWithOffsetMs - lookbackMs < earliestRawTime) { + val lastDownsampleSampleTime = latestDownsampleTimestampFn + val downsampleLp = if (endWithOffsetMs < lastDownsampleSampleTime) { + logicalPlan + } else { + copyWithUpdatedTimeRange(logicalPlan, + TimeRange(p.startMs, latestDownsampleTimestampFn + offsetMillis), lookbackMs) + } + downsampleClusterPlanner.materialize(downsampleLp, qContext) + } else { // Split the query between raw and downsample planners val numStepsInDownsample = (earliestRawTime - startWithOffsetMs + lookbackMs) / p.stepMs - val lastDownsampleInstant = startWithOffsetMs + numStepsInDownsample * p.stepMs + val lastDownsampleInstant = p.startMs + numStepsInDownsample * p.stepMs val firstInstantInRaw = lastDownsampleInstant + p.stepMs val downsampleLp = copyWithUpdatedTimeRange(logicalPlan, diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/LongTimeRangePlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/LongTimeRangePlannerSpec.scala index 888425f7f5..955c541460 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/LongTimeRangePlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/LongTimeRangePlannerSpec.scala @@ -10,7 +10,7 @@ import filodb.core.query.{QueryContext, QuerySession} import filodb.core.store.ChunkSource import filodb.prometheus.ast.TimeStepParams import filodb.prometheus.parse.Parser -import filodb.query.{LogicalPlan, PeriodicSeries, PeriodicSeriesPlan} +import filodb.query.{LogicalPlan, PeriodicSeriesPlan, PeriodicSeriesWithWindowing} import filodb.query.exec._ class LongTimeRangePlannerSpec extends FunSpec with Matchers { @@ -41,9 +41,11 @@ class LongTimeRangePlannerSpec extends FunSpec with Matchers { val rawRetention = 10.minutes val now = System.currentTimeMillis() / 1000 * 1000 val earliestRawTime = now - rawRetention.toMillis + val latestDownsampleTime = now - 4.minutes.toMillis // say it takes 4 minutes to downsample private def disp = InProcessPlanDispatcher - val longTermPlanner = new LongTimeRangePlanner(rawPlanner, downsamplePlanner, earliestRawTime, disp) + val longTermPlanner = new LongTimeRangePlanner(rawPlanner, downsamplePlanner, + earliestRawTime, latestDownsampleTime, disp) it("should direct raw-cluster-only queries to raw planner") { val logicalPlan = Parser.queryRangeToLogicalPlan("rate(foo[2m])", @@ -94,7 +96,42 @@ class LongTimeRangePlannerSpec extends FunSpec with Matchers { downsampleLp.startMs shouldEqual logicalPlan.startMs downsampleLp.endMs shouldEqual rawStart - 1.minute.toMillis + } + + it("should delegate to downsample cluster and omit recent instants when there is a long lookback") { + + val start = now/1000 - 30.minutes.toSeconds + val step = 1.minute.toSeconds + val end = now/1000 + // notice raw data retention is 10m but lookback is 20m + val logicalPlan = Parser.queryRangeToLogicalPlan("rate(foo[20m])", + TimeStepParams(start, step, end)) + .asInstanceOf[PeriodicSeriesPlan] + + val ep = longTermPlanner.materialize(logicalPlan, QueryContext()) + val downsampleLp = ep.asInstanceOf[MockExecPlan] + downsampleLp.lp.asInstanceOf[PeriodicSeriesPlan].startMs shouldEqual logicalPlan.startMs + downsampleLp.lp.asInstanceOf[PeriodicSeriesPlan].endMs shouldEqual latestDownsampleTime + } + + it("should delegate to downsample cluster and retain endTime when there is a long lookback with offset that causes " + + "recent data to not be used") { + + val start = now/1000 - 30.minutes.toSeconds + val step = 1.minute.toSeconds + val end = now/1000 + // notice raw data retention is 10m but lookback is 20m + val logicalPlan = Parser.queryRangeToLogicalPlan("rate(foo[20m] offset 5m)", + TimeStepParams(start, step, end)) + .asInstanceOf[PeriodicSeriesPlan] + + val ep = longTermPlanner.materialize(logicalPlan, QueryContext()) + val downsampleLp = ep.asInstanceOf[MockExecPlan] + downsampleLp.lp.asInstanceOf[PeriodicSeriesPlan].startMs shouldEqual logicalPlan.startMs + // endTime is retained even with long lookback because 5m offset compensates + // for 4m delay in downsample data population + downsampleLp.lp.asInstanceOf[PeriodicSeriesPlan].endMs shouldEqual logicalPlan.endMs } @@ -122,7 +159,7 @@ class LongTimeRangePlannerSpec extends FunSpec with Matchers { val start = now/1000 - 30.minutes.toSeconds val step = 1.minute.toSeconds val end = now/1000 - 2.minutes.toSeconds - val logicalPlan = Parser.queryRangeToLogicalPlan("foo offset 5m", + val logicalPlan = Parser.queryRangeToLogicalPlan("rate(foo[5m] offset 2m)", TimeStepParams(start, step, end)) .asInstanceOf[PeriodicSeriesPlan] @@ -140,15 +177,15 @@ class LongTimeRangePlannerSpec extends FunSpec with Matchers { // find first instant with range available within raw data val rawStart = ((start*1000) to (end*1000) by (step*1000)).find { instant => - instant - 5.minutes.toMillis > earliestRawTime // subtract offset + instant - (5 + 2).minutes.toMillis > earliestRawTime // subtract lookback & offset }.get rawLp.startMs shouldEqual rawStart rawLp.endMs shouldEqual logicalPlan.endMs - rawLp.asInstanceOf[PeriodicSeries].offsetMs.get shouldEqual(300000) + rawLp.asInstanceOf[PeriodicSeriesWithWindowing].offsetMs.get shouldEqual(120000) downsampleLp.startMs shouldEqual logicalPlan.startMs downsampleLp.endMs shouldEqual rawStart - (step * 1000) - downsampleLp.asInstanceOf[PeriodicSeries].offsetMs.get shouldEqual(300000) + downsampleLp.asInstanceOf[PeriodicSeriesWithWindowing].offsetMs.get shouldEqual(120000) } } From 4ea7cd2ba3bd5c54adb2f39827c28335ff3acd34 Mon Sep 17 00:00:00 2001 From: "Brian S. O'Neill" Date: Fri, 8 May 2020 17:17:41 -0700 Subject: [PATCH 06/36] feat(core): Add periodic headroom task. (#746) --- .../memstore/TimeSeriesShard.scala | 69 +++++++++++++++++++ .../filodb.core/store/IngestionConfig.scala | 7 +- .../scala/filodb.memory/BlockManager.scala | 16 +++-- 3 files changed, 86 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala index 137430983d..85f7b6ac65 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala @@ -1,5 +1,6 @@ package filodb.core.memstore +import java.util.concurrent.TimeUnit import java.util.concurrent.locks.StampedLock import scala.collection.mutable.ArrayBuffer @@ -104,6 +105,12 @@ class TimeSeriesShardStats(dataset: DatasetRef, shardNum: Int) { * receiving them, assuming that the clocks are in sync. */ val ingestionClockDelay = Kamon.gauge("ingestion-clock-delay").withTags(TagSet.from(tags)) + + /** + * How much time the ingestion thread was potentially stalled while attempting to ensure + * free space. Unit is milliseconds. + */ + val ingestionHeadroomStall = Kamon.gauge("ingestion-headroom-stall").withTags(TagSet.from(tags)) } object TimeSeriesShard { @@ -312,6 +319,9 @@ class TimeSeriesShard(val ref: DatasetRef, */ private[memstore] final val reclaimLock = new StampedLock + // Requires reclaimLock. + startHeadroomTask(ingestSched) + // The off-heap block store used for encoded chunks private val shardTags = Map("dataset" -> ref.dataset, "shard" -> shardNum.toString) private val blockStore = new PageAlignedBlockManager(blockMemorySize, shardStats.memoryStats, reclaimListener, @@ -1473,6 +1483,65 @@ class TimeSeriesShard(val ref: DatasetRef, }) } + private def startHeadroomTask(sched: Scheduler): Unit = { + sched.scheduleWithFixedDelay(1, 1, TimeUnit.MINUTES, new Runnable { + def run() = ensureFreeBlocks + }) + } + + /** + * Expected to be called via a background task, to periodically ensure that enough blocks + * are free for new allocations. This helps prevent ODP activity from reclaiming immediately + * from itself. + */ + private def ensureFreeBlocks(): Unit = { + val start = System.currentTimeMillis() + val stamp = tryExclusiveReclaimLock() + if (stamp == 0) { + logger.warn(s"shard=$shardNum: ensureFreeBlocks timed out: ${reclaimLock}") + } else { + val numFree = try { + blockStore.ensureFreePercent(storeConfig.ensureHeadroomPercent) + } finally { + reclaimLock.unlockWrite(stamp) + } + val numBytes = numFree * blockStore.blockSizeInBytes + logger.debug(s"shard=$shardNum: ensureFreeBlocks: $numFree ($numBytes bytes)") + } + val stall = System.currentTimeMillis() - start + shardStats.ingestionHeadroomStall.update(stall) + } + + private def tryExclusiveReclaimLock(): Long = { + // Attempting to acquire the exclusive lock must wait for concurrent queries to finish, but + // waiting will also stall new queries from starting. To protect against this, attempt with + // a timeout to let any stalled queries through. To prevent starvation of the exclusive + // lock attempt, increase the timeout each time, but eventually give up. The reason why + // waiting for an exclusive lock causes this problem is that the thread must enqueue itself + // into the lock as a waiter, and all new shared requests must wait their turn. The risk + // with timing out is that if there's a continuous stream of long running queries (more than + // one second), then the exclusive lock will never be acqiured, and then ensureFreeBlocks + // won't be able to do its job. The timeout settings might need to be adjusted in that case. + // Perhaps the timeout should increase automatically if ensureFreeBlocks failed the last time? + // This isn't safe to do until we gain higher confidence that the shared lock is always + // released by queries. + + var timeout = 1; + while (true) { + val stamp = reclaimLock.tryWriteLock(timeout, TimeUnit.MILLISECONDS) + if (stamp != 0) { + return stamp + } + timeout <<= 1 + if (timeout > 1024) { + // Give up after waiting (in total) a little over 2 seconds. + return 0 + } + Thread.`yield`() + } + 0 // never reached, but scala compiler complains otherwise + } + /** * Please use this for testing only - reclaims ALL used offheap blocks. Maybe you are trying to test * on demand paging. diff --git a/core/src/main/scala/filodb.core/store/IngestionConfig.scala b/core/src/main/scala/filodb.core/store/IngestionConfig.scala index 6130c1a3c8..ca5530cc46 100644 --- a/core/src/main/scala/filodb.core/store/IngestionConfig.scala +++ b/core/src/main/scala/filodb.core/store/IngestionConfig.scala @@ -33,6 +33,8 @@ final case class StoreConfig(flushInterval: FiniteDuration, demandPagingParallelism: Int, demandPagingEnabled: Boolean, evictedPkBfCapacity: Int, + // Amount of free blocks to periodically reclaim, as a percent of total number of blocks + ensureHeadroomPercent: Double, // filters on ingested records to log in detail traceFilters: Map[String, String], maxQueryMatches: Int) { @@ -58,7 +60,8 @@ final case class StoreConfig(flushInterval: FiniteDuration, "demand-paging-parallelism" -> demandPagingParallelism, "demand-paging-enabled" -> demandPagingEnabled, "max-query-matches" -> maxQueryMatches, - "evicted-pk-bloom-filter-capacity" -> evictedPkBfCapacity).asJava) + "evicted-pk-bloom-filter-capacity" -> evictedPkBfCapacity, + "ensure-headroom-percent" -> ensureHeadroomPercent).asJava) } final case class AssignShardConfig(address: String, shardList: Seq[Int]) @@ -86,6 +89,7 @@ object StoreConfig { |demand-paging-parallelism = 10 |demand-paging-enabled = true |evicted-pk-bloom-filter-capacity = 5000000 + |ensure-headroom-percent = 5.0 |trace-filters = {} |""".stripMargin) /** Pass in the config inside the store {} */ @@ -116,6 +120,7 @@ object StoreConfig { config.getInt("demand-paging-parallelism"), config.getBoolean("demand-paging-enabled"), config.getInt("evicted-pk-bloom-filter-capacity"), + config.getDouble("ensure-headroom-percent"), config.as[Map[String, String]]("trace-filters"), config.getInt("max-query-matches")) } diff --git a/memory/src/main/scala/filodb.memory/BlockManager.scala b/memory/src/main/scala/filodb.memory/BlockManager.scala index 94cf4c84ab..a7cbc2b5a8 100644 --- a/memory/src/main/scala/filodb.memory/BlockManager.scala +++ b/memory/src/main/scala/filodb.memory/BlockManager.scala @@ -48,14 +48,18 @@ trait BlockManager { /** * Attempts to reclaim as many blocks as necessary to ensure that enough free blocks are * available. + * + * @return numFreeBlocks */ - def ensureFreeBlocks(num: Int): Unit + def ensureFreeBlocks(num: Int): Int /** * Attempts to reclaim as many blocks as necessary to ensure that enough free bytes are * available. The actual amount reclaimed might be higher than requested. + * + * @return numFreeBlocks */ - def ensureFreeBytes(amt: Long): Unit = { + def ensureFreeBytes(amt: Long): Int = { val blocks = (amt + blockSizeInBytes - 1) / blockSizeInBytes ensureFreeBlocks(Math.min(Integer.MAX_VALUE, blocks).toInt) } @@ -66,8 +70,9 @@ trait BlockManager { * requested. * * @param pct percentage: 0.0 to 100.0 + * @return numFreeBlocks */ - def ensureFreePercent(pct: Double): Unit = { + def ensureFreePercent(pct: Double): Int = { ensureFreeBytes((totalMemorySizeInBytes * pct * 0.01).toLong) } @@ -193,11 +198,12 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, } } - override def ensureFreeBlocks(num: Int): Unit = { + override def ensureFreeBlocks(num: Int): Int = { lock.lock() try { - val require = num - freeBlocks.size + val require = num - numFreeBlocks if (require > 0) tryReclaim(require) + numFreeBlocks } finally { lock.unlock() } From d21c18f4d1232ff9dee5df1ceaa5b932041d0834 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Mon, 18 May 2020 13:14:56 -0700 Subject: [PATCH 07/36] feat(core): Allow query execution if data sizing within limit (#748) We now estimate data size based on schema, sample size, time range and block queries that go beyond limit. The user guidance if this occurs is to reduce time range, or number of time series queried. --- conf/timeseries-dev-source.conf | 5 +- core/src/main/resources/filodb-defaults.conf | 5 +- .../DownsampledTimeSeriesShard.scala | 34 +++++++++----- .../memstore/OnDemandPagingShard.scala | 18 +++++++ .../memstore/TimeSeriesShard.scala | 7 +-- .../scala/filodb.core/metadata/Schemas.scala | 47 +++++++++++++++++++ .../filodb.core/store/IngestionConfig.scala | 10 ++-- 7 files changed, 99 insertions(+), 27 deletions(-) diff --git a/conf/timeseries-dev-source.conf b/conf/timeseries-dev-source.conf index 50a532b89a..17cfd9ea8a 100644 --- a/conf/timeseries-dev-source.conf +++ b/conf/timeseries-dev-source.conf @@ -95,9 +95,8 @@ # metric = "bad-metric-to-log" # } - # Maximum number of partitions per shard scanned per query. This is necessary - # to ensure no run-away query hogs memory and destabilizes the server. - # max-query-matches = 250000 + # Limits maximum amount of data a single leaf query can scan + max-data-per-shard-query = 50 MB } downsample { # can be disabled by setting this flag to false diff --git a/core/src/main/resources/filodb-defaults.conf b/core/src/main/resources/filodb-defaults.conf index 7fe98d5dea..06a217126f 100644 --- a/core/src/main/resources/filodb-defaults.conf +++ b/core/src/main/resources/filodb-defaults.conf @@ -163,6 +163,7 @@ filodb { # Enable faster rate/increase/delta calculations. Depends on drop detection in chunks (detectDrops=true) faster-rate = true + } shard-manager { @@ -259,8 +260,8 @@ filodb { # Read parallelism in downsample cluster demand-paging-parallelism = 30 - # Maximum number of TS partitions paged in in downsample cluster queries - max-query-matches = 100000 + # Limits maximum amount of data a single leaf query can scan per shard + max-data-per-shard-query = 100 MB # Write buffer size, in bytes, for blob columns (histograms, UTF8Strings). Since these are variable data types, # we need a maximum size, not a maximum number of items. diff --git a/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesShard.scala b/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesShard.scala index d3467acfb7..097783ae08 100644 --- a/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesShard.scala @@ -54,9 +54,6 @@ class DownsampledTimeSeriesShard(rawDatasetRef: DatasetRef, private val downsampleStoreConfig = StoreConfig(filodbConfig.getConfig("downsampler.downsample-store-config")) - // since all partitions are paged from store, this would be much lower than what is configured for raw data - private val maxQueryMatches = downsampleStoreConfig.maxQueryMatches - private val nextPartitionID = new AtomicInteger(0) private val stats = new DownsampledTimeSeriesShardStats(rawDatasetRef, shardNum) @@ -235,18 +232,15 @@ class DownsampledTimeSeriesShard(rawDatasetRef: DatasetRef, querySession: QuerySession): Observable[ReadablePartition] = { // Step 1: Choose the downsample level depending on the range requested - val downsampledDataset = chooseDownsampleResolution(lookup.chunkMethod) + val (resolution, downsampledDataset) = chooseDownsampleResolution(lookup.chunkMethod) logger.debug(s"Chose resolution $downsampledDataset for chunk method ${lookup.chunkMethod}") + + capDataScannedPerShardCheck(lookup, resolution) + // Step 2: Query Cassandra table for that downsample level using downsampleColStore // Create a ReadablePartition objects that contain the time series data. This can be either a // PagedReadablePartitionOnHeap or PagedReadablePartitionOffHeap. This will be garbage collected/freed // when query is complete. - - if (lookup.partsInMemory.length > maxQueryMatches) - throw new IllegalArgumentException(s"Seeing ${lookup.partsInMemory.length} matching time series per shard. Try " + - s"to narrow your query by adding more filters so there is less than $maxQueryMatches matches " + - s"or request for increasing number of shards this metric lives in") - val partKeys = lookup.partsInMemory.iterator().map(partKeyFromPartId) Observable.fromIterator(partKeys) // 3 times value configured for raw dataset since expected throughput for downsampled cluster is much lower @@ -273,6 +267,18 @@ class DownsampledTimeSeriesShard(rawDatasetRef: DatasetRef, } } + private def capDataScannedPerShardCheck(lookup: PartLookupResult, resolution: Long) = { + lookup.firstSchemaId.foreach { schId => + lookup.chunkMethod match { + case TimeRangeChunkScan(st, end) => + schemas.ensureQueriedDataSizeWithinLimit(schId, lookup.partsInMemory.length, + downsampleStoreConfig.flushInterval.toMillis, + resolution, end - st, downsampleStoreConfig.maxDataPerShardQuery) + case _ => + } + } + } + protected def schemaIDFromPartID(partID: Int): Int = { partKeyIndex.partKeyFromPartId(partID).map { pkBytesRef => val unsafeKeyOffset = PartKeyLuceneIndex.bytesRefToUnsafeOffset(pkBytesRef.offset) @@ -280,12 +286,14 @@ class DownsampledTimeSeriesShard(rawDatasetRef: DatasetRef, }.getOrElse(throw new IllegalStateException(s"PartId $partID returned by lucene, but partKey not found")) } - private def chooseDownsampleResolution(chunkScanMethod: ChunkScanMethod): DatasetRef = { + private def chooseDownsampleResolution(chunkScanMethod: ChunkScanMethod): (Long, DatasetRef) = { chunkScanMethod match { - case AllChunkScan => downsampledDatasetRefs.last // since it is the highest resolution/ttl + case AllChunkScan => + // since it is the highest resolution/ttl + downsampleTtls.last.toMillis -> downsampledDatasetRefs.last case TimeRangeChunkScan(startTime, _) => val ttlIndex = downsampleTtls.indexWhere(t => startTime > System.currentTimeMillis() - t.toMillis) - downsampledDatasetRefs(ttlIndex) + downsampleConfig.resolutions(ttlIndex).toMillis -> downsampledDatasetRefs(ttlIndex) case _ => ??? } } diff --git a/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala b/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala index 707d5b0086..0eb4dd1567 100644 --- a/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala +++ b/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala @@ -51,6 +51,21 @@ TimeSeriesShard(ref, schemas, storeConfig, shardNum, bufferMemoryManager, rawSto .tag("shard", shardNum) .start() + val assumedResolution = 20000 // for now hard-code and assume 30ms as reporting interval + + private def capDataScannedPerShardCheck(lookup: PartLookupResult): Unit = { + lookup.firstSchemaId.foreach { schId => + lookup.chunkMethod match { + case TimeRangeChunkScan(st, end) => + val numMatches = lookup.partsInMemory.length + lookup.partIdsNotInMemory.length + schemas.ensureQueriedDataSizeWithinLimit(schId, numMatches, + storeConfig.flushInterval.toMillis, + assumedResolution, end - st, storeConfig.maxDataPerShardQuery) + case _ => + } + } + } + // NOTE: the current implementation is as follows // 1. Fetch partitions from memStore // 2. Determine, one at a time, what chunks are missing and could be fetched from disk @@ -60,6 +75,9 @@ TimeSeriesShard(ref, schemas, storeConfig, shardNum, bufferMemoryManager, rawSto //scalastyle:off override def scanPartitions(partLookupRes: PartLookupResult, querySession: QuerySession): Observable[ReadablePartition] = { + + capDataScannedPerShardCheck(partLookupRes) + // For now, always read every data column. // 1. We don't have a good way to update just some columns of a chunkset for ODP // 2. Timestamp column almost always needed diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala index 85f7b6ac65..49bc00cce2 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala @@ -1437,17 +1437,13 @@ class TimeSeriesShard(val ref: DatasetRef, val partIds = debox.Buffer.empty[Int] partKeys.flatMap(getPartition).foreach(p => partIds += p.partID) PartLookupResult(shardNum, chunkMethod, partIds, partKeys.headOption.map(RecordSchema.schemaID)) - case FilteredPartitionScan(split, filters) => + case FilteredPartitionScan(_, filters) => // No matter if there are filters or not, need to run things through Lucene so we can discover potential // TSPartitions to read back from disk val matches = partKeyIndex.partIdsFromFilters(filters, chunkMethod.startTime, chunkMethod.endTime) shardStats.queryTimeRangeMins.record((chunkMethod.endTime - chunkMethod.startTime) / 60000 ) Kamon.currentSpan().tag(s"num-partitions-from-index-$shardNum", matches.length) - if (matches.length > storeConfig.maxQueryMatches) - throw new IllegalArgumentException(s"Seeing ${matches.length} matching time series per shard. Try " + - s"to narrow your query by adding more filters so there is less than " + - s"${storeConfig.maxQueryMatches} matches or request that number of shards for the metric be increased") // first find out which partitions are being queried for data not in memory val firstPartId = if (matches.isEmpty) None else Some(matches(0)) @@ -1476,6 +1472,7 @@ class TimeSeriesShard(val ref: DatasetRef, def scanPartitions(iterResult: PartLookupResult, querySession: QuerySession): Observable[ReadablePartition] = { + val partIter = new InMemPartitionIterator2(iterResult.partsInMemory) Observable.fromIterator(partIter.map { p => shardStats.partitionsQueried.increment() diff --git a/core/src/main/scala/filodb.core/metadata/Schemas.scala b/core/src/main/scala/filodb.core/metadata/Schemas.scala index 199e63a2df..d251b1eff5 100644 --- a/core/src/main/scala/filodb.core/metadata/Schemas.scala +++ b/core/src/main/scala/filodb.core/metadata/Schemas.scala @@ -8,6 +8,7 @@ import filodb.core.GlobalConfig import filodb.core.Types._ import filodb.core.binaryrecord2._ import filodb.core.downsample.{ChunkDownsampler, DownsamplePeriodMarker} +import filodb.core.metadata.Column.ColumnType import filodb.core.query.ColumnInfo import filodb.core.store.ChunkSetInfo import filodb.memory.BinaryRegion @@ -260,6 +261,52 @@ final case class Schemas(part: PartitionSchema, schemas.values.foreach { s => _schemas(s.schemaHash) = s } + /** + * This is purely a SWAG to be used for query size estimation. Do not rely for other use cases. + */ + private val bytesPerSampleSwag: Map[Int, Double] = { + + val allSchemas = schemas.values ++ schemas.values.flatMap(_.downsample) + allSchemas.map { s => + val est = s.data.columns.map(_.columnType).map { + case ColumnType.LongColumn => 1 + case ColumnType.IntColumn => 1 + case ColumnType.TimestampColumn => 0.5 + case ColumnType.HistogramColumn => 20 + case ColumnType.DoubleColumn => 2 + case _ => 0 // TODO allow without sizing for now + }.sum + s.schemaHash -> est + }.toMap + } + + /** + * Note this approach below assumes the following for quick size estimation. The sizing is more + * a swag than reality: + * (a) every matched time series ingests at all query times. Looking up start/end times and more + * precise size estimation is costly + * (b) it also assigns bytes per sample based on schema which is much of a swag. In reality, it would depend on + * number of histogram buckets, samples per chunk etc. + */ + def ensureQueriedDataSizeWithinLimit(schemaId: Int, + numTsPartitions: Int, + chunkDurationMillis: Long, + resolutionMs: Long, + queryDurationMs: Long, + dataSizeLimit: Long): Unit = { + val numSamplesPerChunk = chunkDurationMillis / resolutionMs + // find number of chunks to be scanned. Ceil division needed here + val numChunksPerTs = (queryDurationMs + chunkDurationMillis - 1) / chunkDurationMillis + val bytesPerSample = bytesPerSampleSwag(schemaId) + val estDataSize = bytesPerSample * numTsPartitions * numSamplesPerChunk * numChunksPerTs + require(estDataSize < dataSizeLimit, + s"Estimate of $estDataSize bytes exceeds limit of " + + s"$dataSizeLimit bytes queried per shard with $bytesPerSample bytes per sample " + + s"for ${_schemas(schemaId).name} schema. Try one or more of these: " + + s"(a) narrow your query filters to reduce to fewer than the current $numTsPartitions matches " + + s"(b) reduce query time range, currently at ${queryDurationMs / 1000 / 60 } minutes") + } + /** * Returns the Schema for a given schemaID, or UnknownSchema if not found */ diff --git a/core/src/main/scala/filodb.core/store/IngestionConfig.scala b/core/src/main/scala/filodb.core/store/IngestionConfig.scala index ca5530cc46..9c77b82156 100644 --- a/core/src/main/scala/filodb.core/store/IngestionConfig.scala +++ b/core/src/main/scala/filodb.core/store/IngestionConfig.scala @@ -37,7 +37,7 @@ final case class StoreConfig(flushInterval: FiniteDuration, ensureHeadroomPercent: Double, // filters on ingested records to log in detail traceFilters: Map[String, String], - maxQueryMatches: Int) { + maxDataPerShardQuery: Long) { import collection.JavaConverters._ def toConfig: Config = ConfigFactory.parseMap(Map("flush-interval" -> (flushInterval.toSeconds + "s"), @@ -59,7 +59,7 @@ final case class StoreConfig(flushInterval: FiniteDuration, "multi-partition-odp" -> multiPartitionODP, "demand-paging-parallelism" -> demandPagingParallelism, "demand-paging-enabled" -> demandPagingEnabled, - "max-query-matches" -> maxQueryMatches, + "max-data-per-shard-query" -> maxDataPerShardQuery, "evicted-pk-bloom-filter-capacity" -> evictedPkBfCapacity, "ensure-headroom-percent" -> ensureHeadroomPercent).asJava) } @@ -70,11 +70,13 @@ final case class UnassignShardConfig(shardList: Seq[Int]) object StoreConfig { // NOTE: there are no defaults for flush interval and shard memory, those should be explicitly calculated + // default max-data-per-shard-query was calculated as follows: + // 750k TsPartitions * 48 chunksets queried * 2kb per chunkset / 256 shards = 280MB val defaults = ConfigFactory.parseString(""" |disk-time-to-live = 3 days |demand-paged-chunk-retention-period = 72 hours |max-chunks-size = 400 - |max-query-matches = 250000 + |max-data-per-shard-query = 300 MB |max-blob-buffer-size = 15000 |ingestion-buffer-mem-size = 10M |max-buffer-pool-size = 10000 @@ -122,7 +124,7 @@ object StoreConfig { config.getInt("evicted-pk-bloom-filter-capacity"), config.getDouble("ensure-headroom-percent"), config.as[Map[String, String]]("trace-filters"), - config.getInt("max-query-matches")) + config.getMemorySize("max-data-per-shard-query").toBytes) } } From 7ef41672df6f68b23ee82c9d7a7493fd1c63ddd0 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Tue, 19 May 2020 20:56:51 -0700 Subject: [PATCH 08/36] feat(sparkJobs): Ability to trace downsampling of TS partitions (#752) --- core/src/main/resources/filodb-defaults.conf | 9 +++++ .../downsampler/chunk/BatchDownsampler.scala | 34 +++++++++++++------ .../chunk/DownsamplerSettings.scala | 6 ++++ 3 files changed, 39 insertions(+), 10 deletions(-) diff --git a/core/src/main/resources/filodb-defaults.conf b/core/src/main/resources/filodb-defaults.conf index 06a217126f..b74fc6e635 100644 --- a/core/src/main/resources/filodb-defaults.conf +++ b/core/src/main/resources/filodb-defaults.conf @@ -300,6 +300,15 @@ filodb { # tagB3 = value3 #} ] + + # Downsampling of TS Partition will be traced if it matches the filter below + trace-filters = [ + #{ + # tagB1 = value1 + # tagB2 = value2 + # tagB3 = value3 + #} + ] } ds-index-job { 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 71a6a8a94c..5903cab554 100644 --- a/spark-jobs/src/main/scala/filodb/downsampler/chunk/BatchDownsampler.scala +++ b/spark-jobs/src/main/scala/filodb/downsampler/chunk/BatchDownsampler.scala @@ -12,7 +12,7 @@ import filodb.cassandra.columnstore.CassandraColumnStore import filodb.core.{DatasetRef, ErrorResponse, Instance} import filodb.core.binaryrecord2.{RecordBuilder, RecordSchema} import filodb.core.downsample._ -import filodb.core.memstore.{PagedReadablePartition, TimeSeriesPartition, TimeSeriesShardStats} +import filodb.core.memstore._ import filodb.core.metadata.Schemas import filodb.core.store.{AllChunkScan, ChunkSet, RawPartData, ReadablePartition} import filodb.downsampler.DownsamplerContext @@ -135,8 +135,9 @@ class BatchDownsampler(settings: DownsamplerSettings) extends Instance with Seri val pkPairs = schema.partKeySchema.toStringPairs(rawPart.partitionKey, UnsafeUtils.arayOffset) if (settings.isEligibleForDownsample(pkPairs)) { try { + val shouldTrace = settings.shouldTrace(pkPairs) downsamplePart(offHeapMem, rawPart, pagedPartsToFree, downsampledPartsToFree, - downsampledChunksToPersist, userTimeStart, userTimeEndExclusive, dsRecordBuilder) + downsampledChunksToPersist, userTimeStart, userTimeEndExclusive, dsRecordBuilder, shouldTrace) numPartitionsCompleted.increment() } catch { case e: Exception => DownsamplerContext.dsLogger.error(s"Error occurred when downsampling partition $pkPairs", e) @@ -178,6 +179,7 @@ class BatchDownsampler(settings: DownsamplerSettings) extends Instance with Seri * @param downsampledPartsToFree downsample partitions to be freed are added to this mutable list * @param downsampledChunksToPersist downsample chunks to persist are added to this mutable map */ + // scalastyle:off parameter.number private def downsamplePart(offHeapMem: OffHeapMemory, rawPart: RawPartData, pagedPartsToFree: ArrayBuffer[PagedReadablePartition], @@ -185,7 +187,8 @@ class BatchDownsampler(settings: DownsamplerSettings) extends Instance with Seri downsampledChunksToPersist: MMap[FiniteDuration, Iterator[ChunkSet]], userTimeStart: Long, userTimeEndExclusive: Long, - dsRecordBuilder: RecordBuilder) = { + dsRecordBuilder: RecordBuilder, + shouldTrace: Boolean) = { val rawSchemaId = RecordSchema.schemaID(rawPart.partitionKey, UnsafeUtils.arayOffset) val rawPartSchema = schemas(rawSchemaId) @@ -205,15 +208,20 @@ class BatchDownsampler(settings: DownsamplerSettings) extends Instance with Seri // update schema of the partition key to downsample schema RecordBuilder.updateSchema(UnsafeUtils.ZeroPointer, partKeyPtr, downsampleSchema) - val downsampledParts = settings.downsampleResolutions.map { res => - val part = new TimeSeriesPartition(0, downsampleSchema, partKeyPtr, - 0, bufferPool, shardStats, offHeapMem.nativeMemoryManager, 1) - res -> part + val downsampledParts = settings.downsampleResolutions.zip(settings.downsampledDatasetRefs).map { + case (res, ref) => + val part = if (shouldTrace) + new TracingTimeSeriesPartition(0, ref, downsampleSchema, partKeyPtr, + 0, bufferPool, shardStats, offHeapMem.nativeMemoryManager, 1) + else + new TimeSeriesPartition(0, downsampleSchema, partKeyPtr, + 0, bufferPool, shardStats, offHeapMem.nativeMemoryManager, 1) + res -> part }.toMap val downsamplePartSpan = Kamon.spanBuilder("downsample-single-partition-latency").start() - downsampleChunks(offHeapMem, rawReadablePart, downsamplers, periodMarker, - downsampledParts, userTimeStart, userTimeEndExclusive, dsRecordBuilder) + downsampleChunks(offHeapMem, rawReadablePart, downsamplers, periodMarker, downsampledParts, + userTimeStart, userTimeEndExclusive, dsRecordBuilder, shouldTrace) pagedPartsToFree += rawReadablePart downsampledPartsToFree ++= downsampledParts.values @@ -245,7 +253,9 @@ class BatchDownsampler(settings: DownsamplerSettings) extends Instance with Seri downsampleResToPart: Map[FiniteDuration, TimeSeriesPartition], userTimeStart: Long, userTimeEndExclusive: Long, - dsRecordBuilder: RecordBuilder) = { + dsRecordBuilder: RecordBuilder, + shouldTrace: Boolean) = { + val timestampCol = 0 val rawChunksets = rawPartToDownsample.infos(AllChunkScan) @@ -273,6 +283,10 @@ class BatchDownsampler(settings: DownsamplerSettings) extends Instance with Seri periodMarker.periods(rawPartToDownsample, chunkset, resMillis, startRow, endRow).toArray() java.util.Arrays.sort(downsamplePeriods) + if (shouldTrace) + DownsamplerContext.dsLogger.info(s"Downsample Periods for ${part.stringPartition} " + + s"${chunkset.debugString} resolution=$resolution downsamplePeriods=$downsamplePeriods") + try { // for each downsample period var first = startRow diff --git a/spark-jobs/src/main/scala/filodb/downsampler/chunk/DownsamplerSettings.scala b/spark-jobs/src/main/scala/filodb/downsampler/chunk/DownsamplerSettings.scala index 4e6fc01a9b..4ee4f055cf 100644 --- a/spark-jobs/src/main/scala/filodb/downsampler/chunk/DownsamplerSettings.scala +++ b/spark-jobs/src/main/scala/filodb/downsampler/chunk/DownsamplerSettings.scala @@ -68,6 +68,8 @@ class DownsamplerSettings(conf: Config = ConfigFactory.empty()) extends Serializ @transient lazy val blacklist = downsamplerConfig.as[Seq[Map[String, String]]]("blacklist-filters").map(_.toSeq) + @transient lazy val trace = downsamplerConfig.as[Seq[Map[String, String]]]("trace-filters").map(_.toSeq) + /** * Two conditions should satisfy for eligibility: * (a) If whitelist is nonEmpty partKey should match a filter in the whitelist. @@ -81,5 +83,9 @@ class DownsamplerSettings(conf: Config = ConfigFactory.empty()) extends Serializ } } + def shouldTrace(pkPairs: Seq[(String, String)]): Boolean = { + trace.exists(w => w.forall(pkPairs.contains)) + } + } From 1224b28141b702e69acbc66bca1060f0076b60b3 Mon Sep 17 00:00:00 2001 From: "Brian S. O'Neill" Date: Tue, 19 May 2020 21:10:56 -0700 Subject: [PATCH 09/36] feat(core): ODP block allocation shouldn't force a reclamation. (#750) --- .../scala/filodb.memory/BlockManager.scala | 21 +++++++++- .../PageAlignedBlockManagerSpec.scala | 39 ++++++++++++++++++- 2 files changed, 58 insertions(+), 2 deletions(-) diff --git a/memory/src/main/scala/filodb.memory/BlockManager.scala b/memory/src/main/scala/filodb.memory/BlockManager.scala index a7cbc2b5a8..688fef58fe 100644 --- a/memory/src/main/scala/filodb.memory/BlockManager.scala +++ b/memory/src/main/scala/filodb.memory/BlockManager.scala @@ -10,6 +10,8 @@ import kamon.Kamon import kamon.metric.Counter import kamon.tag.TagSet +final case class MemoryRequestException(msg: String) extends Exception(msg) + /** * Allows requesting blocks. */ @@ -169,6 +171,15 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, * 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 + * 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 + * 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], @@ -178,7 +189,15 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, val num: Int = Math.ceil(memorySize / blockSizeInBytes).toInt stats.requestedBlocksMetric.increment(num) - if (freeBlocks.size < num) tryReclaim(num) + if (freeBlocks.size < num) { + if (bucketTime.isEmpty) { + tryReclaim(num) + } else { + val msg = s"Unable to allocate time ordered block(s) without forcing a reclamation: " + + s"num_blocks=$num num_bytes=$memorySize freeBlocks=${freeBlocks.size}" + throw new MemoryRequestException(msg) + } + } if (freeBlocks.size >= num) { val allocated = new Array[Block](num) diff --git a/memory/src/test/scala/filodb.memory/PageAlignedBlockManagerSpec.scala b/memory/src/test/scala/filodb.memory/PageAlignedBlockManagerSpec.scala index c15c1a4ab3..7db1be1180 100644 --- a/memory/src/test/scala/filodb.memory/PageAlignedBlockManagerSpec.scala +++ b/memory/src/test/scala/filodb.memory/PageAlignedBlockManagerSpec.scala @@ -18,7 +18,6 @@ object PageAlignedBlockManagerSpec { class PageAlignedBlockManagerSpec extends FlatSpec with Matchers with BeforeAndAfter { import PageAlignedBlockManagerSpec._ - import collection.JavaConverters._ val pageSize = PageManager.getInstance().pageSize() @@ -114,6 +113,7 @@ class PageAlignedBlockManagerSpec extends FlatSpec with Matchers with BeforeAndA blockManager.releaseBlocks() } + /* No longer valid now that time ordered allocation doesn't force reclamation. it should "allocate and reclaim blocks with time order" in { val stats = new MemoryStats(Map("test5" -> "test5")) // This block manager has 5 blocks capacity @@ -168,6 +168,42 @@ class PageAlignedBlockManagerSpec extends FlatSpec with Matchers with BeforeAndA blockManager.releaseBlocks() } + */ + + it should "fail to allocate time order block" in { + val stats = new MemoryStats(Map("test5" -> "test5")) + // 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.requestBlock(Some(9000L)).map(_.markReclaimable).isDefined shouldEqual true + blockManager.usedBlocksTimeOrdered.get(9000L).size() shouldEqual 1 + + blockManager.numTimeOrderedBlocks shouldEqual 4 + blockManager.usedBlocksTimeOrdered.size shouldEqual 2 + + // reclaim from time ordered blocks should fail now + try { + blockManager.requestBlock(Some(10000L)) + fail + } catch { + case e: MemoryRequestException => // expected + } + + blockManager.releaseBlocks() + } it should ("allocate blocks using BlockMemFactory with ownership and reclaims") in { val stats = new MemoryStats(Map("test5" -> "test5")) @@ -198,6 +234,7 @@ class PageAlignedBlockManagerSpec extends FlatSpec with Matchers with BeforeAndA // 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 From 93e61e7c84ce451db03cc837447e47a04ce5515d Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Tue, 19 May 2020 21:41:05 -0700 Subject: [PATCH 10/36] debug(sparkJobs): Fixing debug string (#753) --- .../main/scala/filodb/downsampler/chunk/BatchDownsampler.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 5903cab554..980beef390 100644 --- a/spark-jobs/src/main/scala/filodb/downsampler/chunk/BatchDownsampler.scala +++ b/spark-jobs/src/main/scala/filodb/downsampler/chunk/BatchDownsampler.scala @@ -285,7 +285,8 @@ class BatchDownsampler(settings: DownsamplerSettings) extends Instance with Seri if (shouldTrace) DownsamplerContext.dsLogger.info(s"Downsample Periods for ${part.stringPartition} " + - s"${chunkset.debugString} resolution=$resolution downsamplePeriods=$downsamplePeriods") + s"${chunkset.debugString} resolution=$resolution " + + s"downsamplePeriods=${downsamplePeriods.mkString(",")}") try { // for each downsample period From a5a8daf8d01fec517de03beb5d49d51b3293dfea Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Wed, 20 May 2020 10:38:05 -0700 Subject: [PATCH 11/36] debug(core) Tracing Partition should log at info level since explitly enabled (#754) --- .../filodb.core/memstore/TimeSeriesPartition.scala | 10 +++++----- 1 file changed, 5 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 a27b1ce90e..dd8e5ee884 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala @@ -452,27 +452,27 @@ class TracingTimeSeriesPartition(partID: Int, TimeSeriesPartition(partID, schema, partitionKey, shard, bufferPool, shardStats, memFactory, initMapSize) { import TimeSeriesPartition._ - _log.debug(s"Creating TracingTimeSeriesPartition dataset=$ref schema=${schema.name} partId=$partID $stringPartition") + _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 = { val ts = row.getLong(0) - _log.debug(s"Ingesting dataset=$ref schema=${schema.name} shard=$shard partId=$partID $stringPartition " + + _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) } override def switchBuffers(blockHolder: BlockMemFactory, encode: Boolean = false): Boolean = { - _log.debug(s"SwitchBuffers dataset=$ref schema=${schema.name} shard=$shard partId=$partID $stringPartition - " + + _log.info(s"SwitchBuffers dataset=$ref schema=${schema.name} shard=$shard partId=$partID $stringPartition - " + s"encode=$encode for currentChunk ${currentInfo.debugString}") super.switchBuffers(blockHolder, encode) } override protected def initNewChunk(startTime: Long, ingestionTime: Long): Unit = { - _log.debug(s"dataset=$ref schema=${schema.name} shard=$shard partId=$partID $stringPartition - " + + _log.info(s"dataset=$ref schema=${schema.name} shard=$shard partId=$partID $stringPartition - " + s"initNewChunk($startTime, $ingestionTime)") super.initNewChunk(startTime, ingestionTime) - _log.debug(s"dataset=$ref schema=${schema.name} shard=$shard partId=$partID $stringPartition - " + + _log.info(s"dataset=$ref schema=${schema.name} shard=$shard partId=$partID $stringPartition - " + s"newly created ChunkInfo ${currentInfo.debugString}") } } From a09187f6e082c867c86602868722265e81a093bd Mon Sep 17 00:00:00 2001 From: TanviBhavsar Date: Thu, 21 May 2020 09:47:56 -0700 Subject: [PATCH 12/36] feat(query, coordinator): implement SinglePartitionPlanner (#749) --- .../queryplanner/CompositePlanner.scala | 42 ----- .../queryplanner/LogicalPlanUtils.scala | 54 +++++- .../queryplanner/LongTimeRangePlanner.scala | 18 +- .../queryplanner/MultiClusterPlanner.scala | 19 --- .../queryplanner/SingleClusterPlanner.scala | 26 +-- .../queryplanner/SinglePartitionPlanner.scala | 76 +++++++++ .../queryplanner/LogicalPlanUtilsSpec.scala | 12 +- .../SinglePartitionPlannerSpec.scala | 160 ++++++++++++++++++ .../filodb/query/util/LogicalPlanUtil.scala | 37 ---- 9 files changed, 316 insertions(+), 128 deletions(-) delete mode 100644 coordinator/src/main/scala/filodb.coordinator/queryplanner/CompositePlanner.scala delete mode 100644 coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiClusterPlanner.scala create mode 100644 coordinator/src/main/scala/filodb.coordinator/queryplanner/SinglePartitionPlanner.scala rename query/src/test/scala/filodb/query/util/LogicalPlanUtilSpec.scala => coordinator/src/test/scala/filodb.coordinator/queryplanner/LogicalPlanUtilsSpec.scala (85%) create mode 100644 coordinator/src/test/scala/filodb.coordinator/queryplanner/SinglePartitionPlannerSpec.scala delete mode 100644 query/src/main/scala/filodb/query/util/LogicalPlanUtil.scala diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/CompositePlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/CompositePlanner.scala deleted file mode 100644 index 78850b68a5..0000000000 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/CompositePlanner.scala +++ /dev/null @@ -1,42 +0,0 @@ -package filodb.coordinator.queryplanner - -import com.typesafe.scalalogging.StrictLogging - -import filodb.coordinator.ShardMapper -import filodb.coordinator.client.QueryCommands.StaticSpreadProvider -import filodb.core.{DatasetRef, SpreadProvider} -import filodb.core.metadata.Schemas -import filodb.core.query.{QueryConfig, QueryContext} -import filodb.query._ -import filodb.query.exec._ - -/** - * Query Planner implementation that composes other planners to provide overall capability - * of high availability, downsampling and (later) multi-cluster partitioning. - */ -class CompositePlanner(dsRef: DatasetRef, - schemas: Schemas, - shardMapperFunc: => ShardMapper, - downsampleMapperFunc: => ShardMapper, - failureProvider: FailureProvider, - earliestRawTimestampFn: => Long, - earliestDownsampleTimestampFn: => Long, - latestDownsampleTimestampFn: => Long, - queryConfig: QueryConfig, - spreadProvider: SpreadProvider = StaticSpreadProvider(), - stitchDispatcher: => PlanDispatcher = { InProcessPlanDispatcher }) extends QueryPlanner - with StrictLogging { - // Note the composition of query planners below using decorator pattern - val rawClusterPlanner = new SingleClusterPlanner(dsRef, schemas, shardMapperFunc, - earliestRawTimestampFn, queryConfig, spreadProvider) - val downsampleClusterPlanner = new SingleClusterPlanner(dsRef, schemas, downsampleMapperFunc, - earliestDownsampleTimestampFn, queryConfig, spreadProvider) - val longTimeRangePlanner = new LongTimeRangePlanner(rawClusterPlanner, downsampleClusterPlanner, - earliestRawTimestampFn, latestDownsampleTimestampFn, stitchDispatcher) - val haPlanner = new HighAvailabilityPlanner(dsRef, longTimeRangePlanner, failureProvider, queryConfig) - //val multiPodPlanner = new MultiClusterPlanner(podLocalityProvider, haPlanner) - - def materialize(rootLogicalPlan: LogicalPlan, options: QueryContext): ExecPlan = { - haPlanner.materialize(rootLogicalPlan, options) - } -} diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala index e0c05dcf90..b050b7452d 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala @@ -1,5 +1,7 @@ package filodb.coordinator.queryplanner +import filodb.core.query.ColumnFilter +import filodb.prometheus.ast.Vectors.PromMetricLabel import filodb.prometheus.ast.WindowConstants import filodb.query._ @@ -29,7 +31,7 @@ object LogicalPlanUtils { case lp: ApplyInstantFunction => getPeriodicSeriesTimeFromLogicalPlan(lp.vectors) case lp: Aggregate => getPeriodicSeriesTimeFromLogicalPlan(lp.vectors) case lp: BinaryJoin => // can assume lhs & rhs have same time - getPeriodicSeriesTimeFromLogicalPlan(lp.lhs) + getPeriodicSeriesTimeFromLogicalPlan(lp.lhs) case lp: ScalarVectorBinaryOperation => getPeriodicSeriesTimeFromLogicalPlan(lp.vector) case lp: ApplyMiscellaneousFunction => getPeriodicSeriesTimeFromLogicalPlan(lp.vectors) case lp: ApplySortFunction => getPeriodicSeriesTimeFromLogicalPlan(lp.vectors) @@ -121,4 +123,54 @@ object LogicalPlanUtils { case _ => 0 } } + + def getMetricName(logicalPlan: LogicalPlan, datasetMetricColumn: String): Option[Set[String]] = { + val metricName = getLabelValueFromLogicalPlan(logicalPlan, PromMetricLabel) + if (metricName.isEmpty) getLabelValueFromLogicalPlan(logicalPlan, datasetMetricColumn) + else metricName + } + + private def getLabelValueFromFilters(filters: Seq[ColumnFilter], labelName: String): Option[Set[String]] = { + val matchingFilters = filters.filter(_.column.equals(labelName)) + if (matchingFilters.isEmpty) + None + else + Some(matchingFilters.head.filter.valuesStrings.map(_.toString)) + } + + def getLabelValueFromLogicalPlan(logicalPlan: LogicalPlan, labelName: String): Option[Set[String]] = { + val labelValues = LogicalPlan.findLeafLogicalPlans(logicalPlan).flatMap { lp => + lp match { + case lp: LabelValues => lp.labelConstraints.get(labelName).map(Set(_)) + case lp: RawSeries => getLabelValueFromFilters(lp.filters, labelName) + case lp: RawChunkMeta => getLabelValueFromFilters(lp.filters, labelName) + case lp: SeriesKeysByFilters => getLabelValueFromFilters(lp.filters, labelName) + case lp: ScalarTimeBasedPlan => Nil // Plan does not have labels + case lp: ScalarFixedDoublePlan => Nil + case lp: ScalarBinaryOperation => Nil + case _ => throw new BadQueryException(s"Invalid logical plan $logicalPlan") + } + } + if (labelValues.isEmpty) { + None + } else { + val res: Set[String] = Set() + // Concatenate results + Some(labelValues.foldLeft(res) { (acc, i) => i.union(acc) }) + } + } + + /** + * Renames Prom AST __name__ label to one based on the actual metric column of the dataset, + * if it is not the prometheus standard + */ + def renameLabels(labels: Seq[String], datasetMetricColumn: String): Seq[String] = + if (datasetMetricColumn != PromMetricLabel) { + labels map { + case PromMetricLabel => datasetMetricColumn + case other: String => other + } + } else { + labels + } } diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LongTimeRangePlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LongTimeRangePlanner.scala index 9e389d621c..eec81ff187 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LongTimeRangePlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LongTimeRangePlanner.scala @@ -2,8 +2,8 @@ package filodb.coordinator.queryplanner import filodb.coordinator.queryplanner.LogicalPlanUtils._ import filodb.core.query.QueryContext -import filodb.query.{LogicalPlan, PeriodicSeriesPlan} -import filodb.query.exec.{ExecPlan, PlanDispatcher, StitchRvsExec} +import filodb.query.{LabelValues, LogicalPlan, PeriodicSeriesPlan, SeriesKeysByFilters} +import filodb.query.exec.{ExecPlan, LabelValuesDistConcatExec, PartKeysDistConcatExec, PlanDispatcher, StitchRvsExec} /** * LongTimeRangePlanner knows about limited retention of raw data, and existence of downsampled data. @@ -66,9 +66,17 @@ class LongTimeRangePlanner(rawClusterPlanner: QueryPlanner, val rawEp = rawClusterPlanner.materialize(rawLp, qContext) StitchRvsExec(qContext, stitchDispatcher, Seq(rawEp, downsampleEp)) } - case _ => - // for now send everything else to raw cluster. Metadata queries are TODO - rawClusterPlanner.materialize(logicalPlan, qContext) + case l: LabelValues => + val rawExec = rawClusterPlanner.materialize(l, qContext) + val downSampleExec = downsampleClusterPlanner.materialize(l, qContext) + LabelValuesDistConcatExec(qContext, rawExec.dispatcher, Seq(rawExec, downSampleExec)) + + case s: SeriesKeysByFilters => + val rawExec = rawClusterPlanner.materialize(s, qContext) + val downSampleExec = downsampleClusterPlanner.materialize(s, qContext) + PartKeysDistConcatExec(qContext, rawExec.dispatcher, Seq(rawExec, downSampleExec)) + + case _ => rawClusterPlanner.materialize(logicalPlan, qContext) } } } diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiClusterPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiClusterPlanner.scala deleted file mode 100644 index 54cef39f8e..0000000000 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiClusterPlanner.scala +++ /dev/null @@ -1,19 +0,0 @@ -package filodb.coordinator.queryplanner - -import filodb.core.query.QueryContext -import filodb.query.LogicalPlan -import filodb.query.exec.ExecPlan - -/** - * MultiClusterPlanner is responsible for planning in situations where time series data is - * distributed across multiple clusters. - * - * This is TBD. - */ -class MultiClusterPlanner extends QueryPlanner { - - def materialize(logicalPlan: LogicalPlan, qContext: QueryContext): ExecPlan = { - ??? - } - -} diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala index 981a2aa97e..680ffc9b50 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala @@ -153,20 +153,6 @@ class SingleClusterPlanner(dsRef: DatasetRef, filters } - /** - * Renames Prom AST __name__ label to one based on the actual metric column of the dataset, - * if it is not the prometheus standard - */ - private def renameLabels(labels: Seq[String]): Seq[String] = - if (dsOptions.metricColumn != PromMetricLabel) { - labels map { - case PromMetricLabel => dsOptions.metricColumn - case other: String => other - } - } else { - labels - } - /** * Walk logical plan tree depth-first and generate execution plans starting from the bottom * @@ -225,10 +211,13 @@ class SingleClusterPlanner(dsRef: DatasetRef, val targetActor = pickDispatcher(stitchedLhs ++ stitchedRhs) val joined = if (lp.operator.isInstanceOf[SetOperator]) Seq(exec.SetOperatorExec(qContext, targetActor, stitchedLhs, stitchedRhs, lp.operator, - renameLabels(lp.on), renameLabels(lp.ignoring), dsOptions.metricColumn)) + LogicalPlanUtils.renameLabels(lp.on, dsOptions.metricColumn), + LogicalPlanUtils.renameLabels(lp.ignoring, dsOptions.metricColumn), dsOptions.metricColumn)) else Seq(BinaryJoinExec(qContext, targetActor, stitchedLhs, stitchedRhs, lp.operator, lp.cardinality, - renameLabels(lp.on), renameLabels(lp.ignoring), renameLabels(lp.include), dsOptions.metricColumn)) + LogicalPlanUtils.renameLabels(lp.on, dsOptions.metricColumn), + LogicalPlanUtils.renameLabels(lp.ignoring, dsOptions.metricColumn), + LogicalPlanUtils.renameLabels(lp.include, dsOptions.metricColumn), dsOptions.metricColumn)) PlanResult(joined, false) } @@ -248,8 +237,9 @@ class SingleClusterPlanner(dsRef: DatasetRef, * Starting off with solution 1 first until (2) or some other approach is decided on. */ toReduceLevel1.plans.foreach { - _.addRangeVectorTransformer(AggregateMapReduce(lp.operator, lp.params, renameLabels(lp.without), - renameLabels(lp.by))) + _.addRangeVectorTransformer(AggregateMapReduce(lp.operator, lp.params, + LogicalPlanUtils.renameLabels(lp.without, dsOptions.metricColumn), + LogicalPlanUtils.renameLabels(lp.by, dsOptions.metricColumn))) } val toReduceLevel2 = diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SinglePartitionPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SinglePartitionPlanner.scala new file mode 100644 index 0000000000..0d2c8f1cdf --- /dev/null +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SinglePartitionPlanner.scala @@ -0,0 +1,76 @@ +package filodb.coordinator.queryplanner + +import filodb.core.query.QueryContext +import filodb.query.{BinaryJoin, LabelValues, LogicalPlan, SeriesKeysByFilters, SetOperator} +import filodb.query.exec._ + +/** + * SinglePartitionPlanner is responsible for planning in situations where time series data is + * distributed across multiple clusters. + * + * @param planners map of clusters names in the local partition to their Planner objects + * @param plannerSelector a function that selects the planner name given the metric name + * + */ +class SinglePartitionPlanner(planners: Map[String, QueryPlanner], plannerSelector: String => String, + datasetMetricColumn: String) + extends QueryPlanner { + + def materialize(logicalPlan: LogicalPlan, qContext: QueryContext): ExecPlan = { + + logicalPlan match { + case lp: BinaryJoin => materializeBinaryJoin(lp, qContext) + case lp: LabelValues => materializeLabelValues(lp, qContext) + case lp: SeriesKeysByFilters => materializeSeriesKeysFilters(lp, qContext) + case _ => materializeSimpleQuery(logicalPlan, qContext) + + } + } + + /** + * Returns planner for first metric in logical plan + * If logical plan does not have metric, first planner present in planners is returned + */ + private def getPlanner(logicalPlan: LogicalPlan): QueryPlanner = { + LogicalPlanUtils.getMetricName(logicalPlan, datasetMetricColumn). + map(x => planners.get(plannerSelector(x.head)).get).getOrElse(planners.values.head) + } + + private def materializeSimpleQuery(logicalPlan: LogicalPlan, qContext: QueryContext): ExecPlan = { + getPlanner(logicalPlan).materialize(logicalPlan, qContext) + } + + private def materializeBinaryJoin(logicalPlan: BinaryJoin, qContext: QueryContext): ExecPlan = { + + val lhsExec = logicalPlan.lhs match { + case b: BinaryJoin => materializeBinaryJoin(b, qContext) + case _ => getPlanner(logicalPlan.lhs).materialize(logicalPlan.lhs, qContext) + } + + val rhsExec = logicalPlan.rhs match { + case b: BinaryJoin => materializeBinaryJoin(b, qContext) + case _ => getPlanner(logicalPlan.rhs).materialize(logicalPlan.rhs, qContext) + } + + if (logicalPlan.operator.isInstanceOf[SetOperator]) + SetOperatorExec(qContext, InProcessPlanDispatcher, Seq(lhsExec), Seq(rhsExec), logicalPlan.operator, + LogicalPlanUtils.renameLabels(logicalPlan.on, datasetMetricColumn), + LogicalPlanUtils.renameLabels(logicalPlan.ignoring, datasetMetricColumn), datasetMetricColumn) + else + BinaryJoinExec(qContext, InProcessPlanDispatcher, Seq(lhsExec), Seq(rhsExec), logicalPlan.operator, + logicalPlan.cardinality, LogicalPlanUtils.renameLabels(logicalPlan.on, datasetMetricColumn), + LogicalPlanUtils.renameLabels(logicalPlan.ignoring, datasetMetricColumn), + LogicalPlanUtils.renameLabels(logicalPlan.include, datasetMetricColumn), datasetMetricColumn) + } + + private def materializeLabelValues(logicalPlan: LogicalPlan, qContext: QueryContext) = { + val execPlans = planners.values.toList.distinct.map(_.materialize(logicalPlan, qContext)) + LabelValuesDistConcatExec(qContext, InProcessPlanDispatcher, execPlans) + } + + private def materializeSeriesKeysFilters(logicalPlan: LogicalPlan, qContext: QueryContext) = { + val execPlans = planners.values.toList.distinct.map(_.materialize(logicalPlan, qContext)) + PartKeysDistConcatExec(qContext, InProcessPlanDispatcher, execPlans) + } +} + diff --git a/query/src/test/scala/filodb/query/util/LogicalPlanUtilSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/LogicalPlanUtilsSpec.scala similarity index 85% rename from query/src/test/scala/filodb/query/util/LogicalPlanUtilSpec.scala rename to coordinator/src/test/scala/filodb.coordinator/queryplanner/LogicalPlanUtilsSpec.scala index 398f800356..6b1a2f1931 100644 --- a/query/src/test/scala/filodb/query/util/LogicalPlanUtilSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/LogicalPlanUtilsSpec.scala @@ -1,4 +1,4 @@ -package filodb.query.util +package filodb.coordinator.queryplanner import org.scalatest.{FunSpec, Matchers} @@ -17,7 +17,7 @@ class LogicalPlanUtilSpec extends FunSpec with Matchers { ColumnFilter("instance", Equals("Inst-0"))), Seq("_name_", "instance"), Some(300000), None) val periodicSeriesWithWindowing = PeriodicSeriesWithWindowing(rawSeries, 1000, 500, 5000, 100, SumOverTime) - val res = LogicalPlanUtil.getLabelValueFromLogicalPlan(periodicSeriesWithWindowing, "_name_") + val res = LogicalPlanUtils.getLabelValueFromLogicalPlan(periodicSeriesWithWindowing, "_name_") res.get.shouldEqual(Set("MetricName")) } @@ -27,7 +27,7 @@ class LogicalPlanUtilSpec extends FunSpec with Matchers { ColumnFilter("instance", In(Set("Inst-0", "Inst-1")))), Seq("_name_", "instance"), Some(300000), None) val periodicSeriesWithWindowing = PeriodicSeriesWithWindowing(rawSeries, 1000, 500, 5000, 100, SumOverTime) - val res = LogicalPlanUtil.getLabelValueFromLogicalPlan(periodicSeriesWithWindowing, "instance") + val res = LogicalPlanUtils.getLabelValueFromLogicalPlan(periodicSeriesWithWindowing, "instance") res.get.shouldEqual(Set("Inst-0", "Inst-1")) } @@ -43,7 +43,7 @@ class LogicalPlanUtilSpec extends FunSpec with Matchers { val binaryJoin = BinaryJoin(lhs, DIV, OneToOne, rhs) - val res = LogicalPlanUtil.getLabelValueFromLogicalPlan(binaryJoin, "_name_") + val res = LogicalPlanUtils.getLabelValueFromLogicalPlan(binaryJoin, "_name_") res.get.shouldEqual(Set("MetricName1")) } @@ -53,7 +53,7 @@ class LogicalPlanUtilSpec extends FunSpec with Matchers { ColumnFilter("instance", Equals("Inst-0"))), Seq("_name_", "instance"), Some(300000), None) val periodicSeriesWithWindowing = PeriodicSeriesWithWindowing(rawSeries, 1000, 500, 5000, 100, SumOverTime) - val res = LogicalPlanUtil.getLabelValueFromLogicalPlan(periodicSeriesWithWindowing, "_name") + val res = LogicalPlanUtils.getLabelValueFromLogicalPlan(periodicSeriesWithWindowing, "_name") res.isEmpty shouldEqual(true) } @@ -69,7 +69,7 @@ class LogicalPlanUtilSpec extends FunSpec with Matchers { val binaryJoin = BinaryJoin(lhs, DIV, OneToOne, rhs) - val res = LogicalPlanUtil.getLabelValueFromLogicalPlan(binaryJoin, "instance") + val res = LogicalPlanUtils.getLabelValueFromLogicalPlan(binaryJoin, "instance") res.get.shouldEqual(Set("Inst-1", "Inst-0")) } diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/SinglePartitionPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/SinglePartitionPlannerSpec.scala new file mode 100644 index 0000000000..35e0c7d9d4 --- /dev/null +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/SinglePartitionPlannerSpec.scala @@ -0,0 +1,160 @@ +package filodb.coordinator.queryplanner + +import akka.actor.ActorSystem +import akka.testkit.TestProbe +import com.typesafe.config.ConfigFactory +import monix.execution.Scheduler +import org.scalatest.{FunSpec, Matchers} + +import filodb.coordinator.ShardMapper +import filodb.core.{DatasetRef, MetricsTestData} +import filodb.core.metadata.Schemas +import filodb.core.query.{PromQlQueryParams, QueryConfig, QueryContext, QuerySession} +import filodb.core.store.ChunkSource +import filodb.prometheus.ast.TimeStepParams +import filodb.prometheus.parse.Parser +import filodb.query._ +import filodb.query.exec._ + +class SinglePartitionPlannerSpec extends FunSpec with Matchers{ + private implicit val system = ActorSystem() + private val node = TestProbe().ref + + private val localMapper = new ShardMapper(32) + for {i <- 0 until 32} localMapper.registerNode(Seq(i), node) + + private val remoteMapper = new ShardMapper(16) + for {i <- 0 until 32} localMapper.registerNode(Seq(i), node) + + private val dataset = MetricsTestData.timeseriesDataset + private val dsRef = dataset.ref + private val schemas = Schemas(dataset.schema) + + private val routingConfigString = "routing {\n buddy {\n http {\n timeout = 10.seconds\n }\n }\n}" + private val routingConfig = ConfigFactory.parseString(routingConfigString) + private val config = ConfigFactory.load("application_test.conf").getConfig("filodb.query"). + withFallback(routingConfig) + private val queryConfig = new QueryConfig(config) + + private val promQlQueryParams = PromQlQueryParams(ConfigFactory.empty, "sum(heap_usage)", 100, 1, 1000, None) + + val localPlanner = new SingleClusterPlanner(dsRef, schemas, localMapper, earliestRetainedTimestampFn = 0, queryConfig) + val remotePlanner = new SingleClusterPlanner(dsRef, schemas, remoteMapper, earliestRetainedTimestampFn = 0, + queryConfig) + + val failureProvider = new FailureProvider { + override def getFailures(datasetRef: DatasetRef, queryTimeRange: TimeRange): Seq[FailureTimeRange] = { + Seq(FailureTimeRange("local", datasetRef, + TimeRange(100, 10000), false)) + } + } + + val highAvailabilityPlanner = new HighAvailabilityPlanner(dsRef, localPlanner, failureProvider, queryConfig) + + class MockExecPlan(val name: String, val lp: LogicalPlan) extends ExecPlan { + override def queryContext: QueryContext = QueryContext() + override def children: Seq[ExecPlan] = Nil + override def submitTime: Long = 1000 + override def dataset: DatasetRef = ??? + override def dispatcher: PlanDispatcher = InProcessPlanDispatcher + override def doExecute(source: ChunkSource, querySession: QuerySession) + (implicit sched: Scheduler): ExecResult = ??? + override protected def args: String = "mock-args" + } + + val rrPlanner1 = new QueryPlanner { + override def materialize(logicalPlan: LogicalPlan, qContext: QueryContext): ExecPlan = { + new MockExecPlan("rules1", logicalPlan) + } + } + + val rrPlanner2 = new QueryPlanner { + override def materialize(logicalPlan: LogicalPlan, qContext: QueryContext): ExecPlan = { + new MockExecPlan("rules2", logicalPlan) + } + } + + val planners = Map("local" -> highAvailabilityPlanner, "rules1" -> rrPlanner1, "rules2" -> rrPlanner2) + val plannerSelector = (metricName: String) => { if (metricName.equals("rr1")) "rules1" + else if (metricName.equals("rr2")) "rules2" else "local" } + + val engine = new SinglePartitionPlanner(planners, plannerSelector, "_metric_") + + it("should generate Exec plan for simple query") { + val lp = Parser.queryToLogicalPlan("test{job = \"app\"}", 1000) + + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + execPlan.isInstanceOf[DistConcatExec] shouldEqual (true) + execPlan.children.length shouldEqual 2 + execPlan.children.head.isInstanceOf[MultiSchemaPartitionsExec] shouldEqual true + execPlan.children.head.rangeVectorTransformers.head.isInstanceOf[PeriodicSamplesMapper] shouldEqual true + } + + it("should generate BinaryJoin Exec plan") { + val lp = Parser.queryToLogicalPlan("test1{job = \"app\"} + test2{job = \"app\"}", 1000) + + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + execPlan.printTree() + execPlan.isInstanceOf[BinaryJoinExec] shouldEqual (true) + execPlan.children.foreach { l1 => + l1.isInstanceOf[DistConcatExec] shouldEqual true + l1.children.foreach { l2 => + l2.isInstanceOf[MultiSchemaPartitionsExec] shouldEqual true + l2.rangeVectorTransformers.size shouldEqual 1 + l2.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) + + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + execPlan.isInstanceOf[BinaryJoinExec] shouldEqual (true) + execPlan.asInstanceOf[BinaryJoinExec].lhs.head.isInstanceOf[BinaryJoinExec] shouldEqual true + execPlan.asInstanceOf[BinaryJoinExec].rhs.head.isInstanceOf[DistConcatExec] 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) + + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + execPlan.printTree() + execPlan.isInstanceOf[BinaryJoinExec] shouldEqual (true) + execPlan.asInstanceOf[BinaryJoinExec].rhs.head.asInstanceOf[MockExecPlan].name shouldEqual ("rules1") + execPlan.asInstanceOf[BinaryJoinExec].lhs.head.isInstanceOf[DistConcatExec] shouldEqual true + } + + it("should generate BinaryJoin Exec plan with remote cluster metrics") { + val lp = Parser.queryToLogicalPlan("rr1{job = \"app\"} + rr2{job = \"app\"}", 1000) + + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + 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") + } + + it("should generate Exec plan for Metadata query") { + val lp = Parser.metadataQueryToLogicalPlan("http_requests_total{job=\"prometheus\", method=\"GET\"}", + TimeStepParams(1000, 10, 2000)) + + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + execPlan.printTree() + execPlan.isInstanceOf[PartKeysDistConcatExec] shouldEqual (true) + execPlan.asInstanceOf[PartKeysDistConcatExec].children.length shouldEqual(3) + + // For Raw and Downsample + execPlan.asInstanceOf[PartKeysDistConcatExec].children(0).isInstanceOf[PartKeysDistConcatExec] shouldEqual true + + execPlan.asInstanceOf[PartKeysDistConcatExec].children(1).asInstanceOf[MockExecPlan].name shouldEqual ("rules1") + execPlan.asInstanceOf[PartKeysDistConcatExec].children(2).asInstanceOf[MockExecPlan].name shouldEqual ("rules2") + } + + it("should generate Exec plan for Scalar query which does not have any metric") { + val lp = Parser.queryToLogicalPlan("time()", 1000) + val execPlan = engine.materialize(lp, QueryContext(origQueryParams = promQlQueryParams)) + execPlan.isInstanceOf[TimeScalarGeneratorExec] shouldEqual true + } + +} + diff --git a/query/src/main/scala/filodb/query/util/LogicalPlanUtil.scala b/query/src/main/scala/filodb/query/util/LogicalPlanUtil.scala deleted file mode 100644 index a64dc70af8..0000000000 --- a/query/src/main/scala/filodb/query/util/LogicalPlanUtil.scala +++ /dev/null @@ -1,37 +0,0 @@ -package filodb.query.util - -import filodb.core.query.ColumnFilter -import filodb.query._ - -object LogicalPlanUtil { - - private def getLabelValueFromFilters(filters: Seq[ColumnFilter], labelName: String): Option[Set[String]] = { - val matchingFilters = filters.filter(_.column.equals(labelName)) - if (matchingFilters.isEmpty) - None - else - Some(matchingFilters.head.filter.valuesStrings.map(_.toString)) - } - - def getLabelValueFromLogicalPlan(logicalPlan: LogicalPlan, labelName: String): Option[Set[String]] = { - val labelValues = LogicalPlan.findLeafLogicalPlans(logicalPlan).flatMap { lp => - lp match { - case lp: LabelValues => lp.labelConstraints.get(labelName).map(Set(_)) - case lp: RawSeries => getLabelValueFromFilters(lp.filters, labelName) - case lp: RawChunkMeta => getLabelValueFromFilters(lp.filters, labelName) - case lp: SeriesKeysByFilters => getLabelValueFromFilters(lp.filters, labelName) - case lp: ScalarTimeBasedPlan => Nil // Plan does not have labels - case lp: ScalarFixedDoublePlan => Nil - case lp: ScalarBinaryOperation => Nil - case _ => throw new BadQueryException("Invalid logical plan") - } - } - if (labelValues.isEmpty) { - None - } else { - var res: Set[String] = Set() - // Concatenate results - Some(labelValues.foldLeft(res) { (acc, i) => i.union(acc) }) - } - } -} From 977187dabc2d0ee18fc40c9a7f1b8913decd5a03 Mon Sep 17 00:00:00 2001 From: Jackson Jeyapaul Date: Thu, 21 May 2020 10:28:40 -0700 Subject: [PATCH 13/36] misc(query): Utility to convert ColumnFilters in LogicalPlan (#751) --- .../queryplanner/LogicalPlanUtils.scala | 37 +---- .../queryplanner/LogicalPlanUtilsSpec.scala | 76 --------- .../scala/filodb.core/query/KeyFilter.scala | 7 + .../main/scala/filodb/query/LogicalPlan.scala | 50 ++++++ .../scala/filodb/query/LogicalPlanSpec.scala | 152 ++++++++++++++++++ 5 files changed, 212 insertions(+), 110 deletions(-) delete mode 100644 coordinator/src/test/scala/filodb.coordinator/queryplanner/LogicalPlanUtilsSpec.scala create mode 100644 query/src/test/scala/filodb/query/LogicalPlanSpec.scala diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala index b050b7452d..92b2cd9642 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala @@ -1,6 +1,5 @@ package filodb.coordinator.queryplanner -import filodb.core.query.ColumnFilter import filodb.prometheus.ast.Vectors.PromMetricLabel import filodb.prometheus.ast.WindowConstants import filodb.query._ @@ -124,42 +123,12 @@ object LogicalPlanUtils { } } - def getMetricName(logicalPlan: LogicalPlan, datasetMetricColumn: String): Option[Set[String]] = { - val metricName = getLabelValueFromLogicalPlan(logicalPlan, PromMetricLabel) - if (metricName.isEmpty) getLabelValueFromLogicalPlan(logicalPlan, datasetMetricColumn) + def getMetricName(logicalPlan: LogicalPlan, datasetMetricColumn: String): Option[Seq[String]] = { + val metricName = LogicalPlan.getLabelValueFromLogicalPlan(logicalPlan, PromMetricLabel) + if (metricName.isEmpty) LogicalPlan.getLabelValueFromLogicalPlan(logicalPlan, datasetMetricColumn) else metricName } - private def getLabelValueFromFilters(filters: Seq[ColumnFilter], labelName: String): Option[Set[String]] = { - val matchingFilters = filters.filter(_.column.equals(labelName)) - if (matchingFilters.isEmpty) - None - else - Some(matchingFilters.head.filter.valuesStrings.map(_.toString)) - } - - def getLabelValueFromLogicalPlan(logicalPlan: LogicalPlan, labelName: String): Option[Set[String]] = { - val labelValues = LogicalPlan.findLeafLogicalPlans(logicalPlan).flatMap { lp => - lp match { - case lp: LabelValues => lp.labelConstraints.get(labelName).map(Set(_)) - case lp: RawSeries => getLabelValueFromFilters(lp.filters, labelName) - case lp: RawChunkMeta => getLabelValueFromFilters(lp.filters, labelName) - case lp: SeriesKeysByFilters => getLabelValueFromFilters(lp.filters, labelName) - case lp: ScalarTimeBasedPlan => Nil // Plan does not have labels - case lp: ScalarFixedDoublePlan => Nil - case lp: ScalarBinaryOperation => Nil - case _ => throw new BadQueryException(s"Invalid logical plan $logicalPlan") - } - } - if (labelValues.isEmpty) { - None - } else { - val res: Set[String] = Set() - // Concatenate results - Some(labelValues.foldLeft(res) { (acc, i) => i.union(acc) }) - } - } - /** * Renames Prom AST __name__ label to one based on the actual metric column of the dataset, * if it is not the prometheus standard diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/LogicalPlanUtilsSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/LogicalPlanUtilsSpec.scala deleted file mode 100644 index 6b1a2f1931..0000000000 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/LogicalPlanUtilsSpec.scala +++ /dev/null @@ -1,76 +0,0 @@ -package filodb.coordinator.queryplanner - -import org.scalatest.{FunSpec, Matchers} - -import filodb.core.query.ColumnFilter -import filodb.core.query.Filter.{Equals, In} -import filodb.query._ -import filodb.query.BinaryOperator.DIV -import filodb.query.Cardinality.OneToOne -import filodb.query.RangeFunctionId.SumOverTime - -class LogicalPlanUtilSpec extends FunSpec with Matchers { - - it("should get MetricName from logicalPlan") { - - val rawSeries = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("_name_", Equals("MetricName")), - ColumnFilter("instance", Equals("Inst-0"))), Seq("_name_", "instance"), Some(300000), None) - val periodicSeriesWithWindowing = PeriodicSeriesWithWindowing(rawSeries, 1000, 500, 5000, 100, SumOverTime) - - val res = LogicalPlanUtils.getLabelValueFromLogicalPlan(periodicSeriesWithWindowing, "_name_") - res.get.shouldEqual(Set("MetricName")) - } - - it("should get LabelName from logicalPlan with filter In") { - - val rawSeries = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("_name_", Equals("MetricName")), - ColumnFilter("instance", In(Set("Inst-0", "Inst-1")))), Seq("_name_", "instance"), Some(300000), None) - val periodicSeriesWithWindowing = PeriodicSeriesWithWindowing(rawSeries, 1000, 500, 5000, 100, SumOverTime) - - val res = LogicalPlanUtils.getLabelValueFromLogicalPlan(periodicSeriesWithWindowing, "instance") - res.get.shouldEqual(Set("Inst-0", "Inst-1")) - } - - it("should get MetricName from BinaryJoin LogicalPlan") { - - val rawSeriesLhs = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("_name_", Equals("MetricName1")), - ColumnFilter("instance", Equals("Inst-0"))), Seq("_name_", "instance"), Some(300000), None) - val lhs = PeriodicSeries(rawSeriesLhs, 1000, 500, 50000) - - val rawSeriesRhs = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("job", Equals("MetricName2")), - ColumnFilter("instance", Equals("Inst-1"))), Seq("job", "instance"), Some(300000), None) - val rhs = PeriodicSeries(rawSeriesRhs, 1000, 500, 50000) - - val binaryJoin = BinaryJoin(lhs, DIV, OneToOne, rhs) - - val res = LogicalPlanUtils.getLabelValueFromLogicalPlan(binaryJoin, "_name_") - res.get.shouldEqual(Set("MetricName1")) - } - - it("should return None if label value is not present in logicalPlan") { - - val rawSeries = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("_name_", Equals("MetricName")), - ColumnFilter("instance", Equals("Inst-0"))), Seq("_name_", "instance"), Some(300000), None) - val periodicSeriesWithWindowing = PeriodicSeriesWithWindowing(rawSeries, 1000, 500, 5000, 100, SumOverTime) - - val res = LogicalPlanUtils.getLabelValueFromLogicalPlan(periodicSeriesWithWindowing, "_name") - res.isEmpty shouldEqual(true) - } - - it("should concatenate results from lhs and rhs for BinaryJoin LogicalPlan") { - - val rawSeriesLhs = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("_name_", Equals("MetricName1")), - ColumnFilter("instance", Equals("Inst-0"))), Seq("_name_", "instance"), Some(300000), None) - val lhs = PeriodicSeries(rawSeriesLhs, 1000, 500, 50000) - - val rawSeriesRhs = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("job", Equals("MetricName2")), - ColumnFilter("instance", Equals("Inst-1"))), Seq("job", "instance"), Some(300000), None) - val rhs = PeriodicSeries(rawSeriesRhs, 1000, 500, 50000) - - val binaryJoin = BinaryJoin(lhs, DIV, OneToOne, rhs) - - val res = LogicalPlanUtils.getLabelValueFromLogicalPlan(binaryJoin, "instance") - res.get.shouldEqual(Set("Inst-1", "Inst-0")) - } - -} diff --git a/core/src/main/scala/filodb.core/query/KeyFilter.scala b/core/src/main/scala/filodb.core/query/KeyFilter.scala index 4c07934cff..b10a59533f 100644 --- a/core/src/main/scala/filodb.core/query/KeyFilter.scala +++ b/core/src/main/scala/filodb.core/query/KeyFilter.scala @@ -7,17 +7,20 @@ import filodb.memory.format.{SingleValueRowReader, UTF8Wrapper, ZeroCopyUTF8Stri sealed trait Filter { def filterFunc: Any => Boolean + def operatorString: String def valuesStrings : Set[Any] } object Filter { final case class Equals(value: Any) extends Filter { val filterFunc: Any => Boolean = (item: Any) => value.equals(item) + val operatorString: String = "=" def valuesStrings: Set[Any] = Set(value) } final case class In(values: Set[Any]) extends Filter { val filterFunc: (Any) => Boolean = (item: Any) => values.contains(item) + val operatorString: String = "in" def valuesStrings: Set[Any] = values } @@ -25,23 +28,27 @@ object Filter { private val leftFunc = left.filterFunc private val rightFunc = right.filterFunc val filterFunc: (Any) => Boolean = (item: Any) => leftFunc(item) && rightFunc(item) + val operatorString: String = "&&" def valuesStrings: Set[Any] = left.valuesStrings.union(right.valuesStrings) } final case class NotEquals(value: Any) extends Filter { val filterFunc: (Any) => Boolean = (item: Any) => !value.equals(item) + val operatorString: String = "!=" def valuesStrings: Set[Any] = Set(value) } final case class EqualsRegex(value: Any) extends Filter { val pattern = Pattern.compile(value.toString, Pattern.DOTALL) val filterFunc: (Any) => Boolean = (item: Any) => pattern.matcher(item.toString).matches() + val operatorString: String = "=~" def valuesStrings: Set[Any] = Set(value) } final case class NotEqualsRegex(value: Any) extends Filter { val pattern = Pattern.compile(value.toString, Pattern.DOTALL) val filterFunc: (Any) => Boolean = (item: Any) => !pattern.matcher(item.toString).matches() + val operatorString: String = "!~" def valuesStrings: Set[Any] = Set(value) } diff --git a/query/src/main/scala/filodb/query/LogicalPlan.scala b/query/src/main/scala/filodb/query/LogicalPlan.scala index d67354f91a..f449552d3e 100644 --- a/query/src/main/scala/filodb/query/LogicalPlan.scala +++ b/query/src/main/scala/filodb/query/LogicalPlan.scala @@ -336,6 +336,10 @@ case class ApplyAbsentFunction(vectors: PeriodicSeriesPlan, override def endMs: Long = vectors.endMs } +case class LabelValueOperator(columnName: String, value: Seq[String], operator: String) + +case class LabelValueOperatorGroup(labelValueOperators: Seq[LabelValueOperator]) + object LogicalPlan { /** * Get leaf Logical Plans @@ -347,5 +351,51 @@ object LogicalPlan { case _ => Seq(logicalPlan) } } + + def getLabelValueFromLogicalPlan(logicalPlan: LogicalPlan, labelName: String): Option[Seq[String]] = { + getLabelValueFromLogicalPlan(getLabelValueOperatorsFromLogicalPlan(logicalPlan), labelName) + } + + def getLabelValueFromLogicalPlan(labelValues: Option[Seq[LabelValueOperatorGroup]], + labelName: String): Option[Seq[String]] = { + labelValues match { + case None => None + case _ => labelValues.get.flatMap(group => + group.labelValueOperators.flatMap(lops => { + lops.columnName.equals(labelName) match { + case true => lops.value + case false => Seq() + } + })).distinct match { + case Nil => None + case lVFilters: Seq[String] => Some(lVFilters) + } + } + } + + private def getLabelValueOpsFromFilters(filters: Seq[ColumnFilter]): Option[LabelValueOperatorGroup] = { + Some(LabelValueOperatorGroup(filters.map(cf => LabelValueOperator(cf.column, + cf.filter.valuesStrings.map(_.toString).toSeq.sorted, cf.filter.operatorString)))) + } + + def getLabelValueOperatorsFromLogicalPlan(logicalPlan: LogicalPlan): Option[Seq[LabelValueOperatorGroup]] = { + LogicalPlan.findLeafLogicalPlans(logicalPlan).flatMap { lp => + lp match { + case lp: LabelValues => Some( + LabelValueOperatorGroup( + lp.labelConstraints.map(lbc => LabelValueOperator(lbc._1, Seq(lbc._2), "=")).toSeq)) + case lp: RawSeries => getLabelValueOpsFromFilters(lp.filters) + case lp: RawChunkMeta => getLabelValueOpsFromFilters(lp.filters) + case lp: SeriesKeysByFilters => getLabelValueOpsFromFilters(lp.filters) + case _: ScalarTimeBasedPlan => Nil // Plan does not have labels + case _: ScalarFixedDoublePlan => Nil + case _: ScalarBinaryOperation => Nil + case _ => throw new BadQueryException(s"Invalid logical plan $logicalPlan") + } + } match { + case Nil => None + case groupSeq: Seq[LabelValueOperatorGroup] => Some(groupSeq) + } + } } //scalastyle:on number.of.types \ No newline at end of file diff --git a/query/src/test/scala/filodb/query/LogicalPlanSpec.scala b/query/src/test/scala/filodb/query/LogicalPlanSpec.scala new file mode 100644 index 0000000000..57e9fd0f6f --- /dev/null +++ b/query/src/test/scala/filodb/query/LogicalPlanSpec.scala @@ -0,0 +1,152 @@ +package filodb.query + +import filodb.core.query.{ColumnFilter, RangeParams} +import filodb.core.query.Filter.{Equals, EqualsRegex, In, NotEquals, NotEqualsRegex} +import filodb.query.BinaryOperator.DIV +import filodb.query.Cardinality.OneToOne +import filodb.query.RangeFunctionId.SumOverTime +import org.scalatest.{FunSpec, Matchers} + +class LogicalPlanSpec extends FunSpec with Matchers { + + it("should get labelValueOps from logicalPlan") { + + val rawSeries = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("_name_", Equals("MetricName")), + ColumnFilter("instance", NotEquals("Inst-0"))), Seq("_name_", "instance"), Some(300000), None) + val periodicSeriesWithWindowing = PeriodicSeriesWithWindowing(rawSeries, 1000, 500, 5000, 100, SumOverTime) + + val res = LogicalPlan.getLabelValueOperatorsFromLogicalPlan(periodicSeriesWithWindowing) + res.get.size.shouldEqual(1) + res.get(0).labelValueOperators.size.shouldEqual(2) + res.get(0).labelValueOperators(0).columnName.shouldEqual("_name_") + res.get(0).labelValueOperators(0).value.shouldEqual(Seq("MetricName")) + res.get(0).labelValueOperators(0).operator.shouldEqual("=") + res.get(0).labelValueOperators(1).columnName.shouldEqual("instance") + res.get(0).labelValueOperators(1).value.shouldEqual(Seq("Inst-0")) + res.get(0).labelValueOperators(1).operator.shouldEqual("!=") + } + + it("should get labelValueOps from logicalPlan with filter In") { + + val rawSeries = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("_name_", Equals("MetricName")), + ColumnFilter("instance", In(Set("Inst-1", "Inst-0")))), Seq("_name_", "instance"), Some(300000), None) + val periodicSeriesWithWindowing = PeriodicSeriesWithWindowing(rawSeries, 1000, 500, 5000, 100, SumOverTime) + + val res = LogicalPlan.getLabelValueOperatorsFromLogicalPlan(periodicSeriesWithWindowing) + res.get.size.shouldEqual(1) + res.get(0).labelValueOperators.size.shouldEqual(2) + res.get(0).labelValueOperators(0).columnName.shouldEqual("_name_") + res.get(0).labelValueOperators(0).value.shouldEqual(Seq("MetricName")) + res.get(0).labelValueOperators(0).operator.shouldEqual("=") + res.get(0).labelValueOperators(1).columnName.shouldEqual("instance") + res.get(0).labelValueOperators(1).value.shouldEqual(Seq("Inst-0", "Inst-1")) + res.get(0).labelValueOperators(1).operator.shouldEqual("in") + } + + it("should get labelValueOps from BinaryJoin LogicalPlan") { + + val rawSeriesLhs = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("_name_", Equals("MetricName1")), + ColumnFilter("instance", EqualsRegex("Inst-0"))), Seq("_name_", "instance"), Some(300000), None) + val lhs = PeriodicSeries(rawSeriesLhs, 1000, 500, 50000) + + val rawSeriesRhs = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("job", Equals("MetricName2")), + ColumnFilter("instance", NotEqualsRegex("Inst-1"))), Seq("job", "instance"), Some(300000), None) + val rhs = PeriodicSeries(rawSeriesRhs, 1000, 500, 50000) + + val binaryJoin = BinaryJoin(lhs, DIV, OneToOne, rhs) + + val res = LogicalPlan.getLabelValueOperatorsFromLogicalPlan(binaryJoin) + res.get.size.shouldEqual(2) + res.get(0).labelValueOperators.size.shouldEqual(2) + res.get(0).labelValueOperators(0).columnName.shouldEqual("_name_") + res.get(0).labelValueOperators(0).value.shouldEqual(Seq("MetricName1")) + res.get(0).labelValueOperators(0).operator.shouldEqual("=") + res.get(0).labelValueOperators(1).columnName.shouldEqual("instance") + res.get(0).labelValueOperators(1).value.shouldEqual(Seq("Inst-0")) + res.get(0).labelValueOperators(1).operator.shouldEqual("=~") + res.get(1).labelValueOperators.size.shouldEqual(2) + res.get(1).labelValueOperators(0).columnName.shouldEqual("job") + res.get(1).labelValueOperators(0).value.shouldEqual(Seq("MetricName2")) + res.get(1).labelValueOperators(0).operator.shouldEqual("=") + res.get(1).labelValueOperators(1).columnName.shouldEqual("instance") + res.get(1).labelValueOperators(1).value.shouldEqual(Seq("Inst-1")) + res.get(1).labelValueOperators(1).operator.shouldEqual("!~") + } + + it("should get labelValueOps fail for scalar logicalPlan") { + val periodicSeriesWithWindowing = ScalarTimeBasedPlan(ScalarFunctionId.Year, RangeParams(1000, 500, 5000)) + val res = LogicalPlan.getLabelValueOperatorsFromLogicalPlan(periodicSeriesWithWindowing) + res.isEmpty should be (true) + intercept[NoSuchElementException] { res.get } + } + + it("should get MetricName fail for scalar logicalPlan") { + val periodicSeriesWithWindowing = ScalarTimeBasedPlan(ScalarFunctionId.Year, RangeParams(1000, 500, 5000)) + val res = LogicalPlan.getLabelValueFromLogicalPlan(periodicSeriesWithWindowing, "_name_") + res.isEmpty should be (true) + intercept[NoSuchElementException] { res.get } + } + + it("should get MetricName from logicalPlan") { + + val rawSeries = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("_name_", Equals("MetricName")), + ColumnFilter("instance", Equals("Inst-0"))), Seq("_name_", "instance"), Some(300000), None) + val periodicSeriesWithWindowing = PeriodicSeriesWithWindowing(rawSeries, 1000, 500, 5000, 100, SumOverTime) + + val res = LogicalPlan.getLabelValueFromLogicalPlan(periodicSeriesWithWindowing, "_name_") + res.get.shouldEqual(Seq("MetricName")) + } + + it("should get LabelName from logicalPlan with filter In") { + + val rawSeries = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("_name_", Equals("MetricName")), + ColumnFilter("instance", In(Set("Inst-0", "Inst-1")))), Seq("_name_", "instance"), Some(300000), None) + val periodicSeriesWithWindowing = PeriodicSeriesWithWindowing(rawSeries, 1000, 500, 5000, 100, SumOverTime) + + val res = LogicalPlan.getLabelValueFromLogicalPlan(periodicSeriesWithWindowing, "instance") + res.get.shouldEqual(Seq("Inst-0", "Inst-1")) + } + + it("should get MetricName from BinaryJoin LogicalPlan") { + + val rawSeriesLhs = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("_name_", Equals("MetricName1")), + ColumnFilter("instance", Equals("Inst-0"))), Seq("_name_", "instance"), Some(300000), None) + val lhs = PeriodicSeries(rawSeriesLhs, 1000, 500, 50000) + + val rawSeriesRhs = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("job", Equals("MetricName2")), + ColumnFilter("instance", Equals("Inst-1"))), Seq("job", "instance"), Some(300000), None) + val rhs = PeriodicSeries(rawSeriesRhs, 1000, 500, 50000) + + val binaryJoin = BinaryJoin(lhs, DIV, OneToOne, rhs) + + val res = LogicalPlan.getLabelValueFromLogicalPlan(binaryJoin, "_name_") + res.get.shouldEqual(Seq("MetricName1")) + } + + it("should return None if label value is not present in logicalPlan") { + + val rawSeries = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("_name_", Equals("MetricName")), + ColumnFilter("instance", Equals("Inst-0"))), Seq("_name_", "instance"), Some(300000), None) + val periodicSeriesWithWindowing = PeriodicSeriesWithWindowing(rawSeries, 1000, 500, 5000, 100, SumOverTime) + + val res = LogicalPlan.getLabelValueFromLogicalPlan(periodicSeriesWithWindowing, "_name") + res.isEmpty shouldEqual(true) + } + + it("should concatenate results from lhs and rhs for BinaryJoin LogicalPlan") { + + val rawSeriesLhs = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("_name_", Equals("MetricName1")), + ColumnFilter("instance", Equals("Inst-0"))), Seq("_name_", "instance"), Some(300000), None) + val lhs = PeriodicSeries(rawSeriesLhs, 1000, 500, 50000) + + val rawSeriesRhs = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("job", Equals("MetricName2")), + ColumnFilter("instance", Equals("Inst-1"))), Seq("job", "instance"), Some(300000), None) + val rhs = PeriodicSeries(rawSeriesRhs, 1000, 500, 50000) + + val binaryJoin = BinaryJoin(lhs, DIV, OneToOne, rhs) + + val res = LogicalPlan.getLabelValueFromLogicalPlan(binaryJoin, "instance") + res.get.shouldEqual(Seq("Inst-0", "Inst-1")) + } + +} From aaa39316dde8ed2392a370dfbfd7954b64968918 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Thu, 21 May 2020 17:22:44 -0700 Subject: [PATCH 14/36] feat(sparkJobs): If tracing downsampler, log chunks read from cassandra (#756) (#757) --- .../filodb/downsampler/chunk/BatchDownsampler.scala | 12 ++++++++++++ 1 file changed, 12 insertions(+) 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 980beef390..ca69d75bc0 100644 --- a/spark-jobs/src/main/scala/filodb/downsampler/chunk/BatchDownsampler.scala +++ b/spark-jobs/src/main/scala/filodb/downsampler/chunk/BatchDownsampler.scala @@ -259,6 +259,8 @@ class BatchDownsampler(settings: DownsamplerSettings) extends Instance with Seri val timestampCol = 0 val rawChunksets = rawPartToDownsample.infos(AllChunkScan) + require(downsamplers.size > 1, s"Number of downsamplers for ${rawPartToDownsample.stringPartition} should be > 1") + // for each chunk while (rawChunksets.hasNext) { val chunkset = rawChunksets.nextInfoReader @@ -270,6 +272,16 @@ class BatchDownsampler(settings: DownsamplerSettings) extends Instance with Seri val tsAcc = chunkset.vectorAccessor(timestampCol) val tsReader = rawPartToDownsample.chunkReader(timestampCol, tsAcc, tsPtr).asLongReader + if (shouldTrace) { + downsamplers.zipWithIndex.foreach { case (d, i) => + val ptr = chunkset.vectorAddress(i) + val acc = chunkset.vectorAccessor(i) + val reader = rawPartToDownsample.chunkReader(i, acc, ptr) + DownsamplerContext.dsLogger.info(s"Hex Vectors: Col $i for ${rawPartToDownsample.stringPartition} uses " + + s"downsampler ${d.encoded} vector=${reader.toHexString(acc, ptr)}RemoveEOL") + } + } + val startRow = tsReader.binarySearch(tsAcc, tsPtr, userTimeStart) & 0x7fffffff // userTimeEndExclusive-1 since ceilingIndex does an inclusive check val endRow = Math.min(tsReader.ceilingIndex(tsAcc, tsPtr, userTimeEndExclusive - 1), chunkset.numRows - 1) From 9ea3cd7b1a3409278374d60ad00610e2e48cdec8 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Thu, 21 May 2020 17:56:44 -0700 Subject: [PATCH 15/36] debug(core): Debug Full ODP code path (#758) --- .../memstore/OnDemandPagingShard.scala | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala b/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala index 0eb4dd1567..1d1dfd3493 100644 --- a/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala +++ b/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala @@ -133,6 +133,13 @@ TimeSeriesShard(ref, schemas, storeConfig, shardNum, bufferMemoryManager, rawSto } else { Observable.fromTask(odpPartTask(partIdsNotInMemory, partKeyBytesToPage, pagingMethods, partLookupRes.chunkMethod)).flatMap { odpParts => + // which thread are we on right now ? + logger.debug(s"Finished creating full ODP partitions ${odpParts.map(_.partID)}") + if(logger.underlying.isDebugEnabled) { + partKeyBytesToPage.zip(pagingMethods).foreach { case (pk, method) => + logger.debug(s"Paging in chunks for partId=${getPartition(pk).get.partID} chunkMethod=$method") + } + } if (partKeyBytesToPage.nonEmpty) { val span = startODPSpan() Observable.fromIterable(partKeyBytesToPage.zip(pagingMethods)) @@ -142,6 +149,7 @@ TimeSeriesShard(ref, schemas, storeConfig, shardNum, bufferMemoryManager, rawSto .asyncBoundary(strategy) // This is needed so future computations happen in a different thread .defaultIfEmpty(getPartition(partBytes).get) .headL + // headL since we are fetching a SinglePartition above } .doOnTerminate(ex => span.finish()) } else { @@ -161,11 +169,12 @@ TimeSeriesShard(ref, schemas, storeConfig, shardNum, bufferMemoryManager, rawSto private def odpPartTask(partIdsNotInMemory: Buffer[Int], partKeyBytesToPage: ArrayBuffer[Array[Byte]], methods: ArrayBuffer[ChunkScanMethod], chunkMethod: ChunkScanMethod) = if (partIdsNotInMemory.nonEmpty) { - createODPPartitionsTask(partIdsNotInMemory, { case (bytes, offset) => + createODPPartitionsTask(partIdsNotInMemory, { case (pId, bytes, offset) => val partKeyBytes = if (offset == UnsafeUtils.arayOffset) bytes else BinaryRegionLarge.asNewByteArray(bytes, offset) partKeyBytesToPage += partKeyBytes methods += chunkMethod + logger.debug(s"Finished creating part for full odp. Now need to page partId=$pId chunkMethod=$chunkMethod") shardStats.partitionsRestored.increment() }).executeOn(ingestSched).asyncBoundary // asyncBoundary above will cause subsequent map operations to run on designated scheduler for task or observable @@ -179,7 +188,7 @@ TimeSeriesShard(ref, schemas, storeConfig, shardNum, bufferMemoryManager, rawSto * to create TSPartitions for partIDs found in Lucene but not in in-memory data structures * It runs in ingestion thread so it can correctly verify which ones to actually create or not */ - private def createODPPartitionsTask(partIDs: Buffer[Int], callback: (Array[Byte], Int) => Unit): + private def createODPPartitionsTask(partIDs: Buffer[Int], callback: (Int, Array[Byte], Int) => Unit): Task[Seq[TimeSeriesPartition]] = Task { assertThreadName(IngestSchedName) require(partIDs.nonEmpty) @@ -202,7 +211,7 @@ TimeSeriesShard(ref, schemas, storeConfig, shardNum, bufferMemoryManager, rawSto } finally { partSetLock.unlockWrite(stamp) } - callback(partKeyBytesRef.bytes, unsafeKeyOffset) + callback(part.partID, partKeyBytesRef.bytes, unsafeKeyOffset) part } // create the partition and update data structures (but no need to add to Lucene!) From 5a5bcb1574eff10f1cb70b1911892a2fe68c66e0 Mon Sep 17 00:00:00 2001 From: "Brian S. O'Neill" Date: Wed, 27 May 2020 11:42:38 -0700 Subject: [PATCH 16/36] fix(core): Avoid reclamation when queries are running. (#759) --- .../memstore/TimeSeriesShard.scala | 77 ++--------- .../scala/filodb.memory/BlockManager.scala | 126 +++++++++++++++++- 2 files changed, 131 insertions(+), 72 deletions(-) diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala index 49bc00cce2..00b0dc38e4 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala @@ -105,12 +105,6 @@ class TimeSeriesShardStats(dataset: DatasetRef, shardNum: Int) { * receiving them, assuming that the clocks are in sync. */ val ingestionClockDelay = Kamon.gauge("ingestion-clock-delay").withTags(TagSet.from(tags)) - - /** - * How much time the ingestion thread was potentially stalled while attempting to ensure - * free space. Unit is milliseconds. - */ - val ingestionHeadroomStall = Kamon.gauge("ingestion-headroom-stall").withTags(TagSet.from(tags)) } object TimeSeriesShard { @@ -313,21 +307,21 @@ class TimeSeriesShard(val ref: DatasetRef, // occurs in the common case, when there isn't any contention reading from partSet. private[memstore] final val partSetLock = new StampedLock + // The off-heap block store used for encoded chunks + private val shardTags = Map("dataset" -> ref.dataset, "shard" -> shardNum.toString) + private val blockStore = new PageAlignedBlockManager(blockMemorySize, shardStats.memoryStats, reclaimListener, + storeConfig.numPagesPerBlock) + private val blockFactoryPool = new BlockMemFactoryPool(blockStore, maxMetaSize, shardTags) + /** * Lock that protects chunks from being reclaimed from Memstore. * This is needed to prevent races between ODP queries and reclaims. */ - private[memstore] final val reclaimLock = new StampedLock + private[memstore] final val reclaimLock = blockStore.reclaimLock - // Requires reclaimLock. + // Requires blockStore. startHeadroomTask(ingestSched) - // The off-heap block store used for encoded chunks - private val shardTags = Map("dataset" -> ref.dataset, "shard" -> shardNum.toString) - private val blockStore = new PageAlignedBlockManager(blockMemorySize, shardStats.memoryStats, reclaimListener, - storeConfig.numPagesPerBlock) - private val blockFactoryPool = new BlockMemFactoryPool(blockStore, maxMetaSize, shardTags) - // 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, @@ -1482,63 +1476,10 @@ class TimeSeriesShard(val ref: DatasetRef, private def startHeadroomTask(sched: Scheduler): Unit = { sched.scheduleWithFixedDelay(1, 1, TimeUnit.MINUTES, new Runnable { - def run() = ensureFreeBlocks + def run() = blockStore.ensureHeadroom(storeConfig.ensureHeadroomPercent) }) } - /** - * Expected to be called via a background task, to periodically ensure that enough blocks - * are free for new allocations. This helps prevent ODP activity from reclaiming immediately - * from itself. - */ - private def ensureFreeBlocks(): Unit = { - val start = System.currentTimeMillis() - val stamp = tryExclusiveReclaimLock() - if (stamp == 0) { - logger.warn(s"shard=$shardNum: ensureFreeBlocks timed out: ${reclaimLock}") - } else { - val numFree = try { - blockStore.ensureFreePercent(storeConfig.ensureHeadroomPercent) - } finally { - reclaimLock.unlockWrite(stamp) - } - val numBytes = numFree * blockStore.blockSizeInBytes - logger.debug(s"shard=$shardNum: ensureFreeBlocks: $numFree ($numBytes bytes)") - } - val stall = System.currentTimeMillis() - start - shardStats.ingestionHeadroomStall.update(stall) - } - - private def tryExclusiveReclaimLock(): Long = { - // Attempting to acquire the exclusive lock must wait for concurrent queries to finish, but - // waiting will also stall new queries from starting. To protect against this, attempt with - // a timeout to let any stalled queries through. To prevent starvation of the exclusive - // lock attempt, increase the timeout each time, but eventually give up. The reason why - // waiting for an exclusive lock causes this problem is that the thread must enqueue itself - // into the lock as a waiter, and all new shared requests must wait their turn. The risk - // with timing out is that if there's a continuous stream of long running queries (more than - // one second), then the exclusive lock will never be acqiured, and then ensureFreeBlocks - // won't be able to do its job. The timeout settings might need to be adjusted in that case. - // Perhaps the timeout should increase automatically if ensureFreeBlocks failed the last time? - // This isn't safe to do until we gain higher confidence that the shared lock is always - // released by queries. - - var timeout = 1; - while (true) { - val stamp = reclaimLock.tryWriteLock(timeout, TimeUnit.MILLISECONDS) - if (stamp != 0) { - return stamp - } - timeout <<= 1 - if (timeout > 1024) { - // Give up after waiting (in total) a little over 2 seconds. - return 0 - } - Thread.`yield`() - } - 0 // never reached, but scala compiler complains otherwise - } - /** * Please use this for testing only - reclaims ALL used offheap blocks. Maybe you are trying to test * on demand paging. diff --git a/memory/src/main/scala/filodb.memory/BlockManager.scala b/memory/src/main/scala/filodb.memory/BlockManager.scala index 688fef58fe..f29a6297fa 100644 --- a/memory/src/main/scala/filodb.memory/BlockManager.scala +++ b/memory/src/main/scala/filodb.memory/BlockManager.scala @@ -2,7 +2,9 @@ package filodb.memory import java.lang.{Long => jLong} import java.util +import java.util.concurrent.TimeUnit import java.util.concurrent.locks.ReentrantLock +import java.util.concurrent.locks.StampedLock import com.kenai.jffi.{MemoryIO, PageManager} import com.typesafe.scalalogging.StrictLogging @@ -104,6 +106,18 @@ class MemoryStats(tags: Map[String, String]) { val timeOrderedBlocksReclaimedMetric = Kamon.counter("blockstore-time-ordered-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 + * free space. Unit is milliseconds. + */ + val blockHeadroomStall = Kamon.gauge("blockstore-headroom-stall").withTags(TagSet.from(tags)) + + /** + * How much time a thread was stalled while attempting to acquire the reclaim lock. + * Unit is milliseconds. + */ + val blockReclaimStall = Kamon.gauge("blockstore-reclaim-stall").withTags(TagSet.from(tags)) } final case class ReclaimEvent(block: Block, reclaimTime: Long, oldOwner: Option[BlockMemFactory], remaining: Long) @@ -142,6 +156,9 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, protected val lock = new ReentrantLock() + // Acquired when reclaiming on demand. Acquire shared lock to prevent block reclamation. + final val reclaimLock = new StampedLock + override def blockSizeInBytes: Long = PageManager.getInstance().pageSize() * numPagesPerBlock def availablePreAllocated: Long = numFreeBlocks * blockSizeInBytes @@ -184,14 +201,14 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, override def requestBlocks(memorySize: Long, bucketTime: Option[Long], bmf: Option[BlockMemFactory] = None): Seq[Block] = { + val num: Int = Math.ceil(memorySize / blockSizeInBytes).toInt + stats.requestedBlocksMetric.increment(num) + lock.lock() try { - val num: Int = Math.ceil(memorySize / blockSizeInBytes).toInt - stats.requestedBlocksMetric.increment(num) - if (freeBlocks.size < num) { if (bucketTime.isEmpty) { - tryReclaim(num) + tryReclaimOnDemand(num) } else { val msg = s"Unable to allocate time ordered block(s) without forcing a reclamation: " + s"num_blocks=$num num_bytes=$memorySize freeBlocks=${freeBlocks.size}" @@ -217,6 +234,107 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, } } + /** + * Internal variant of the tryReclaim method which is called when blocks are requested, but + * none are available. Instead of blindly reclaiming blocks, it attempts to exclusively + * acquire the reclaim lock. By doing so, it avoids reclaiming blocks which are currently + * being accessed. To work properly, all threads which require this protection must hold the + * shared reclaimLock. To prevent indefinite stalls, this method times out lock acquisition, + * logs an error, and then reclaims anyhow. + * + * This method must be called with the primary lock object held. To avoid deadlock, this + * method releases and re-acquires the lock. + */ + private def tryReclaimOnDemand(num: Int): Unit = { + lock.unlock() + var stamp: Long = 0 + try { + val start = System.currentTimeMillis() + // Give up after waiting (in total) a little over 16 seconds. + stamp = tryExclusiveReclaimLock(8192) + + if (stamp == 0) { + // Don't stall ingestion forever. Some queries might return invalid results because + // the lock isn't held. If the lock state is broken, then ingestion is really stuck + // and the node must be restarted. Queries should always release the lock. + logger.error(s"Lock for BlockManager.tryReclaimOnDemand timed out: ${reclaimLock}") + } + + val stall = System.currentTimeMillis() - start + stats.blockReclaimStall.update(stall) + } finally { + lock.lock() + } + + try { + if (numFreeBlocks < num) { // double check since lock was released + tryReclaim(num) + } + } finally { + if (stamp != 0) { + reclaimLock.unlockWrite(stamp) + } + } + } + + private def tryExclusiveReclaimLock(finalTimeoutMillis: Int): Long = { + // Attempting to acquire the exclusive lock must wait for concurrent queries to finish, but + // waiting will also stall new queries from starting. To protect against this, attempt with + // a timeout to let any stalled queries through. To prevent starvation of the exclusive + // lock attempt, increase the timeout each time, but eventually give up. The reason why + // waiting for an exclusive lock causes this problem is that the thread must enqueue itself + // into the lock as a waiter, and all new shared requests must wait their turn. The risk + // with timing out is that if there's a continuous stream of long running queries (more than + // one second), then the exclusive lock will never be acqiured, and then ensureFreeBlocks + // won't be able to do its job. The timeout settings might need to be adjusted in that case. + // Perhaps the timeout should increase automatically if ensureFreeBlocks failed the last time? + // This isn't safe to do until we gain higher confidence that the shared lock is always + // released by queries. + + var timeout = 1; + while (true) { + val stamp = reclaimLock.tryWriteLock(timeout, TimeUnit.MILLISECONDS) + if (stamp != 0) { + return stamp + } + timeout <<= 1 + if (timeout > finalTimeoutMillis) { + return 0 + } + Thread.`yield`() + } + 0 // never reached, but scala compiler complains otherwise + } + + /** + * Expected to be called via a background task, to periodically ensure that enough blocks + * are free for new allocations. This helps prevent ODP activity from reclaiming immediately + * from itself. + * + * @param pct percentage: 0.0 to 100.0 + */ + def ensureHeadroom(pct: Double): Int = { + var numFree: Int = 0 + val start = System.currentTimeMillis() + // Give up after waiting (in total) a little over 2 seconds. + val stamp = tryExclusiveReclaimLock(1024) + if (stamp == 0) { + logger.warn(s"Lock for BlockManager.ensureFreePercent timed out: ${reclaimLock}") + numFree = numFreeBlocks + } else { + try { + numFree = ensureFreePercent(pct) + } finally { + reclaimLock.unlockWrite(stamp) + } + val numBytes = numFree * blockSizeInBytes + logger.debug(s"BlockManager.ensureFreePercent numFree: $numFree ($numBytes bytes)") + } + val stall = System.currentTimeMillis() - start + stats.blockHeadroomStall.update(stall) + numFree + } + override def ensureFreeBlocks(num: Int): Int = { lock.lock() try { From c91b2b87bad04ee7b4c8dd3ccca573cb394685d1 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Wed, 27 May 2020 15:21:59 -0700 Subject: [PATCH 17/36] fix(jmh): Fix PartKeyIndexBenchmark (#762) --- .../filodb.jmh/PartKeyIndexBenchmark.scala | 18 +++++++++++++----- 1 file changed, 13 insertions(+), 5 deletions(-) diff --git a/jmh/src/main/scala/filodb.jmh/PartKeyIndexBenchmark.scala b/jmh/src/main/scala/filodb.jmh/PartKeyIndexBenchmark.scala index 2f7ce8a750..59d77b46e7 100644 --- a/jmh/src/main/scala/filodb.jmh/PartKeyIndexBenchmark.scala +++ b/jmh/src/main/scala/filodb.jmh/PartKeyIndexBenchmark.scala @@ -12,7 +12,7 @@ import scalaxy.loops._ import filodb.core.DatasetRef import filodb.core.binaryrecord2.RecordBuilder import filodb.core.memstore.PartKeyLuceneIndex -import filodb.core.metadata.Schemas.promCounter +import filodb.core.metadata.Schemas.untyped import filodb.core.query.{ColumnFilter, Filter} import filodb.memory.{BinaryRegionConsumer, MemFactory} import filodb.timeseries.TestTimeseriesProducer @@ -23,17 +23,25 @@ class PartKeyIndexBenchmark { org.slf4j.LoggerFactory.getLogger("filodb").asInstanceOf[Logger].setLevel(Level.ERROR) val ref = DatasetRef("prometheus") - val partKeyIndex = new PartKeyLuceneIndex(ref, promCounter.partition, 0, 1.hour) + val partKeyIndex = new PartKeyLuceneIndex(ref, untyped.partition, 0, 1.hour) val numSeries = 1000000 - val partKeyData = TestTimeseriesProducer.timeSeriesData(0, numSeries) take numSeries + val ingestBuilder = new RecordBuilder(MemFactory.onHeapFactory) + val untypedData = TestTimeseriesProducer.timeSeriesData(0, numSeries) take numSeries + untypedData.foreach(_.addToBuilder(ingestBuilder)) + val partKeyBuilder = new RecordBuilder(MemFactory.onHeapFactory) - partKeyData.foreach(_.addToBuilder(partKeyBuilder)) + + val converter = new BinaryRegionConsumer { + def onNext(base: Any, offset: Long): Unit = untyped.comparator.buildPartKeyFromIngest(base, offset, partKeyBuilder) + } + // Build part keys from the ingestion records + ingestBuilder.allContainers.head.consumeRecords(converter) var partId = 1 val now = System.currentTimeMillis() val consumer = new BinaryRegionConsumer { def onNext(base: Any, offset: Long): Unit = { - val partKey = promCounter.partition.binSchema.asByteArray(base, offset) + val partKey = untyped.partition.binSchema.asByteArray(base, offset) partKeyIndex.addPartKey(partKey, partId, now)() partId += 1 } From 07bcc3eb682a4bacfdbc1ae17f6379d06679d3fb Mon Sep 17 00:00:00 2001 From: "Brian S. O'Neill" Date: Wed, 27 May 2020 19:43:03 -0700 Subject: [PATCH 18/36] feat(core): Reduce block size to 400KB. (#760) --- .../scala/filodb.core/store/IngestionConfig.scala | 6 +++++- .../main/scala/filodb.memory/BlockDetective.scala | 4 ++-- .../main/scala/filodb.memory/BlockManager.scala | 14 +++++++------- 3 files changed, 14 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/filodb.core/store/IngestionConfig.scala b/core/src/main/scala/filodb.core/store/IngestionConfig.scala index 9c77b82156..6deedbd3d2 100644 --- a/core/src/main/scala/filodb.core/store/IngestionConfig.scala +++ b/core/src/main/scala/filodb.core/store/IngestionConfig.scala @@ -72,6 +72,10 @@ object StoreConfig { // NOTE: there are no defaults for flush interval and shard memory, those should be explicitly calculated // default max-data-per-shard-query was calculated as follows: // 750k TsPartitions * 48 chunksets queried * 2kb per chunkset / 256 shards = 280MB + + // The num-block-pages setting when multiplied by the page size (4KB) defines the + // BlockManager block size. When num-block-pages is 100, the effective block size is 400KB. + val defaults = ConfigFactory.parseString(""" |disk-time-to-live = 3 days |demand-paged-chunk-retention-period = 72 hours @@ -82,7 +86,7 @@ object StoreConfig { |max-buffer-pool-size = 10000 |num-partitions-to-evict = 1000 |groups-per-shard = 60 - |num-block-pages = 1000 + |num-block-pages = 100 |failure-retries = 3 |retry-delay = 15 seconds |part-index-flush-max-delay = 60 seconds diff --git a/memory/src/main/scala/filodb.memory/BlockDetective.scala b/memory/src/main/scala/filodb.memory/BlockDetective.scala index be5ce818cf..0126a2c3df 100644 --- a/memory/src/main/scala/filodb.memory/BlockDetective.scala +++ b/memory/src/main/scala/filodb.memory/BlockDetective.scala @@ -12,8 +12,8 @@ object BlockDetective { def containsPtr(ptr: BinaryRegion.NativePointer, blocks: Seq[Block]): Seq[Block] = blocks.filter { blk => ptr >= blk.address && ptr < (blk.address + blk.capacity) } - def containsPtr(ptr: BinaryRegion.NativePointer, blocks: java.util.List[Block]): Seq[Block] = - containsPtr(ptr, blocks.asScala) + def containsPtr(ptr: BinaryRegion.NativePointer, blocks: java.lang.Iterable[Block]): Seq[Block] = + containsPtr(ptr, Seq() ++ blocks.asScala) /** * Produces a string report containing reclaim history and ownership changes for diff --git a/memory/src/main/scala/filodb.memory/BlockManager.scala b/memory/src/main/scala/filodb.memory/BlockManager.scala index f29a6297fa..f1d9a5d750 100644 --- a/memory/src/main/scala/filodb.memory/BlockManager.scala +++ b/memory/src/main/scala/filodb.memory/BlockManager.scala @@ -149,9 +149,9 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, protected var firstPageAddress: Long = 0L - protected val freeBlocks: util.LinkedList[Block] = allocate() - protected[memory] val usedBlocks: util.LinkedList[Block] = new util.LinkedList[Block]() - protected[memory] val usedBlocksTimeOrdered = new util.TreeMap[Long, util.LinkedList[Block]] + 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]] val reclaimLog = new collection.mutable.Queue[ReclaimEvent] protected val lock = new ReentrantLock() @@ -346,9 +346,9 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, } } - protected def allocate(): util.LinkedList[Block] = { + protected def allocate(): util.ArrayDeque[Block] = { val numBlocks: Int = Math.floor(totalMemorySizeInBytes / blockSizeInBytes).toInt - val blocks = new util.LinkedList[Block]() + val blocks = new util.ArrayDeque[Block]() logger.info(s"Allocating $numBlocks blocks of $blockSizeInBytes bytes each, total $totalMemorySizeInBytes") firstPageAddress = MemoryIO.getCheckedInstance().allocateMemory(totalMemorySizeInBytes, false) for (i <- 0 until numBlocks) { @@ -363,7 +363,7 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, block.markInUse bucketTime match { case Some(bucket) => val blockList = Option(usedBlocksTimeOrdered.get(bucket)).getOrElse { - val list = new util.LinkedList[Block]() + val list = new util.ArrayDeque[Block]() usedBlocksTimeOrdered.put(bucket, list) list } @@ -405,7 +405,7 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, s"usedBlocksTimeOrdered=${usedBlocksTimeOrdered.asScala.toList.map{case(n, l) => (n, l.size)}}") } - def reclaimFrom(list: util.LinkedList[Block], reclaimedCounter: Counter): 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) { From 1142c725b67c464c362a6719392f09da4b253b36 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Fri, 29 May 2020 11:07:29 -0700 Subject: [PATCH 19/36] perf(coord): Timeout query immediately if it stayed in mailbox for too long (#761) --- .../scala/filodb.coordinator/QueryActor.scala | 9 +++++ .../filodb.jmh/PartKeyIndexBenchmark.scala | 36 ++++++++++++++----- project/plugins.sbt | 2 +- run_benchmarks.sh | 1 + 4 files changed, 39 insertions(+), 9 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala b/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala index e34f36103d..1cf6d4ebeb 100644 --- a/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala +++ b/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala @@ -124,6 +124,7 @@ final class QueryActor(memStore: MemStore, } private def processLogicalPlan2Query(q: LogicalPlan2Query, replyTo: ActorRef) = { + checkTimeout(q.qContext) // This is for CLI use only. Always prefer clients to materialize logical plan lpRequests.increment() try { @@ -138,6 +139,7 @@ final class QueryActor(memStore: MemStore, } private def processExplainPlanQuery(q: ExplainPlan2Query, replyTo: ActorRef): Unit = { + checkTimeout(q.qContext) try { val execPlan = queryPlanner.materialize(q.logicalPlan, q.qContext) replyTo ! execPlan @@ -161,6 +163,13 @@ final class QueryActor(memStore: MemStore, } } + def checkTimeout(queryContext: QueryContext): Unit = { + // 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) + throw QueryTimeoutException(queryTimeElapsed, this.getClass.getName) + } + def receive: Receive = { case q: LogicalPlan2Query => val replyTo = sender() processLogicalPlan2Query(q, replyTo) diff --git a/jmh/src/main/scala/filodb.jmh/PartKeyIndexBenchmark.scala b/jmh/src/main/scala/filodb.jmh/PartKeyIndexBenchmark.scala index 59d77b46e7..8067b8f7f7 100644 --- a/jmh/src/main/scala/filodb.jmh/PartKeyIndexBenchmark.scala +++ b/jmh/src/main/scala/filodb.jmh/PartKeyIndexBenchmark.scala @@ -53,9 +53,10 @@ class PartKeyIndexBenchmark { @BenchmarkMode(Array(Mode.Throughput)) @OutputTimeUnit(TimeUnit.SECONDS) def partIdsLookupWithEqualsFilters(): Unit = { - for ( i <- 0 to 20 optimized) { + for ( i <- 0 until 8 optimized) { partKeyIndex.partIdsFromFilters( - Seq(ColumnFilter("job", Filter.Equals(s"App-$i")), + Seq(ColumnFilter("_ns_", Filter.Equals(s"App-$i")), + ColumnFilter("_ws_", Filter.Equals("demo")), ColumnFilter("host", Filter.EqualsRegex("H0")), ColumnFilter("__name__", Filter.Equals("heap_usage"))), now, @@ -63,13 +64,29 @@ class PartKeyIndexBenchmark { } } + @Benchmark + @BenchmarkMode(Array(Mode.Throughput)) + @OutputTimeUnit(TimeUnit.SECONDS) + def emptyPartIdsLookupWithEqualsFilters(): Unit = { + for ( i <- 0 until 8 optimized) { + partKeyIndex.partIdsFromFilters( + Seq(ColumnFilter("_ns_", Filter.Equals(s"App-${i + 200}")), + ColumnFilter("_ws_", Filter.Equals("demo")), + ColumnFilter("host", Filter.EqualsRegex("H0")), + ColumnFilter("__name__", Filter.Equals("heap_usage"))), + now, + now + 1000) + } + } + @Benchmark @BenchmarkMode(Array(Mode.Throughput)) @OutputTimeUnit(TimeUnit.SECONDS) def partIdsLookupWithSuffixRegexFilters(): Unit = { - for ( i <- 0 to 20 optimized) { + for ( i <- 0 until 8 optimized) { partKeyIndex.partIdsFromFilters( - Seq(ColumnFilter("job", Filter.Equals(s"App-$i")), + Seq(ColumnFilter("_ns_", Filter.Equals(s"App-$i")), + ColumnFilter("_ws_", Filter.Equals("demo")), ColumnFilter("__name__", Filter.Equals("heap_usage")), ColumnFilter("instance", Filter.EqualsRegex("Instance-2.*"))), now, @@ -81,9 +98,10 @@ class PartKeyIndexBenchmark { @BenchmarkMode(Array(Mode.Throughput)) @OutputTimeUnit(TimeUnit.SECONDS) def partIdsLookupWithPrefixRegexFilters(): Unit = { - for ( i <- 0 to 20 optimized) { + for ( i <- 0 until 8 optimized) { partKeyIndex.partIdsFromFilters( - Seq(ColumnFilter("job", Filter.Equals(s"App-$i")), + Seq(ColumnFilter("_ns_", Filter.Equals(s"App-$i")), + ColumnFilter("_ws_", Filter.Equals("demo")), ColumnFilter("__name__", Filter.Equals("heap_usage")), ColumnFilter("instance", Filter.EqualsRegex(".*2"))), now, @@ -95,8 +113,10 @@ class PartKeyIndexBenchmark { @BenchmarkMode(Array(Mode.Throughput)) @OutputTimeUnit(TimeUnit.SECONDS) def startTimeLookupWithPartId(): Unit = { - for ( i <- 0 to 20 optimized) { - partKeyIndex.startTimeFromPartId(i) + for ( i <- 0 until 8 optimized) { + val pIds = debox.Buffer.empty[Int] + for ( j <- i * 1000 to i * 1000 + 1000 optimized) { pIds += j } + partKeyIndex.startTimeFromPartIds(pIds.iterator()) } } diff --git a/project/plugins.sbt b/project/plugins.sbt index 54f4063a75..63de00f3d3 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -2,7 +2,7 @@ addSbtPlugin("org.scalastyle" %% "scalastyle-sbt-plugin" % "1.0.0") addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.14.5") -addSbtPlugin("pl.project13.scala" % "sbt-jmh" % "0.3.3") +addSbtPlugin("pl.project13.scala" % "sbt-jmh" % "0.3.7") addSbtPlugin("com.typesafe.sbt" % "sbt-multi-jvm" % "0.4.0") diff --git a/run_benchmarks.sh b/run_benchmarks.sh index 01e72d416e..cdbe31c166 100755 --- a/run_benchmarks.sh +++ b/run_benchmarks.sh @@ -1,6 +1,7 @@ #!/bin/bash sbt "jmh/jmh:run -rf json -i 15 -wi 10 -f3 -jvmArgsAppend -XX:MaxInlineLevel=20 \ -jvmArgsAppend -Xmx4g -jvmArgsAppend -XX:MaxInlineSize=99 \ + -prof jmh.extras.JFR:dir=/tmp/filo-jmh \ filodb.jmh.QueryHiCardInMemoryBenchmark \ filodb.jmh.QueryInMemoryBenchmark \ filodb.jmh.QueryAndIngestBenchmark \ From 21d929751bcacce8956c5225c428255162a29567 Mon Sep 17 00:00:00 2001 From: "Brian S. O'Neill" Date: Mon, 1 Jun 2020 14:29:23 -0700 Subject: [PATCH 20/36] feat(memory): Updates for stall time. (#764) --- .../scala/filodb.memory/BlockManager.scala | 22 ++++++++++--------- 1 file changed, 12 insertions(+), 10 deletions(-) diff --git a/memory/src/main/scala/filodb.memory/BlockManager.scala b/memory/src/main/scala/filodb.memory/BlockManager.scala index f1d9a5d750..fbf5f9e624 100644 --- a/memory/src/main/scala/filodb.memory/BlockManager.scala +++ b/memory/src/main/scala/filodb.memory/BlockManager.scala @@ -109,15 +109,15 @@ class MemoryStats(tags: Map[String, String]) { /** * How much time a thread was potentially stalled while attempting to ensure - * free space. Unit is milliseconds. + * free space. Unit is nanoseconds. */ - val blockHeadroomStall = Kamon.gauge("blockstore-headroom-stall").withTags(TagSet.from(tags)) + val blockHeadroomStall = Kamon.counter("blockstore-headroom-stall-nanos").withTags(TagSet.from(tags)) /** * How much time a thread was stalled while attempting to acquire the reclaim lock. - * Unit is milliseconds. + * Unit is nanoseconds. */ - val blockReclaimStall = Kamon.gauge("blockstore-reclaim-stall").withTags(TagSet.from(tags)) + val blockReclaimStall = Kamon.counter("blockstore-reclaim-stall-nanos").withTags(TagSet.from(tags)) } final case class ReclaimEvent(block: Block, reclaimTime: Long, oldOwner: Option[BlockMemFactory], remaining: Long) @@ -249,7 +249,7 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, lock.unlock() var stamp: Long = 0 try { - val start = System.currentTimeMillis() + val start = System.nanoTime() // Give up after waiting (in total) a little over 16 seconds. stamp = tryExclusiveReclaimLock(8192) @@ -258,10 +258,12 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, // the lock isn't held. If the lock state is broken, then ingestion is really stuck // and the node must be restarted. Queries should always release the lock. logger.error(s"Lock for BlockManager.tryReclaimOnDemand timed out: ${reclaimLock}") + } else { + logger.debug("Lock for BlockManager.tryReclaimOnDemand aquired") } - val stall = System.currentTimeMillis() - start - stats.blockReclaimStall.update(stall) + val stall = System.nanoTime() - start + stats.blockReclaimStall.increment(stall) } finally { lock.lock() } @@ -315,7 +317,7 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, */ def ensureHeadroom(pct: Double): Int = { var numFree: Int = 0 - val start = System.currentTimeMillis() + val start = System.nanoTime() // Give up after waiting (in total) a little over 2 seconds. val stamp = tryExclusiveReclaimLock(1024) if (stamp == 0) { @@ -330,8 +332,8 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, val numBytes = numFree * blockSizeInBytes logger.debug(s"BlockManager.ensureFreePercent numFree: $numFree ($numBytes bytes)") } - val stall = System.currentTimeMillis() - start - stats.blockHeadroomStall.update(stall) + val stall = System.nanoTime() - start + stats.blockHeadroomStall.increment(stall) numFree } From 08d3698c0330ef07bd42b7ffd83b07e7340ace8a Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Mon, 1 Jun 2020 21:15:25 -0700 Subject: [PATCH 21/36] feat(jmh): Change QueryInMemoryBenchmark to simulate high-spread queries (#765) Allows us to simulate the case where data is spread over many shards, and explore query issues with that setup --- .../filodb.jmh/QueryInMemoryBenchmark.scala | 20 +++++++++++-------- 1 file changed, 12 insertions(+), 8 deletions(-) diff --git a/jmh/src/main/scala/filodb.jmh/QueryInMemoryBenchmark.scala b/jmh/src/main/scala/filodb.jmh/QueryInMemoryBenchmark.scala index 4a66a75793..727f1a8158 100644 --- a/jmh/src/main/scala/filodb.jmh/QueryInMemoryBenchmark.scala +++ b/jmh/src/main/scala/filodb.jmh/QueryInMemoryBenchmark.scala @@ -42,14 +42,14 @@ class QueryInMemoryBenchmark extends StrictLogging { import client.QueryCommands._ import NodeClusterActor._ - val numShards = 2 + val numShards = 64 val numSamples = 720 // 2 hours * 3600 / 10 sec interval val numSeries = 100 val startTime = System.currentTimeMillis - (3600*1000) val numQueries = 500 // Please make sure this number matches the OperationsPerInvocation below val queryIntervalMin = 55 // # minutes between start and stop val queryStep = 150 // # of seconds between each query sample "step" - val spread = 1 + val spread = 6 // TODO: move setup and ingestion to another trait val system = ActorSystem("test", ConfigFactory.load("filodb-defaults.conf")) @@ -70,8 +70,8 @@ class QueryInMemoryBenchmark extends StrictLogging { val storeConf = StoreConfig(ConfigFactory.parseString(""" | flush-interval = 1h - | shard-mem-size = 512MB - | ingestion-buffer-mem-size = 50MB + | shard-mem-size = 96MB + | ingestion-buffer-mem-size = 30MB | groups-per-shard = 4 | demand-paging-enabled = false """.stripMargin)) @@ -127,9 +127,13 @@ class QueryInMemoryBenchmark extends StrictLogging { LogicalPlan2Query(dataset.ref, plan, QueryContext(Some(new StaticSpreadProvider(SpreadChange(0, 1))), 20000)) } + var queriesSucceeded = 0 + var queriesFailed = 0 + @TearDown def shutdownFiloActors(): Unit = { cluster.shutdown() + println(s"Succeeded: $queriesSucceeded Failed: $queriesFailed") } // Window = 5 min and step=2.5 min, so 50% overlap @@ -141,8 +145,8 @@ class QueryInMemoryBenchmark extends StrictLogging { val futures = (0 until numQueries).map { n => val f = asyncAsk(coordinator, queryCommands(n % queryCommands.length)) f.onSuccess { - case q: QueryResult2 => - case e: QError => throw new RuntimeException(s"Query error $e") + case q: QueryResult2 => queriesSucceeded += 1 + case e: QError => queriesFailed += 1 } f } @@ -164,8 +168,8 @@ class QueryInMemoryBenchmark extends StrictLogging { val futures = (0 until numQueries).map { n => val f = asyncAsk(coordinator, queryCommands2(n % queryCommands2.length)) f.onSuccess { - case q: QueryResult2 => - case e: QError => throw new RuntimeException(s"Query error $e") + case q: QueryResult2 => queriesSucceeded += 1 + case e: QError => queriesFailed += 1 } f } From 061c6f139f8c9ed2a490442f57b13bd34d6cd209 Mon Sep 17 00:00:00 2001 From: Brian O'Neill Date: Tue, 2 Jun 2020 15:10:44 -0700 Subject: [PATCH 22/36] Update version. --- version.sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/version.sbt b/version.sbt index b2a239ee71..e7363ced9f 100644 --- a/version.sbt +++ b/version.sbt @@ -1 +1 @@ -version in ThisBuild := "0.9.6.integration-SNAPSHOT" +version in ThisBuild := "0.9.7.integration-SNAPSHOT" From 01664988996c6166a4d495a217f5bbaf680f0b07 Mon Sep 17 00:00:00 2001 From: Jackson Jeyapaul Date: Wed, 3 Jun 2020 12:29:17 -0700 Subject: [PATCH 23/36] misc(query): Do not query downsample cluster for metadata queries (#768) --- .../queryplanner/LongTimeRangePlanner.scala | 14 ++------------ 1 file changed, 2 insertions(+), 12 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LongTimeRangePlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LongTimeRangePlanner.scala index eec81ff187..8d34a14146 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LongTimeRangePlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LongTimeRangePlanner.scala @@ -2,8 +2,8 @@ package filodb.coordinator.queryplanner import filodb.coordinator.queryplanner.LogicalPlanUtils._ import filodb.core.query.QueryContext -import filodb.query.{LabelValues, LogicalPlan, PeriodicSeriesPlan, SeriesKeysByFilters} -import filodb.query.exec.{ExecPlan, LabelValuesDistConcatExec, PartKeysDistConcatExec, PlanDispatcher, StitchRvsExec} +import filodb.query.{LogicalPlan, PeriodicSeriesPlan} +import filodb.query.exec.{ExecPlan, PlanDispatcher, StitchRvsExec} /** * LongTimeRangePlanner knows about limited retention of raw data, and existence of downsampled data. @@ -66,16 +66,6 @@ class LongTimeRangePlanner(rawClusterPlanner: QueryPlanner, val rawEp = rawClusterPlanner.materialize(rawLp, qContext) StitchRvsExec(qContext, stitchDispatcher, Seq(rawEp, downsampleEp)) } - case l: LabelValues => - val rawExec = rawClusterPlanner.materialize(l, qContext) - val downSampleExec = downsampleClusterPlanner.materialize(l, qContext) - LabelValuesDistConcatExec(qContext, rawExec.dispatcher, Seq(rawExec, downSampleExec)) - - case s: SeriesKeysByFilters => - val rawExec = rawClusterPlanner.materialize(s, qContext) - val downSampleExec = downsampleClusterPlanner.materialize(s, qContext) - PartKeysDistConcatExec(qContext, rawExec.dispatcher, Seq(rawExec, downSampleExec)) - case _ => rawClusterPlanner.materialize(logicalPlan, qContext) } } From 9c543ffa6b45858190a953004edc82e890ed79a2 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Thu, 4 Jun 2020 13:50:01 -0700 Subject: [PATCH 24/36] bug(core): Release unwanted vectors eagerly when serving queries from downsampled data (#770) Other fixes: * Also fixing DownsampleMainSpec to use synthetic timestamps to bypass Query Planner restriction on data TTL * Fixed timeout check to reply with QueryError instead of failing actor --- README.md | 2 +- .../scala/filodb.coordinator/QueryActor.scala | 98 ++++---- .../queryplanner/ScalarQueriesSpec.scala | 80 +++---- .../DownsampledTimeSeriesShard.scala | 16 +- .../DownsampledTimeSeriesStore.scala | 5 +- .../memstore/OnDemandPagingShard.scala | 3 +- .../memstore/PagedReadablePartition.scala | 19 +- .../memstore/TimeSeriesMemStore.scala | 5 +- .../memstore/TimeSeriesShard.scala | 1 + .../scala/filodb.core/metadata/Schemas.scala | 2 +- .../scala/filodb.core/store/ChunkSource.scala | 7 +- .../query/exec/InProcessPlanDispatcher.scala | 3 +- .../exec/MultiSchemaPartitionsExec.scala | 2 +- .../query/exec/SelectChunkInfosExec.scala | 2 +- .../downsampler/DownsamplerMainSpec.scala | 211 +++++++++--------- 15 files changed, 247 insertions(+), 209 deletions(-) diff --git a/README.md b/README.md index 1192836ccc..c7f9230c04 100644 --- a/README.md +++ b/README.md @@ -340,7 +340,7 @@ If you had run the unit test `DownsamplerMainSpec` which populates data into the dataset, you can query downsample results by visiting the following URL: ``` -curl "http://localhost:8080/promql/prometheus/api/v1/query_range?query=my_counter\{_ws_='my_ws',_ns_='my_ns'\}&start=1574272801&end=1574273042&step=10&verbose=true&spread=2" +curl "http://localhost:9080/promql/prometheus/api/v1/query_range?query=my_counter\{_ws_='my_ws',_ns_='my_ns'\}&start=74372801&end=74373042&step=10&verbose=true&spread=2" ``` #### Local Scale Testing diff --git a/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala b/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala index 1cf6d4ebeb..5cfcc14a21 100644 --- a/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala +++ b/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala @@ -98,56 +98,60 @@ final class QueryActor(memStore: MemStore, private val queryErrors = Kamon.counter("queryactor-query-errors").withTags(TagSet.from(tags)) def execPhysicalPlan2(q: ExecPlan, replyTo: ActorRef): Unit = { - epRequests.increment() - Kamon.currentSpan().tag("query", q.getClass.getSimpleName) - Kamon.currentSpan().tag("query-id", q.queryContext.queryId) - val querySession = QuerySession(q.queryContext, queryConfig) - q.execute(memStore, querySession)(queryScheduler) - .foreach { res => - FiloSchedulers.assertThreadName(QuerySchedName) - 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 => - // 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) + 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) + q.execute(memStore, querySession)(queryScheduler) + .foreach { res => + FiloSchedulers.assertThreadName(QuerySchedName) + 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 => + // 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) + } } private def processLogicalPlan2Query(q: LogicalPlan2Query, replyTo: ActorRef) = { - checkTimeout(q.qContext) - // This is for CLI use only. Always prefer clients to materialize logical plan - lpRequests.increment() - try { - val execPlan = queryPlanner.materialize(q.logicalPlan, q.qContext) - self forward execPlan - } catch { - case NonFatal(ex) => - if (!ex.isInstanceOf[BadQueryException]) // dont log user errors - logger.error(s"Exception while materializing logical plan", ex) - replyTo ! QueryError("unknown", ex) + if (checkTimeout(q.qContext, replyTo)) { + // This is for CLI use only. Always prefer clients to materialize logical plan + lpRequests.increment() + try { + val execPlan = queryPlanner.materialize(q.logicalPlan, q.qContext) + self forward execPlan + } catch { + case NonFatal(ex) => + if (!ex.isInstanceOf[BadQueryException]) // dont log user errors + logger.error(s"Exception while materializing logical plan", ex) + replyTo ! QueryError("unknown", ex) + } } } private def processExplainPlanQuery(q: ExplainPlan2Query, replyTo: ActorRef): Unit = { - checkTimeout(q.qContext) - try { - val execPlan = queryPlanner.materialize(q.logicalPlan, q.qContext) - replyTo ! execPlan - } catch { - case NonFatal(ex) => - if (!ex.isInstanceOf[BadQueryException]) // dont log user errors - logger.error(s"Exception while materializing logical plan", ex) - replyTo ! QueryError("unknown", ex) + if (checkTimeout(q.qContext, replyTo)) { + try { + val execPlan = queryPlanner.materialize(q.logicalPlan, q.qContext) + replyTo ! execPlan + } catch { + case NonFatal(ex) => + if (!ex.isInstanceOf[BadQueryException]) // dont log user errors + logger.error(s"Exception while materializing logical plan", ex) + replyTo ! QueryError("unknown", ex) + } } } @@ -163,11 +167,13 @@ final class QueryActor(memStore: MemStore, } } - def checkTimeout(queryContext: QueryContext): Unit = { + 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) - throw QueryTimeoutException(queryTimeElapsed, this.getClass.getName) + if (queryTimeElapsed >= queryContext.queryTimeoutMillis) { + replyTo ! QueryError("Actor mailbox timeout", QueryTimeoutException(queryTimeElapsed, this.getClass.getName)) + false + } else true } def receive: Receive = { diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/ScalarQueriesSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/ScalarQueriesSpec.scala index 7f41b3ad8f..609bbc1186 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/ScalarQueriesSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/ScalarQueriesSpec.scala @@ -69,9 +69,9 @@ class ScalarQueriesSpec extends FunSpec with Matchers { """T~ScalarFunctionMapper(function=Scalar, funcParams=List()) |-E~DistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1153666897]) |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=3, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1153666897]) + |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=3, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1153666897]) |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=19, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1153666897]) + |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=19, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(test))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1153666897]) """.stripMargin maskDispatcher(execPlan.printTree()) shouldEqual (maskDispatcher(expected)) } @@ -99,11 +99,11 @@ class ScalarQueriesSpec extends FunSpec with Matchers { |-T~ScalarOperationMapper(operator=ADD, scalarOnLhs=false) |--FA1~TimeFuncArgs(RangeParams(1000,1000,1000)) |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1510751596]) + |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1510751596]) |-T~ScalarOperationMapper(operator=ADD, scalarOnLhs=false) |--FA1~TimeFuncArgs(RangeParams(1000,1000,1000)) |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None ) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1510751596])""".stripMargin + |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1510751596])""".stripMargin maskDispatcher(execPlan.printTree()) shouldEqual (maskDispatcher(expected)) } @@ -124,21 +124,21 @@ class ScalarQueriesSpec extends FunSpec with Matchers { |--T~ScalarFunctionMapper(function=Scalar, funcParams=List()) |---E~DistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1643642770]) |----T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1643642770]) + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1643642770]) |----T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1643642770]) + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1643642770]) |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=15, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(node_info))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1643642770]) + |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=15, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(node_info))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1643642770]) |-T~ScalarOperationMapper(operator=ADD, scalarOnLhs=true) |--FA1~ |--T~ScalarFunctionMapper(function=Scalar, funcParams=List()) |---E~DistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1643642770]) |----T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1643642770]) + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1643642770]) |----T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1643642770]) + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1643642770]) |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=31, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(node_info))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1643642770])""".stripMargin + |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=31, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(node_info))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1643642770])""".stripMargin maskDispatcher(execPlan.printTree()) shouldEqual (maskDispatcher(expected)) } @@ -154,11 +154,11 @@ class ScalarQueriesSpec extends FunSpec with Matchers { |-T~ScalarOperationMapper(operator=ADD, scalarOnLhs=true) |--FA1~StaticFuncArgs(10.0,RangeParams(1000,1000,1000)) |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1110105620]) + |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1110105620]) |-T~ScalarOperationMapper(operator=ADD, scalarOnLhs=true) |--FA1~StaticFuncArgs(10.0,RangeParams(1000,1000,1000)) |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1110105620]) + |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1110105620]) |""".stripMargin maskDispatcher(execPlan.printTree()) shouldEqual (maskDispatcher(expected)) } @@ -187,11 +187,11 @@ class ScalarQueriesSpec extends FunSpec with Matchers { |-T~ScalarOperationMapper(operator=ADD, scalarOnLhs=false) |--FA1~StaticFuncArgs(10.0,RangeParams(1000,1000,1000)) |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1245070935]) + |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1245070935]) |-T~ScalarOperationMapper(operator=ADD, scalarOnLhs=false) |--FA1~StaticFuncArgs(10.0,RangeParams(1000,1000,1000)) |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1245070935]) + |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1245070935]) """.stripMargin maskDispatcher(execPlan.printTree()) shouldEqual (maskDispatcher(expected)) } @@ -206,9 +206,9 @@ class ScalarQueriesSpec extends FunSpec with Matchers { |-T~ScalarFunctionMapper(function=Scalar, funcParams=List()) |--E~DistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-2114470773]) |---T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-2114470773]) + |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-2114470773]) |---T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-2114470773]) + |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-2114470773]) |-E~TimeScalarGeneratorExec(params = RangeParams(1000,1000,1000), function = Time) on InProcessPlanDispatcher""".stripMargin maskDispatcher(execPlan.printTree()) shouldEqual (maskDispatcher(expected)) } @@ -223,15 +223,15 @@ class ScalarQueriesSpec extends FunSpec with Matchers { |-T~ScalarFunctionMapper(function=Scalar, funcParams=List()) |--E~DistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1081611650]) |---T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1081611650]) + |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1081611650]) |---T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1081611650]) + |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1081611650]) |-T~ScalarFunctionMapper(function=Scalar, funcParams=List()) |--E~DistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1081611650]) |---T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=15, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(node_info))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1081611650]) + |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=15, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(node_info))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1081611650]) |---T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=31, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(node_info))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1081611650])""".stripMargin + |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=31, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(node_info))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1081611650])""".stripMargin maskDispatcher(execPlan.printTree()) shouldEqual (maskDispatcher(expected)) } @@ -245,9 +245,9 @@ class ScalarQueriesSpec extends FunSpec with Matchers { |-T~ScalarFunctionMapper(function=Scalar, funcParams=List()) |--E~DistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#856852588]) |---T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#856852588]) + |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#856852588]) |---T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#856852588]) + |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#856852588]) |""".stripMargin maskDispatcher(execPlan.printTree()) shouldEqual (maskDispatcher(expected)) } @@ -315,21 +315,21 @@ class ScalarQueriesSpec extends FunSpec with Matchers { |--T~ScalarFunctionMapper(function=Scalar, funcParams=List()) |---E~DistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#79055924]) |----T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#79055924]) + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#79055924]) |----T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#79055924]) + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#79055924]) |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=15, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(node_info))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#79055924]) + |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=15, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(node_info))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#79055924]) |-T~InstantVectorFunctionMapper(function=ClampMax) |--FA1~ |--T~ScalarFunctionMapper(function=Scalar, funcParams=List()) |---E~DistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#79055924]) |----T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#79055924]) + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#79055924]) |----T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#79055924]) + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#79055924]) |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=31, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(node_info))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#79055924])""".stripMargin + |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=31, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(node_info))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#79055924])""".stripMargin maskDispatcher(execPlan.printTree()) shouldEqual (maskDispatcher(expected)) } @@ -365,13 +365,13 @@ class ScalarQueriesSpec extends FunSpec with Matchers { |--T~ScalarOperationMapper(operator=ADD, scalarOnLhs=true) |---FA1~StaticFuncArgs(2.0,RangeParams(1000,1000,1000)) |---T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1098511474]) + |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1098511474]) |-T~ScalarOperationMapper(operator=LSS_BOOL, scalarOnLhs=true) |--FA1~StaticFuncArgs(1.0,RangeParams(1000,1000,1000)) |--T~ScalarOperationMapper(operator=ADD, scalarOnLhs=true) |---FA1~StaticFuncArgs(2.0,RangeParams(1000,1000,1000)) |---T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1098511474]) + |----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-1098511474]) |""".stripMargin maskDispatcher(execPlan.printTree()) shouldEqual (maskDispatcher(expected)) } @@ -389,21 +389,21 @@ class ScalarQueriesSpec extends FunSpec with Matchers { |--T~ScalarFunctionMapper(function=Scalar, funcParams=List()) |---E~DistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-879546200]) |----T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=15, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(node_info))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-879546200]) + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=15, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(node_info))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-879546200]) |----T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=31, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(node_info))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-879546200]) + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=31, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(node_info))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-879546200]) |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-879546200]) + |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-879546200]) |-T~ScalarOperationMapper(operator=GTR_BOOL, scalarOnLhs=true) |--FA1~ |--T~ScalarFunctionMapper(function=Scalar, funcParams=List()) |---E~DistConcatExec() on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-879546200]) |----T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=15, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(node_info))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-879546200]) + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=15, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(node_info))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-879546200]) |----T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=31, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(node_info))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-879546200]) + |-----E~MultiSchemaPartitionsExec(dataset=timeseries, shard=31, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(node_info))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-879546200]) |--T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-879546200]) + |---E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#-879546200]) |""".stripMargin maskDispatcher(execPlan.printTree()) shouldEqual (maskDispatcher(expected)) } @@ -417,13 +417,13 @@ class ScalarQueriesSpec extends FunSpec with Matchers { val expected = """E~BinaryJoinExec(binaryOp=GTR_BOOL, on=List(), ignoring=List()) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1392317349]) |-T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |--E~MultiSchemaPartitionsExec(dataset=timeseries, shard=15, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(node_info))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1392317349]) + |--E~MultiSchemaPartitionsExec(dataset=timeseries, shard=15, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(node_info))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1392317349]) |-T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |--E~MultiSchemaPartitionsExec(dataset=timeseries, shard=31, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(node_info))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1392317349]) + |--E~MultiSchemaPartitionsExec(dataset=timeseries, shard=31, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(node_info))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1392317349]) |-T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |--E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1392317349]) + |--E~MultiSchemaPartitionsExec(dataset=timeseries, shard=5, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1392317349]) |-T~PeriodicSamplesMapper(start=1000000, step=1000000, end=1000000, window=None, functionId=None, rawSource=true, offsetMs=None) - |--E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1392317349]) + |--E~MultiSchemaPartitionsExec(dataset=timeseries, shard=21, chunkMethod=TimeRangeChunkScan(700000,1000000), filters=List(ColumnFilter(job,Equals(app)), ColumnFilter(__name__,Equals(http_requests_total))), colName=None, schema=None) on ActorPlanDispatcher(Actor[akka://default/system/testProbe-1#1392317349]) |""".stripMargin maskDispatcher(execPlan.printTree()) shouldEqual (maskDispatcher(expected)) } diff --git a/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesShard.scala b/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesShard.scala index 097783ae08..9f32528a9e 100644 --- a/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesShard.scala @@ -14,7 +14,7 @@ import monix.eval.Task import monix.execution.{CancelableFuture, Scheduler, UncaughtExceptionReporter} import monix.reactive.Observable -import filodb.core.DatasetRef +import filodb.core.{DatasetRef, Types} import filodb.core.binaryrecord2.RecordSchema import filodb.core.memstore._ import filodb.core.metadata.Schemas @@ -229,6 +229,7 @@ class DownsampledTimeSeriesShard(rawDatasetRef: DatasetRef, } def scanPartitions(lookup: PartLookupResult, + colIds: Seq[Types.ColumnId], querySession: QuerySession): Observable[ReadablePartition] = { // Step 1: Choose the downsample level depending on the range requested @@ -256,13 +257,13 @@ class DownsampledTimeSeriesShard(rawDatasetRef: DatasetRef, SinglePartitionScan(partBytes, shardNum), lookup.chunkMethod) .map { pd => - val part = makePagedPartition(pd, lookup.firstSchemaId.get) + val part = makePagedPartition(pd, lookup.firstSchemaId.get, colIds) stats.partitionsQueried.increment() stats.chunksQueried.increment(part.numChunks) partLoadSpan.finish() part } - .defaultIfEmpty(makePagedPartition(RawPartData(partBytes, Seq.empty), lookup.firstSchemaId.get)) + .defaultIfEmpty(makePagedPartition(RawPartData(partBytes, Seq.empty), lookup.firstSchemaId.get, colIds)) .headL } } @@ -292,19 +293,22 @@ class DownsampledTimeSeriesShard(rawDatasetRef: DatasetRef, // since it is the highest resolution/ttl downsampleTtls.last.toMillis -> downsampledDatasetRefs.last case TimeRangeChunkScan(startTime, _) => - val ttlIndex = downsampleTtls.indexWhere(t => startTime > System.currentTimeMillis() - t.toMillis) + var ttlIndex = downsampleTtls.indexWhere(t => startTime > System.currentTimeMillis() - t.toMillis) + // -1 return value means query startTime is before the earliest retention. Just pick the highest resolution + if (ttlIndex == -1) ttlIndex = downsampleTtls.size - 1 downsampleConfig.resolutions(ttlIndex).toMillis -> downsampledDatasetRefs(ttlIndex) case _ => ??? } } - private def makePagedPartition(part: RawPartData, firstSchemaId: Int): ReadablePartition = { + private def makePagedPartition(part: RawPartData, firstSchemaId: Int, + colIds: Seq[Types.ColumnId]): ReadablePartition = { val schemaId = RecordSchema.schemaID(part.partitionKey, UnsafeUtils.arayOffset) if (schemaId != firstSchemaId) { throw SchemaMismatch(schemas.schemaName(firstSchemaId), schemas.schemaName(schemaId)) } // FIXME It'd be nice to pass in the correct partId here instead of -1 - new PagedReadablePartition(schemas(schemaId), shardNum, -1, part) + new PagedReadablePartition(schemas(schemaId), shardNum, -1, part, colIds) } /** diff --git a/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesStore.scala b/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesStore.scala index 548fdd4fff..0c546965b6 100644 --- a/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesStore.scala +++ b/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesStore.scala @@ -10,7 +10,7 @@ import monix.execution.{CancelableFuture, Scheduler} import monix.reactive.Observable import org.jctools.maps.NonBlockingHashMapLong -import filodb.core.{DatasetRef, Response} +import filodb.core.{DatasetRef, Response, Types} import filodb.core.memstore._ import filodb.core.metadata.Schemas import filodb.core.query.{ColumnFilter, QuerySession} @@ -101,6 +101,7 @@ extends MemStore with StrictLogging { def scanPartitions(ref: DatasetRef, lookupRes: PartLookupResult, + colIds: Seq[Types.ColumnId], querySession: QuerySession): Observable[ReadablePartition] = { val shard = datasets(ref).get(lookupRes.shard) @@ -108,7 +109,7 @@ extends MemStore with StrictLogging { throw new IllegalArgumentException(s"Shard $shard of dataset $ref is not assigned to " + s"this node. Was it was recently reassigned to another node? Prolonged occurrence indicates an issue.") } - shard.scanPartitions(lookupRes, querySession) + shard.scanPartitions(lookupRes, colIds, querySession) } def activeShards(dataset: DatasetRef): Seq[Int] = diff --git a/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala b/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala index 1d1dfd3493..454009546d 100644 --- a/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala +++ b/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala @@ -10,7 +10,7 @@ import monix.eval.Task import monix.execution.Scheduler import monix.reactive.{Observable, OverflowStrategy} -import filodb.core.DatasetRef +import filodb.core.{DatasetRef, Types} import filodb.core.binaryrecord2.RecordSchema import filodb.core.downsample.{DownsampleConfig, DownsamplePublisher} import filodb.core.metadata.Schemas @@ -74,6 +74,7 @@ TimeSeriesShard(ref, schemas, storeConfig, shardNum, bufferMemoryManager, rawSto // Definitely room for improvement, such as fetching multiple partitions at once, more parallelism, etc. //scalastyle:off override def scanPartitions(partLookupRes: PartLookupResult, + colIds: Seq[Types.ColumnId], querySession: QuerySession): Observable[ReadablePartition] = { capDataScannedPerShardCheck(partLookupRes) diff --git a/core/src/main/scala/filodb.core/memstore/PagedReadablePartition.scala b/core/src/main/scala/filodb.core/memstore/PagedReadablePartition.scala index 6ce238e04b..82f7852e71 100644 --- a/core/src/main/scala/filodb.core/memstore/PagedReadablePartition.scala +++ b/core/src/main/scala/filodb.core/memstore/PagedReadablePartition.scala @@ -1,7 +1,10 @@ package filodb.core.memstore +import java.nio.ByteBuffer + import com.typesafe.scalalogging.StrictLogging +import filodb.core.Types import filodb.core.Types.ChunkID import filodb.core.metadata.Schema import filodb.core.store._ @@ -9,6 +12,7 @@ import filodb.memory.format.UnsafeUtils object PagedReadablePartition extends StrictLogging { val _log = logger + val emptyByteBuffer = ByteBuffer.allocate(0) } /** @@ -20,11 +24,24 @@ object PagedReadablePartition extends StrictLogging { * * Any ChunkScanMethod will return results from all available chunks. This optimization * is done since that check would already done and does not need to be repeated. + * + * @param colIds the colIds that need to be retained. Leave empty if all are needed. */ class PagedReadablePartition(override val schema: Schema, override val shard: Int, override val partID: Int, - partData: RawPartData) extends ReadablePartition { + partData: RawPartData, + colIds: Seq[Types.ColumnId] = Seq.empty) extends ReadablePartition { + + import PagedReadablePartition._ + val notNeededColIds = if (colIds.nonEmpty) schema.dataInfos.indices.toSet -- colIds.toSet + else Set.empty + partData.chunkSets.foreach { vectors => + // release vectors that are not needed so they can be GCed quickly before scans + // finish. This is a temporary workaround since we dont have ability to fetch + // specific columns from Cassandra + notNeededColIds.foreach(i => vectors.vectors(i) = emptyByteBuffer) + } override def numChunks: Int = partData.chunkSets.length diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala index 8458c22eea..c9166de1da 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala @@ -10,7 +10,7 @@ import monix.execution.{CancelableFuture, Scheduler} import monix.reactive.Observable import org.jctools.maps.NonBlockingHashMapLong -import filodb.core.{DatasetRef, Response} +import filodb.core.{DatasetRef, Response, Types} import filodb.core.downsample.{DownsampleConfig, DownsamplePublisher} import filodb.core.metadata.Schemas import filodb.core.query.{ColumnFilter, QuerySession} @@ -200,6 +200,7 @@ extends MemStore with StrictLogging { def scanPartitions(ref: DatasetRef, iter: PartLookupResult, + colIds: Seq[Types.ColumnId], querySession: QuerySession): Observable[ReadablePartition] = { val shard = datasets(ref).get(iter.shard) @@ -207,7 +208,7 @@ extends MemStore with StrictLogging { throw new IllegalArgumentException(s"Shard ${iter.shard} of dataset $ref is not assigned to " + s"this node. Was it was recently reassigned to another node? Prolonged occurrence indicates an issue.") } - shard.scanPartitions(iter, querySession) + shard.scanPartitions(iter, colIds, querySession) } def lookupPartitions(ref: DatasetRef, diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala index 00b0dc38e4..a40b90beb6 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala @@ -1465,6 +1465,7 @@ class TimeSeriesShard(val ref: DatasetRef, } def scanPartitions(iterResult: PartLookupResult, + colIds: Seq[Types.ColumnId], querySession: QuerySession): Observable[ReadablePartition] = { val partIter = new InMemPartitionIterator2(iterResult.partsInMemory) diff --git a/core/src/main/scala/filodb.core/metadata/Schemas.scala b/core/src/main/scala/filodb.core/metadata/Schemas.scala index d251b1eff5..e144941f9f 100644 --- a/core/src/main/scala/filodb.core/metadata/Schemas.scala +++ b/core/src/main/scala/filodb.core/metadata/Schemas.scala @@ -300,7 +300,7 @@ final case class Schemas(part: PartitionSchema, val bytesPerSample = bytesPerSampleSwag(schemaId) val estDataSize = bytesPerSample * numTsPartitions * numSamplesPerChunk * numChunksPerTs require(estDataSize < dataSizeLimit, - s"Estimate of $estDataSize bytes exceeds limit of " + + s"With match of $numTsPartitions time series, estimate of $estDataSize bytes exceeds limit of " + s"$dataSizeLimit bytes queried per shard with $bytesPerSample bytes per sample " + s"for ${_schemas(schemaId).name} schema. Try one or more of these: " + s"(a) narrow your query filters to reduce to fewer than the current $numTsPartitions matches " + diff --git a/core/src/main/scala/filodb.core/store/ChunkSource.scala b/core/src/main/scala/filodb.core/store/ChunkSource.scala index b42161fb0b..35efdcc104 100644 --- a/core/src/main/scala/filodb.core/store/ChunkSource.scala +++ b/core/src/main/scala/filodb.core/store/ChunkSource.scala @@ -82,13 +82,14 @@ trait ChunkSource extends RawChunkSource with StrictLogging { querySession: QuerySession): Observable[ReadablePartition] = { logger.debug(s"scanPartitions dataset=$ref shard=${partMethod.shard} " + s"partMethod=$partMethod chunkMethod=$chunkMethod") - scanPartitions(ref, lookupPartitions(ref, partMethod, chunkMethod, querySession), querySession) + scanPartitions(ref, lookupPartitions(ref, partMethod, chunkMethod, querySession), columnIDs, querySession) } // Internal API that needs to actually be implemented def scanPartitions(ref: DatasetRef, lookupRes: PartLookupResult, + colIds: Seq[Types.ColumnId], querySession: QuerySession): Observable[ReadablePartition] // internal method to find # of groups in a dataset @@ -134,12 +135,12 @@ trait ChunkSource extends RawChunkSource with StrictLogging { val numGroups = groupsInDataset(ref) val filteredParts = if (filterSchemas) { - scanPartitions(ref, lookupRes, querySession) + scanPartitions(ref, lookupRes, columnIDs, querySession) .filter { p => p.schema.schemaHash == schema.schemaHash && p.hasChunks(lookupRes.chunkMethod) } } else { lookupRes.firstSchemaId match { case Some(reqSchemaId) => - scanPartitions(ref, lookupRes, querySession).filter { p => + scanPartitions(ref, lookupRes, columnIDs, querySession).filter { p => if (p.schema.schemaHash != reqSchemaId) throw SchemaMismatch(Schemas.global.schemaName(reqSchemaId), p.schema.name) p.hasChunks(lookupRes.chunkMethod) diff --git a/query/src/main/scala/filodb/query/exec/InProcessPlanDispatcher.scala b/query/src/main/scala/filodb/query/exec/InProcessPlanDispatcher.scala index f9f7c4cfe0..14e794a786 100644 --- a/query/src/main/scala/filodb/query/exec/InProcessPlanDispatcher.scala +++ b/query/src/main/scala/filodb/query/exec/InProcessPlanDispatcher.scala @@ -5,7 +5,7 @@ import monix.eval.Task import monix.execution.Scheduler import monix.reactive.Observable -import filodb.core.DatasetRef +import filodb.core.{DatasetRef, Types} import filodb.core.memstore.PartLookupResult import filodb.core.metadata.Schemas import filodb.core.query.{EmptyQueryConfig, QueryConfig, QuerySession} @@ -44,6 +44,7 @@ case object InProcessPlanDispatcher extends PlanDispatcher { case class UnsupportedChunkSource() extends ChunkSource { def scanPartitions(ref: DatasetRef, iter: PartLookupResult, + colIds: Seq[Types.ColumnId], querySession: QuerySession): Observable[ReadablePartition] = throw new UnsupportedOperationException("This operation is not supported") diff --git a/query/src/main/scala/filodb/query/exec/MultiSchemaPartitionsExec.scala b/query/src/main/scala/filodb/query/exec/MultiSchemaPartitionsExec.scala index c7f6ecade3..748fcdac18 100644 --- a/query/src/main/scala/filodb/query/exec/MultiSchemaPartitionsExec.scala +++ b/query/src/main/scala/filodb/query/exec/MultiSchemaPartitionsExec.scala @@ -104,7 +104,7 @@ final case class MultiSchemaPartitionsExec(queryContext: QueryContext, } protected def args: String = s"dataset=$dataset, shard=$shard, " + - s"chunkMethod=$chunkMethod, filters=$filters, colName=$colName" + s"chunkMethod=$chunkMethod, filters=$filters, colName=$colName, schema=$schema" // Print inner node's details for debugging override def curNodeText(level: Int): String = { diff --git a/query/src/main/scala/filodb/query/exec/SelectChunkInfosExec.scala b/query/src/main/scala/filodb/query/exec/SelectChunkInfosExec.scala index b449ffbefb..915624457d 100644 --- a/query/src/main/scala/filodb/query/exec/SelectChunkInfosExec.scala +++ b/query/src/main/scala/filodb/query/exec/SelectChunkInfosExec.scala @@ -54,7 +54,7 @@ final case class SelectChunkInfosExec(queryContext: QueryContext, val partCols = dataSchema.partitionInfos val numGroups = source.groupsInDataset(dataset) Kamon.currentSpan().mark("creating-scanpartitions") - val rvs = source.scanPartitions(dataset, lookupRes, querySession) + val rvs = source.scanPartitions(dataset, lookupRes, Seq.empty, querySession) .filter(_.hasChunks(chunkMethod)) .map { partition => source.stats.incrReadPartitions(1) diff --git a/spark-jobs/src/test/scala/filodb/downsampler/DownsamplerMainSpec.scala b/spark-jobs/src/test/scala/filodb/downsampler/DownsamplerMainSpec.scala index 61a44936a5..b02f9d6840 100644 --- a/spark-jobs/src/test/scala/filodb/downsampler/DownsamplerMainSpec.scala +++ b/spark-jobs/src/test/scala/filodb/downsampler/DownsamplerMainSpec.scala @@ -78,7 +78,7 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w val gaugeLowFreqName = "my_gauge_low_freq" var gaugeLowFreqPartKeyBytes: Array[Byte] = _ - val lastSampleTime = 1574373042000L + val lastSampleTime = 74373042000L val pkUpdateHour = hour(lastSampleTime) val metricNames = Seq(gaugeName, gaugeLowFreqName, counterName, histName, untypedName) @@ -112,20 +112,20 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w untypedPartKeyBytes = part.partKeyBytes val rawSamples = Stream( - Seq(1574372801000L, 3d, untypedName, seriesTags), - Seq(1574372802000L, 5d, untypedName, seriesTags), + Seq(74372801000L, 3d, untypedName, seriesTags), + Seq(74372802000L, 5d, untypedName, seriesTags), - Seq(1574372861000L, 9d, untypedName, seriesTags), - Seq(1574372862000L, 11d, untypedName, seriesTags), + Seq(74372861000L, 9d, untypedName, seriesTags), + Seq(74372862000L, 11d, untypedName, seriesTags), - Seq(1574372921000L, 13d, untypedName, seriesTags), - Seq(1574372922000L, 15d, untypedName, seriesTags), + Seq(74372921000L, 13d, untypedName, seriesTags), + Seq(74372922000L, 15d, untypedName, seriesTags), - Seq(1574372981000L, 17d, untypedName, seriesTags), - Seq(1574372982000L, 15d, untypedName, seriesTags), + Seq(74372981000L, 17d, untypedName, seriesTags), + Seq(74372982000L, 15d, untypedName, seriesTags), - Seq(1574373041000L, 13d, untypedName, seriesTags), - Seq(1574373042000L, 11d, untypedName, seriesTags) + Seq(74373041000L, 13d, untypedName, seriesTags), + Seq(74373042000L, 11d, untypedName, seriesTags) ) MachineMetricsData.records(rawDataset, rawSamples).records.foreach { case (base, offset) => @@ -136,7 +136,7 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w val chunks = part.makeFlushChunks(offheapMem.blockMemFactory) rawColStore.write(rawDataset.ref, Observable.fromIterator(chunks)).futureValue - val pk = PartKeyRecord(untypedPartKeyBytes, 1574372801000L, 1574373042000L, Some(150)) + val pk = PartKeyRecord(untypedPartKeyBytes, 74372801000L, 74373042000L, Some(150)) rawColStore.writePartKeys(rawDataset.ref, 0, Observable.now(pk), 259200, pkUpdateHour).futureValue } @@ -154,20 +154,20 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w gaugePartKeyBytes = part.partKeyBytes val rawSamples = Stream( - Seq(1574372801000L, 3d, gaugeName, seriesTags), - Seq(1574372802000L, 5d, gaugeName, seriesTags), + Seq(74372801000L, 3d, gaugeName, seriesTags), + Seq(74372802000L, 5d, gaugeName, seriesTags), - Seq(1574372861000L, 9d, gaugeName, seriesTags), - Seq(1574372862000L, 11d, gaugeName, seriesTags), + Seq(74372861000L, 9d, gaugeName, seriesTags), + Seq(74372862000L, 11d, gaugeName, seriesTags), - Seq(1574372921000L, 13d, gaugeName, seriesTags), - Seq(1574372922000L, 15d, gaugeName, seriesTags), + Seq(74372921000L, 13d, gaugeName, seriesTags), + Seq(74372922000L, 15d, gaugeName, seriesTags), - Seq(1574372981000L, 17d, gaugeName, seriesTags), - Seq(1574372982000L, 15d, gaugeName, seriesTags), + Seq(74372981000L, 17d, gaugeName, seriesTags), + Seq(74372982000L, 15d, gaugeName, seriesTags), - Seq(1574373041000L, 13d, gaugeName, seriesTags), - Seq(1574373042000L, 11d, gaugeName, seriesTags) + Seq(74373041000L, 13d, gaugeName, seriesTags), + Seq(74373042000L, 11d, gaugeName, seriesTags) ) MachineMetricsData.records(rawDataset, rawSamples).records.foreach { case (base, offset) => @@ -178,7 +178,7 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w val chunks = part.makeFlushChunks(offheapMem.blockMemFactory) rawColStore.write(rawDataset.ref, Observable.fromIterator(chunks)).futureValue - val pk = PartKeyRecord(gaugePartKeyBytes, 1574372801000L, 1574373042000L, Some(150)) + val pk = PartKeyRecord(gaugePartKeyBytes, 74372801000L, 74373042000L, Some(150)) rawColStore.writePartKeys(rawDataset.ref, 0, Observable.now(pk), 259200, pkUpdateHour).futureValue } @@ -196,18 +196,18 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w gaugeLowFreqPartKeyBytes = part.partKeyBytes val rawSamples = Stream( - Seq(1574372801000L, 3d, gaugeName, seriesTags), - Seq(1574372802000L, 5d, gaugeName, seriesTags), + Seq(74372801000L, 3d, gaugeName, seriesTags), + Seq(74372802000L, 5d, gaugeName, seriesTags), // skip next minute - Seq(1574372921000L, 13d, gaugeName, seriesTags), - Seq(1574372922000L, 15d, gaugeName, seriesTags), + Seq(74372921000L, 13d, gaugeName, seriesTags), + Seq(74372922000L, 15d, gaugeName, seriesTags), // skip next minute - Seq(1574373041000L, 13d, gaugeName, seriesTags), - Seq(1574373042000L, 11d, gaugeName, seriesTags) + Seq(74373041000L, 13d, gaugeName, seriesTags), + Seq(74373042000L, 11d, gaugeName, seriesTags) ) MachineMetricsData.records(rawDataset, rawSamples).records.foreach { case (base, offset) => @@ -218,7 +218,7 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w val chunks = part.makeFlushChunks(offheapMem.blockMemFactory) rawColStore.write(rawDataset.ref, Observable.fromIterator(chunks)).futureValue - val pk = PartKeyRecord(gaugeLowFreqPartKeyBytes, 1574372801000L, 1574373042000L, Some(150)) + val pk = PartKeyRecord(gaugeLowFreqPartKeyBytes, 74372801000L, 74373042000L, Some(150)) rawColStore.writePartKeys(rawDataset.ref, 0, Observable.now(pk), 259200, pkUpdateHour).futureValue } @@ -236,24 +236,24 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w counterPartKeyBytes = part.partKeyBytes val rawSamples = Stream( - Seq(1574372801000L, 3d, counterName, seriesTags), - Seq(1574372801500L, 4d, counterName, seriesTags), - Seq(1574372802000L, 5d, counterName, seriesTags), + Seq(74372801000L, 3d, counterName, seriesTags), + Seq(74372801500L, 4d, counterName, seriesTags), + Seq(74372802000L, 5d, counterName, seriesTags), - Seq(1574372861000L, 9d, counterName, seriesTags), - Seq(1574372861500L, 10d, counterName, seriesTags), - Seq(1574372862000L, 11d, counterName, seriesTags), + Seq(74372861000L, 9d, counterName, seriesTags), + Seq(74372861500L, 10d, counterName, seriesTags), + Seq(74372862000L, 11d, counterName, seriesTags), - Seq(1574372921000L, 2d, counterName, seriesTags), - Seq(1574372921500L, 7d, counterName, seriesTags), - Seq(1574372922000L, 15d, counterName, seriesTags), + Seq(74372921000L, 2d, counterName, seriesTags), + Seq(74372921500L, 7d, counterName, seriesTags), + Seq(74372922000L, 15d, counterName, seriesTags), - Seq(1574372981000L, 17d, counterName, seriesTags), - Seq(1574372981500L, 1d, counterName, seriesTags), - Seq(1574372982000L, 15d, counterName, seriesTags), + Seq(74372981000L, 17d, counterName, seriesTags), + Seq(74372981500L, 1d, counterName, seriesTags), + Seq(74372982000L, 15d, counterName, seriesTags), - Seq(1574373041000L, 18d, counterName, seriesTags), - Seq(1574373042000L, 20d, counterName, seriesTags) + Seq(74373041000L, 18d, counterName, seriesTags), + Seq(74373042000L, 20d, counterName, seriesTags) ) MachineMetricsData.records(rawDataset, rawSamples).records.foreach { case (base, offset) => @@ -264,7 +264,7 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w val chunks = part.makeFlushChunks(offheapMem.blockMemFactory) rawColStore.write(rawDataset.ref, Observable.fromIterator(chunks)).futureValue - val pk = PartKeyRecord(counterPartKeyBytes, 1574372801000L, 1574373042000L, Some(1)) + val pk = PartKeyRecord(counterPartKeyBytes, 74372801000L, 74373042000L, Some(1)) rawColStore.writePartKeys(rawDataset.ref, 0, Observable.now(pk), 259200, pkUpdateHour).futureValue } @@ -283,24 +283,24 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w val bucketScheme = CustomBuckets(Array(3d, 10d, Double.PositiveInfinity)) val rawSamples = Stream( // time, sum, count, hist, name, tags - Seq(1574372801000L, 0d, 1d, LongHistogram(bucketScheme, Array(0L, 0, 1)), histName, seriesTags), - Seq(1574372801500L, 2d, 3d, LongHistogram(bucketScheme, Array(0L, 2, 3)), histName, seriesTags), - Seq(1574372802000L, 5d, 6d, LongHistogram(bucketScheme, Array(2L, 5, 6)), histName, seriesTags), + Seq(74372801000L, 0d, 1d, LongHistogram(bucketScheme, Array(0L, 0, 1)), histName, seriesTags), + Seq(74372801500L, 2d, 3d, LongHistogram(bucketScheme, Array(0L, 2, 3)), histName, seriesTags), + Seq(74372802000L, 5d, 6d, LongHistogram(bucketScheme, Array(2L, 5, 6)), histName, seriesTags), - Seq(1574372861000L, 9d, 9d, LongHistogram(bucketScheme, Array(2L, 5, 9)), histName, seriesTags), - Seq(1574372861500L, 10d, 10d, LongHistogram(bucketScheme, Array(2L, 5, 10)), histName, seriesTags), - Seq(1574372862000L, 11d, 14d, LongHistogram(bucketScheme, Array(2L, 8, 14)), histName, seriesTags), + Seq(74372861000L, 9d, 9d, LongHistogram(bucketScheme, Array(2L, 5, 9)), histName, seriesTags), + Seq(74372861500L, 10d, 10d, LongHistogram(bucketScheme, Array(2L, 5, 10)), histName, seriesTags), + Seq(74372862000L, 11d, 14d, LongHistogram(bucketScheme, Array(2L, 8, 14)), histName, seriesTags), - Seq(1574372921000L, 2d, 2d, LongHistogram(bucketScheme, Array(0L, 0, 2)), histName, seriesTags), - Seq(1574372921500L, 7d, 9d, LongHistogram(bucketScheme, Array(1L, 7, 9)), histName, seriesTags), - Seq(1574372922000L, 15d, 19d, LongHistogram(bucketScheme, Array(1L, 15, 19)), histName, seriesTags), + Seq(74372921000L, 2d, 2d, LongHistogram(bucketScheme, Array(0L, 0, 2)), histName, seriesTags), + Seq(74372921500L, 7d, 9d, LongHistogram(bucketScheme, Array(1L, 7, 9)), histName, seriesTags), + Seq(74372922000L, 15d, 19d, LongHistogram(bucketScheme, Array(1L, 15, 19)), histName, seriesTags), - Seq(1574372981000L, 17d, 21d, LongHistogram(bucketScheme, Array(2L, 16, 21)), histName, seriesTags), - Seq(1574372981500L, 1d, 1d, LongHistogram(bucketScheme, Array(0L, 1, 1)), histName, seriesTags), - Seq(1574372982000L, 15d, 15d, LongHistogram(bucketScheme, Array(0L, 15, 15)), histName, seriesTags), + Seq(74372981000L, 17d, 21d, LongHistogram(bucketScheme, Array(2L, 16, 21)), histName, seriesTags), + Seq(74372981500L, 1d, 1d, LongHistogram(bucketScheme, Array(0L, 1, 1)), histName, seriesTags), + Seq(74372982000L, 15d, 15d, LongHistogram(bucketScheme, Array(0L, 15, 15)), histName, seriesTags), - Seq(1574373041000L, 18d, 19d, LongHistogram(bucketScheme, Array(1L, 16, 19)), histName, seriesTags), - Seq(1574373042000L, 20d, 25d, LongHistogram(bucketScheme, Array(4L, 20, 25)), histName, seriesTags) + Seq(74373041000L, 18d, 19d, LongHistogram(bucketScheme, Array(1L, 16, 19)), histName, seriesTags), + Seq(74373042000L, 20d, 25d, LongHistogram(bucketScheme, Array(4L, 20, 25)), histName, seriesTags) ) MachineMetricsData.records(rawDataset, rawSamples).records.foreach { case (base, offset) => @@ -311,7 +311,7 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w val chunks = part.makeFlushChunks(offheapMem.blockMemFactory) rawColStore.write(rawDataset.ref, Observable.fromIterator(chunks)).futureValue - val pk = PartKeyRecord(histPartKeyBytes, 1574372801000L, 1574373042000L, Some(199)) + val pk = PartKeyRecord(histPartKeyBytes, 74372801000L, 74373042000L, Some(199)) rawColStore.writePartKeys(rawDataset.ref, 0, Observable.now(pk), 259200, pkUpdateHour).futureValue } @@ -415,14 +415,19 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w // time, min, max, sum, count, avg downsampledData1 shouldEqual Seq( - (1574372802000L, 3.0, 5.0, 8.0, 2.0, 4.0), - (1574372862000L, 9.0, 11.0, 20.0, 2.0, 10.0), - (1574372922000L, 13.0, 15.0, 28.0, 2.0, 14.0), - (1574372982000L, 15.0, 17.0, 32.0, 2.0, 16.0), - (1574373042000L, 11.0, 13.0, 24.0, 2.0, 12.0) + (74372802000L, 3.0, 5.0, 8.0, 2.0, 4.0), + (74372862000L, 9.0, 11.0, 20.0, 2.0, 10.0), + (74372922000L, 13.0, 15.0, 28.0, 2.0, 14.0), + (74372982000L, 15.0, 17.0, 32.0, 2.0, 16.0), + (74373042000L, 11.0, 13.0, 24.0, 2.0, 12.0) ) } + /* + Tip: After running this spec, you can bring up the local downsample server and hit following URL on browser + http://localhost:9080/promql/prometheus/api/v1/query_range?query=my_gauge%7B_ws_%3D%27my_ws%27%2C_ns_%3D%27my_ns%27%7D&start=74372801&end=74373042&step=10&verbose=true&spread=2 + */ + it("should read and verify low freq gauge in cassandra using PagedReadablePartition for 1-min downsampled data") { val dsGaugeLowFreqPartKeyBytes = RecordBuilder.buildDownsamplePartKey(gaugeLowFreqPartKeyBytes, @@ -445,9 +450,9 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w // time, min, max, sum, count, avg downsampledData1 shouldEqual Seq( - (1574372802000L, 3.0, 5.0, 8.0, 2.0, 4.0), - (1574372922000L, 13.0, 15.0, 28.0, 2.0, 14.0), - (1574373042000L, 11.0, 13.0, 24.0, 2.0, 12.0) + (74372802000L, 3.0, 5.0, 8.0, 2.0, 4.0), + (74372922000L, 13.0, 15.0, 28.0, 2.0, 14.0), + (74373042000L, 11.0, 13.0, 24.0, 2.0, 12.0) ) } @@ -474,19 +479,19 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w // time, counter downsampledData1 shouldEqual Seq( - (1574372801000L, 3d), - (1574372802000L, 5d), + (74372801000L, 3d), + (74372802000L, 5d), - (1574372862000L, 11d), + (74372862000L, 11d), - (1574372921000L, 2d), - (1574372922000L, 15d), + (74372921000L, 2d), + (74372922000L, 15d), - (1574372981000L, 17d), - (1574372981500L, 1d), - (1574372982000L, 15d), + (74372981000L, 17d), + (74372981500L, 1d), + (74372982000L, 15d), - (1574373042000L, 20d) + (74373042000L, 20d) ) } @@ -519,19 +524,19 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w // time, sum, count, histogram downsampledData1 shouldEqual Seq( - (1574372801000L, 0d, 1d, Seq(0d, 0d, 1d)), - (1574372802000L, 5d, 6d, Seq(2d, 5d, 6d)), + (74372801000L, 0d, 1d, Seq(0d, 0d, 1d)), + (74372802000L, 5d, 6d, Seq(2d, 5d, 6d)), - (1574372862000L, 11d, 14d, Seq(2d, 8d, 14d)), + (74372862000L, 11d, 14d, Seq(2d, 8d, 14d)), - (1574372921000L, 2d, 2d, Seq(0d, 0d, 2d)), - (1574372922000L, 15d, 19d, Seq(1d, 15d, 19d)), + (74372921000L, 2d, 2d, Seq(0d, 0d, 2d)), + (74372922000L, 15d, 19d, Seq(1d, 15d, 19d)), - (1574372981000L, 17d, 21d, Seq(2d, 16d, 21d)), - (1574372981500L, 1d, 1d, Seq(0d, 1d, 1d)), - (1574372982000L, 15d, 15d, Seq(0d, 15d, 15d)), + (74372981000L, 17d, 21d, Seq(2d, 16d, 21d)), + (74372981500L, 1d, 1d, Seq(0d, 1d, 1d)), + (74372982000L, 15d, 15d, Seq(0d, 15d, 15d)), - (1574373042000L, 20d, 25d, Seq(4d, 20d, 25d)) + (74373042000L, 20d, 25d, Seq(4d, 20d, 25d)) ) } @@ -555,8 +560,8 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w // time, min, max, sum, count, avg downsampledData2 shouldEqual Seq( - (1574372982000L, 3.0, 17.0, 88.0, 8.0, 11.0), - (1574373042000L, 11.0, 13.0, 24.0, 2.0, 12.0) + (74372982000L, 3.0, 17.0, 88.0, 8.0, 11.0), + (74373042000L, 11.0, 13.0, 24.0, 2.0, 12.0) ) } @@ -583,18 +588,18 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w // time, counter downsampledData1 shouldEqual Seq( - (1574372801000L, 3d), + (74372801000L, 3d), - (1574372862000L, 11d), + (74372862000L, 11d), - (1574372921000L, 2d), + (74372921000L, 2d), - (1574372981000L, 17d), - (1574372981500L, 1d), + (74372981000L, 17d), + (74372981500L, 1d), - (1574372982000L, 15.0d), + (74372982000L, 15.0d), - (1574373042000L, 20.0d) + (74373042000L, 20.0d) ) } @@ -626,13 +631,13 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w // time, sum, count, histogram downsampledData1 shouldEqual Seq( - (1574372801000L, 0d, 1d, Seq(0d, 0d, 1d)), - (1574372862000L, 11d, 14d, Seq(2d, 8d, 14d)), - (1574372921000L, 2d, 2d, Seq(0d, 0d, 2d)), - (1574372981000L, 17d, 21d, Seq(2d, 16d, 21d)), - (1574372981500L, 1d, 1d, Seq(0d, 1d, 1d)), - (1574372982000L, 15.0d, 15.0d, Seq(0.0, 15.0, 15.0)), - (1574373042000L, 20.0d, 25.0d, Seq(4.0, 20.0, 25.0)) + (74372801000L, 0d, 1d, Seq(0d, 0d, 1d)), + (74372862000L, 11d, 14d, Seq(2d, 8d, 14d)), + (74372921000L, 2d, 2d, Seq(0d, 0d, 2d)), + (74372981000L, 17d, 21d, Seq(2d, 16d, 21d)), + (74372981500L, 1d, 1d, Seq(0d, 1d, 1d)), + (74372982000L, 15.0d, 15.0d, Seq(0.0, 15.0, 15.0)), + (74373042000L, 20.0d, 25.0d, Seq(4.0, 20.0, 25.0)) ) } @@ -710,7 +715,7 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w queryScheduler.shutdown() res.result.size shouldEqual 1 res.result.head.rows.map(r => (r.getLong(0), r.getDouble(1))).toList shouldEqual - List((1574372982000L, 88.0), (1574373042000L, 24.0)) + List((74372982000L, 88.0), (74373042000L, 24.0)) } it ("should fail when cardinality buster is not configured with any delete filters") { From 191434c10f6902ff5529b684e8e2bffacf8bbb90 Mon Sep 17 00:00:00 2001 From: "Brian S. O'Neill" Date: Thu, 4 Jun 2020 16:30:19 -0700 Subject: [PATCH 25/36] feat(coordinator): Ensure that stop message is sent when state is resync'd. (#772) --- .../filodb.coordinator/IngestionActor.scala | 19 +++++++++++++++---- 1 file changed, 15 insertions(+), 4 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/IngestionActor.scala b/coordinator/src/main/scala/filodb.coordinator/IngestionActor.scala index 4ecb672c8d..7a9f21a412 100644 --- a/coordinator/src/main/scala/filodb.coordinator/IngestionActor.scala +++ b/coordinator/src/main/scala/filodb.coordinator/IngestionActor.scala @@ -109,6 +109,7 @@ private[filodb] final class IngestionActor(ref: DatasetRef, * reconciles any differences. It does so by stopping ingestion for shards that aren't mapped * to this node, and it starts ingestion for those that are. */ + // scalastyle:off method.length private def resync(state: ShardIngestionState, origin: ActorRef): Unit = { if (invalid(state.ref)) { logger.error(s"$state is invalid for this ingester '$ref'.") @@ -144,7 +145,13 @@ private[filodb] final class IngestionActor(ref: DatasetRef, } } else { val status = state.map.statuses(shard) - logger.info(s"Will stop ingestion of for dataset=$ref shard=$shard due to status ${status}") + if (shardsToStop.contains(shard)) { + logger.info(s"Will stop ingestion for dataset=$ref shard=$shard due to status ${status}") + } else { + // Already stopped. Send the message again in case it got dropped. + logger.info(s"Stopping ingestion again for dataset=$ref shard=$shard due to status ${status}") + sendStopMessage(shard) + } } } } @@ -237,9 +244,7 @@ private[filodb] final class IngestionActor(ref: DatasetRef, // Define a cancel task to run when ingestion is stopped. val onCancel = Task { logger.info(s"Ingestion cancel task invoked for dataset=$ref shard=$shard") - val stopped = IngestionStopped(ref, shard) - self ! stopped - statusActor ! stopped + sendStopMessage(shard) } val shardIngestionEnd = memStore.ingestStream(ref, @@ -268,6 +273,12 @@ private[filodb] final class IngestionActor(ref: DatasetRef, } } + private def sendStopMessage(shard: Int): Unit = { + val stopped = IngestionStopped(ref, shard) + self ! stopped + statusActor ! stopped + } + import Iterators._ /** From 08c401e5b67f4743092025c4f775b3f3ce22ce2d Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Thu, 4 Jun 2020 16:42:54 -0700 Subject: [PATCH 26/36] debug(core): More ODP logging to debug full ODP (#773) --- .../filodb.core/memstore/DemandPagedChunkStore.scala | 1 + .../filodb.core/memstore/OnDemandPagingShard.scala | 10 ++++++++-- 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/filodb.core/memstore/DemandPagedChunkStore.scala b/core/src/main/scala/filodb.core/memstore/DemandPagedChunkStore.scala index dcdd3f75a1..9c268c77af 100644 --- a/core/src/main/scala/filodb.core/memstore/DemandPagedChunkStore.scala +++ b/core/src/main/scala/filodb.core/memstore/DemandPagedChunkStore.scala @@ -69,6 +69,7 @@ extends RawToPartitionMaker with StrictLogging { FiloSchedulers.assertThreadName(FiloSchedulers.PopulateChunksSched) // Find the right partition given the partition key tsShard.getPartition(rawPartition.partitionKey).map { tsPart => + logger.debug(s"Populating paged chunks for shard=${tsShard.shardNum} partId=${tsPart.partID}") tsShard.shardStats.partitionsPagedFromColStore.increment() tsShard.shardStats.numChunksPagedIn.increment(rawPartition.chunkSets.size) // One chunkset at a time, load them into offheap and populate the partition diff --git a/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala b/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala index 454009546d..017d322f0c 100644 --- a/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala +++ b/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala @@ -143,6 +143,12 @@ TimeSeriesShard(ref, schemas, storeConfig, shardNum, bufferMemoryManager, rawSto } if (partKeyBytesToPage.nonEmpty) { val span = startODPSpan() + logger.debug(s"Preparing to page chunks for partIdsFullOdp=$partIdsNotInMemory " + + s" partIdsPartialOdp=$inMemOdp " + + s" from cass shard=$shardNum " + + s" partKeyBytesToPageSize=${partKeyBytesToPage.size}" + + s" pagingMethodsSize=${pagingMethods.size}" + + s" pagingMethods=$pagingMethods") Observable.fromIterable(partKeyBytesToPage.zip(pagingMethods)) .mapAsync(storeConfig.demandPagingParallelism) { case (partBytes, method) => rawStore.readRawPartitions(ref, maxChunkTime, SinglePartitionScan(partBytes, shardNum), method) @@ -168,13 +174,13 @@ TimeSeriesShard(ref, schemas, storeConfig, shardNum, bufferMemoryManager, rawSto // 3. Deal with partitions no longer in memory but still indexed in Lucene. // Basically we need to create TSPartitions for them in the ingest thread -- if there's enough memory private def odpPartTask(partIdsNotInMemory: Buffer[Int], partKeyBytesToPage: ArrayBuffer[Array[Byte]], - methods: ArrayBuffer[ChunkScanMethod], chunkMethod: ChunkScanMethod) = + pagingMethods: ArrayBuffer[ChunkScanMethod], chunkMethod: ChunkScanMethod) = if (partIdsNotInMemory.nonEmpty) { createODPPartitionsTask(partIdsNotInMemory, { case (pId, bytes, offset) => val partKeyBytes = if (offset == UnsafeUtils.arayOffset) bytes else BinaryRegionLarge.asNewByteArray(bytes, offset) partKeyBytesToPage += partKeyBytes - methods += chunkMethod + pagingMethods += chunkMethod logger.debug(s"Finished creating part for full odp. Now need to page partId=$pId chunkMethod=$chunkMethod") shardStats.partitionsRestored.increment() }).executeOn(ingestSched).asyncBoundary From da85df7e35bbe63ef7510eef4a15cc97e5cd63bb Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Fri, 5 Jun 2020 14:08:25 -0700 Subject: [PATCH 27/36] bug(core): Wrong PartKey Bytes was being used for Full ODP (#774) For full-odp, PK bytes is looked up using Lucene. However, BytesRef length was not being taken into consideration when formulating Part Key on-heap byte array. This fix does a copy of the array taking length into consideration. --- .../memstore/OnDemandPagingShard.scala | 24 +++++++------------ .../scala/filodb.core/metadata/Schemas.scala | 14 ++++++++--- 2 files changed, 20 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala b/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala index 017d322f0c..28d32780bf 100644 --- a/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala +++ b/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala @@ -1,5 +1,7 @@ package filodb.core.memstore +import java.util + import scala.collection.mutable.ArrayBuffer import scala.concurrent.ExecutionContext @@ -16,9 +18,7 @@ import filodb.core.downsample.{DownsampleConfig, DownsamplePublisher} import filodb.core.metadata.Schemas import filodb.core.query.QuerySession import filodb.core.store._ -import filodb.memory.BinaryRegionLarge import filodb.memory.MemFactory -import filodb.memory.format.UnsafeUtils /** * Extends TimeSeriesShard with on-demand paging functionality by populating in-memory partitions with chunks from @@ -134,7 +134,7 @@ TimeSeriesShard(ref, schemas, storeConfig, shardNum, bufferMemoryManager, rawSto } else { Observable.fromTask(odpPartTask(partIdsNotInMemory, partKeyBytesToPage, pagingMethods, partLookupRes.chunkMethod)).flatMap { odpParts => - // which thread are we on right now ? + assertThreadName(QuerySchedName) logger.debug(s"Finished creating full ODP partitions ${odpParts.map(_.partID)}") if(logger.underlying.isDebugEnabled) { partKeyBytesToPage.zip(pagingMethods).foreach { case (pk, method) => @@ -143,12 +143,6 @@ TimeSeriesShard(ref, schemas, storeConfig, shardNum, bufferMemoryManager, rawSto } if (partKeyBytesToPage.nonEmpty) { val span = startODPSpan() - logger.debug(s"Preparing to page chunks for partIdsFullOdp=$partIdsNotInMemory " + - s" partIdsPartialOdp=$inMemOdp " + - s" from cass shard=$shardNum " + - s" partKeyBytesToPageSize=${partKeyBytesToPage.size}" + - s" pagingMethodsSize=${pagingMethods.size}" + - s" pagingMethods=$pagingMethods") Observable.fromIterable(partKeyBytesToPage.zip(pagingMethods)) .mapAsync(storeConfig.demandPagingParallelism) { case (partBytes, method) => rawStore.readRawPartitions(ref, maxChunkTime, SinglePartitionScan(partBytes, shardNum), method) @@ -176,10 +170,8 @@ TimeSeriesShard(ref, schemas, storeConfig, shardNum, bufferMemoryManager, rawSto private def odpPartTask(partIdsNotInMemory: Buffer[Int], partKeyBytesToPage: ArrayBuffer[Array[Byte]], pagingMethods: ArrayBuffer[ChunkScanMethod], chunkMethod: ChunkScanMethod) = if (partIdsNotInMemory.nonEmpty) { - createODPPartitionsTask(partIdsNotInMemory, { case (pId, bytes, offset) => - val partKeyBytes = if (offset == UnsafeUtils.arayOffset) bytes - else BinaryRegionLarge.asNewByteArray(bytes, offset) - partKeyBytesToPage += partKeyBytes + createODPPartitionsTask(partIdsNotInMemory, { case (pId, pkBytes) => + partKeyBytesToPage += pkBytes pagingMethods += chunkMethod logger.debug(s"Finished creating part for full odp. Now need to page partId=$pId chunkMethod=$chunkMethod") shardStats.partitionsRestored.increment() @@ -195,7 +187,7 @@ TimeSeriesShard(ref, schemas, storeConfig, shardNum, bufferMemoryManager, rawSto * to create TSPartitions for partIDs found in Lucene but not in in-memory data structures * It runs in ingestion thread so it can correctly verify which ones to actually create or not */ - private def createODPPartitionsTask(partIDs: Buffer[Int], callback: (Int, Array[Byte], Int) => Unit): + private def createODPPartitionsTask(partIDs: Buffer[Int], callback: (Int, Array[Byte]) => Unit): Task[Seq[TimeSeriesPartition]] = Task { assertThreadName(IngestSchedName) require(partIDs.nonEmpty) @@ -218,7 +210,9 @@ TimeSeriesShard(ref, schemas, storeConfig, shardNum, bufferMemoryManager, rawSto } finally { partSetLock.unlockWrite(stamp) } - callback(part.partID, partKeyBytesRef.bytes, unsafeKeyOffset) + val pkBytes = util.Arrays.copyOfRange(partKeyBytesRef.bytes, partKeyBytesRef.offset, + partKeyBytesRef.offset + partKeyBytesRef.length) + callback(part.partID, pkBytes) part } // create the partition and update data structures (but no need to add to Lucene!) diff --git a/core/src/main/scala/filodb.core/metadata/Schemas.scala b/core/src/main/scala/filodb.core/metadata/Schemas.scala index e144941f9f..cfb245bb1a 100644 --- a/core/src/main/scala/filodb.core/metadata/Schemas.scala +++ b/core/src/main/scala/filodb.core/metadata/Schemas.scala @@ -1,6 +1,7 @@ package filodb.core.metadata import com.typesafe.config.Config +import com.typesafe.scalalogging.StrictLogging import net.ceedubs.ficus.Ficus._ import org.scalactic._ @@ -280,6 +281,12 @@ final case class Schemas(part: PartitionSchema, }.toMap } + private def bytesPerSampleSwagString = bytesPerSampleSwag.map { case e => + schemaName(e._1) + ": " + e._2 + } + + Schemas._log.info(s"bytesPerSampleSwag: $bytesPerSampleSwagString") + /** * Note this approach below assumes the following for quick size estimation. The sizing is more * a swag than reality: @@ -301,8 +308,7 @@ final case class Schemas(part: PartitionSchema, val estDataSize = bytesPerSample * numTsPartitions * numSamplesPerChunk * numChunksPerTs require(estDataSize < dataSizeLimit, s"With match of $numTsPartitions time series, estimate of $estDataSize bytes exceeds limit of " + - s"$dataSizeLimit bytes queried per shard with $bytesPerSample bytes per sample " + - s"for ${_schemas(schemaId).name} schema. Try one or more of these: " + + s"$dataSizeLimit bytes queried per shard for ${_schemas(schemaId).name} schema. Try one or more of these: " + s"(a) narrow your query filters to reduce to fewer than the current $numTsPartitions matches " + s"(b) reduce query time range, currently at ${queryDurationMs / 1000 / 60 } minutes") } @@ -336,11 +342,13 @@ final case class Schemas(part: PartitionSchema, * } * }}} */ -object Schemas { +object Schemas extends StrictLogging { import java.nio.charset.StandardCharsets.UTF_8 import Accumulation._ import Dataset._ + val _log = logger + val rowKeyIDs = Seq(0) // First or timestamp column is always the row keys val UnknownSchema = UnsafeUtils.ZeroPointer.asInstanceOf[Schema] From 819d4e113f420f0b145bfc6da6716daa4e6f9320 Mon Sep 17 00:00:00 2001 From: "Brian S. O'Neill" Date: Tue, 9 Jun 2020 09:48:50 -0700 Subject: [PATCH 28/36] fix(memory,core) Add a Latch class and use it instead of StampedLock. --- .../memstore/TimeSeriesShard.scala | 5 +- memory/src/main/java/filodb/memory/Latch.java | 1098 +++++++++++++++++ .../scala/filodb.memory/BlockManager.scala | 31 +- 3 files changed, 1115 insertions(+), 19 deletions(-) create mode 100644 memory/src/main/java/filodb/memory/Latch.java diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala index a40b90beb6..9c275ed245 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala @@ -1417,9 +1417,8 @@ class TimeSeriesShard(val ref: DatasetRef, def lookupPartitions(partMethod: PartitionScanMethod, chunkMethod: ChunkScanMethod, querySession: QuerySession): PartLookupResult = { - val reclaimReadLock = reclaimLock.asReadLock() - querySession.lock = Some(reclaimReadLock) - reclaimReadLock.lock() + querySession.lock = Some(reclaimLock) + reclaimLock.lock() // any exceptions thrown here should be caught by a wrapped Task. // At the end, MultiSchemaPartitionsExec.execute releases the lock when the task is complete partMethod match { diff --git a/memory/src/main/java/filodb/memory/Latch.java b/memory/src/main/java/filodb/memory/Latch.java new file mode 100644 index 0000000000..2a25e96874 --- /dev/null +++ b/memory/src/main/java/filodb/memory/Latch.java @@ -0,0 +1,1098 @@ +/* + * Copyright (C) 2011-2017 Cojen.org + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * This class is copied from the Tupl project and modified slightly. In particular, it doesn't + * support LatchCondition, and it doesn't use Parker. Also, modified to implement the Lock + * interface for acquiring shared locks. + */ + +package filodb.memory; + +import java.lang.invoke.MethodHandles; +import java.lang.invoke.VarHandle; + +import java.util.concurrent.TimeUnit; + +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.LockSupport; + +/** + * Non-reentrant read-write latch, designed for throughout over fairness. Implementation + * doesn't track thread ownership or check for illegal usage. As a result, it typically + * outperforms ReentrantLock and built-in Java synchronization. Although latch acquisition is + * typically unfair, waiting threads aren't starved indefinitely. + * + * @author Brian S O'Neill + */ +public class Latch implements Lock { + public static final int UNLATCHED = 0, EXCLUSIVE = 0x80000000, SHARED = 1; + + static final int SPIN_LIMIT = Runtime.getRuntime().availableProcessors() > 1 ? 1 << 10 : 1; + + static final VarHandle cStateHandle, cFirstHandle, cLastHandle, + cWaiterHandle, cWaitStateHandle, cPrevHandle, cNextHandle; + + static { + try { + cStateHandle = + MethodHandles.lookup().findVarHandle + (Latch.class, "mLatchState", int.class); + + cFirstHandle = + MethodHandles.lookup().findVarHandle + (Latch.class, "mLatchFirst", WaitNode.class); + + cLastHandle = + MethodHandles.lookup().findVarHandle + (Latch.class, "mLatchLast", WaitNode.class); + + cWaiterHandle = + MethodHandles.lookup().findVarHandle + (WaitNode.class, "mWaiter", Object.class); + + cWaitStateHandle = + MethodHandles.lookup().findVarHandle + (WaitNode.class, "mWaitState", int.class); + + cPrevHandle = + MethodHandles.lookup().findVarHandle + (WaitNode.class, "mPrev", WaitNode.class); + + cNextHandle = + MethodHandles.lookup().findVarHandle + (WaitNode.class, "mNext", WaitNode.class); + } catch (Throwable e) { + throw new Error(e); + } + } + + private static void uncaught(Throwable e) { + Thread t = Thread.currentThread(); + t.getUncaughtExceptionHandler().uncaughtException(t, e); + } + + /* + unlatched: 0 latch is available + shared: 1..0x7fffffff latch is held shared + exclusive: 0x80000000 latch is held exclusively + illegal: 0x80000001..0xffffffff illegal exclusive state + */ + volatile int mLatchState; + + // Queue of waiting threads. + private volatile WaitNode mLatchFirst; + private volatile WaitNode mLatchLast; + + public Latch() { + } + + /** + * @param initialState UNLATCHED, EXCLUSIVE, or SHARED + */ + public Latch(int initialState) { + // Assume that this latch instance is published to other threads safely, and so a + // volatile store isn't required. + cStateHandle.set(this, initialState); + } + + @Override + public void lock() { + acquireShared(); + } + + @Override + public void lockInterruptibly() throws InterruptedException { + acquireSharedInterruptibly(); + } + + @Override + public boolean tryLock() { + return tryAcquireShared(); + } + + @Override + public boolean tryLock(long time, TimeUnit unit) throws InterruptedException { + return tryAcquireSharedNanos(unit.toNanos(time)); + } + + @Override + public void unlock() { + releaseShared(); + } + + @Override + public Condition newCondition() { + throw new UnsupportedOperationException(); + } + + boolean isHeldExclusive() { + return mLatchState == EXCLUSIVE; + } + + /** + * Try to acquire the exclusive latch, barging ahead of any waiting threads if possible. + */ + public boolean tryAcquireExclusive() { + return doTryAcquireExclusive(); + } + + private boolean doTryAcquireExclusive() { + return mLatchState == 0 && cStateHandle.compareAndSet(this, 0, EXCLUSIVE); + } + + private void doAcquireExclusiveSpin() { + while (!doTryAcquireExclusive()) { + Thread.onSpinWait(); + } + } + + /** + * Attempt to acquire the exclusive latch, aborting if interrupted. + * + * @param nanosTimeout pass negative for infinite timeout + */ + public boolean tryAcquireExclusiveNanos(long nanosTimeout) throws InterruptedException { + return doTryAcquireExclusiveNanos(nanosTimeout); + } + + private boolean doTryAcquireExclusiveNanos(long nanosTimeout) throws InterruptedException { + if (doTryAcquireExclusive()) { + return true; + } + + if (nanosTimeout == 0) { + return false; + } + + boolean result; + try { + result = acquire(new Timed(nanosTimeout)); + } catch (Throwable e) { + // Possibly an OutOfMemoryError. + if (nanosTimeout < 0) { + // Caller isn't expecting an exception, so spin. + doAcquireExclusiveSpin(); + return true; + } + return false; + } + + return checkTimedResult(result, nanosTimeout); + } + + /** + * Acquire the exclusive latch, barging ahead of any waiting threads if possible. + */ + public void acquireExclusive() { + if (!doTryAcquireExclusive()) { + doAcquireExclusive(); + } + } + + /** + * Caller should have already called tryAcquireExclusive. + */ + private void doAcquireExclusive() { + try { + acquire(new WaitNode()); + } catch (Throwable e) { + // Possibly an OutOfMemoryError. Caller isn't expecting an exception, so spin. + doAcquireExclusiveSpin(); + } + } + + /** + * Acquire the exclusive latch, aborting if interrupted. + */ + public void acquireExclusiveInterruptibly() throws InterruptedException { + doTryAcquireExclusiveNanos(-1); + } + + /** + * Invokes the given continuation upon the latch being acquired exclusively. When acquired, + * the continuation is run by the current thread, or it's enqueued to be run by a thread + * which releases the latch. The releasing thread actually retains the latch and runs the + * continuation, effectively transferring latch ownership. The continuation must not + * explicitly release the latch, although it can downgrade the latch. Any exception thrown + * by the continuation is passed to the uncaught exception handler of the running thread, + * and then the latch is released. + * + * @param cont called with latch held + */ + public void uponExclusive(Runnable cont) { + if (!doTryAcquireExclusive()) enqueue: { + WaitNode node; + try { + node = new WaitNode(cont, WaitNode.SIGNALED); + } catch (Throwable e) { + // Possibly an OutOfMemoryError. Caller isn't expecting an exception, so spin. + doAcquireExclusiveSpin(); + break enqueue; + } + + WaitNode prev = enqueue(node); + + boolean acquired = doTryAcquireExclusive(); + + if (node.mWaiter == null) { + // Continuation already ran or is running right now. + if (acquired) { + releaseExclusive(); + } + return; + } + + if (!acquired) { + return; + } + + cWaiterHandle.setOpaque(node, null); + + // Acquired while still in the queue. Remove the node now, releasing memory. + if (mLatchFirst != node) { + remove(node, prev); + } else { + removeFirst(node); + } + } + + try { + cont.run(); + } catch (Throwable e) { + uncaught(e); + } + + releaseExclusive(); + } + + /** + * Downgrade the held exclusive latch into a shared latch. Caller must later call + * releaseShared instead of releaseExclusive. + */ + public final void downgrade() { + mLatchState = 1; + + while (true) { + // Sweep through the queue, waking up a contiguous run of shared waiters. + final WaitNode first = first(); + if (first == null) { + return; + } + + WaitNode node = first; + while (true) { + Object waiter = node.mWaiter; + if (waiter != null) { + if (node instanceof Shared) { + cStateHandle.getAndAdd(this, 1); + if (cWaiterHandle.compareAndSet(node, waiter, null)) { + LockSupport.unpark((Thread) waiter); + } else { + // Already unparked, so fix the share count. + cStateHandle.getAndAdd(this, -1); + } + } else { + if (node != first) { + // Advance the queue past any shared waiters that were encountered. + mLatchFirst = node; + } + return; + } + } + + WaitNode next = node.mNext; + + if (next == null) { + // Queue is now empty, unless an enqueue is in progress. + if (cLastHandle.compareAndSet(this, node, null)) { + cFirstHandle.compareAndSet(this, first, null); + return; + } + // Sweep from the start again. + break; + } + + node = next; + } + } + } + + /** + * Release the held exclusive latch. + */ + public final void releaseExclusive() { + int trials = 0; + while (true) { + WaitNode last = mLatchLast; + + if (last == null) { + // No waiters, so release the latch. + mLatchState = 0; + + // Need to check if any waiters again, due to race with enqueue. If cannot + // immediately re-acquire the latch, then let the new owner (which barged in) + // unpark the waiters when it releases the latch. + last = mLatchLast; + if (last == null || !cStateHandle.compareAndSet(this, 0, EXCLUSIVE)) { + return; + } + } + + // Although the last waiter has been observed to exist, the first waiter field + // might not be set yet. + WaitNode first = mLatchFirst; + + unpark: if (first != null) { + Object waiter = first.mWaiter; + + if (waiter != null) { + if (first instanceof Shared) { + // TODO: can this be combined into one downgrade step? + downgrade(); + if (doReleaseShared()) { + return; + } + trials = 0; + continue; + } + + if (first.mWaitState != WaitNode.SIGNALED) { + // Unpark the waiter, but allow another thread to barge in. + mLatchState = 0; + LockSupport.unpark((Thread) waiter); + return; + } + } + + // Remove first from the queue. + { + WaitNode next = first.mNext; + if (next != null) { + mLatchFirst = next; + } else { + // Queue is now empty, unless an enqueue is in progress. + if (last != first || !cLastHandle.compareAndSet(this, last, null)) { + break unpark; + } + cFirstHandle.compareAndSet(this, last, null); + } + } + + if (waiter != null && cWaiterHandle.compareAndSet(first, waiter, null)) { + // Fair handoff to waiting thread or continuation. + if (waiter instanceof Thread) { + LockSupport.unpark((Thread) waiter); + return; + } + try { + ((Runnable) waiter).run(); + } catch (Throwable e) { + uncaught(e); + } + if (!isHeldExclusive()) { + if (mLatchState <= 0) { + throw new IllegalStateException + ("Illegal latch state: " + mLatchState + ", caused by " + waiter); + } + if (doReleaseShared()) { + return; + } + } + trials = 0; + continue; + } + } + + trials = spin(trials); + } + } + + /** + * Convenience method, which releases the held exclusive or shared latch. + * + * @param exclusive call releaseExclusive if true, else call releaseShared + */ + public final void release(boolean exclusive) { + if (exclusive) { + releaseExclusive(); + } else { + releaseShared(); + } + } + + /** + * Releases an exclusive or shared latch. + */ + public final void releaseEither() { + if (((int) cStateHandle.get(this)) == EXCLUSIVE) { + releaseExclusive(); + } else { + releaseShared(); + } + } + + /** + * Try to acquire a shared latch, barging ahead of any waiting threads if possible. + */ + public boolean tryAcquireShared() { + return doTryAcquireShared(); + } + + private boolean doTryAcquireShared() { + WaitNode first = mLatchFirst; + if (first != null && !(first instanceof Shared)) { + return false; + } + int state = mLatchState; + return state >= 0 && cStateHandle.compareAndSet(this, state, state + 1); + } + + private void doAcquireSharedSpin() { + while (!doTryAcquireShared()) { + Thread.onSpinWait(); + } + } + + /** + * Attempt to acquire a shared latch, aborting if interrupted. + * + * @param nanosTimeout pass negative for infinite timeout + */ + public boolean tryAcquireSharedNanos(long nanosTimeout) throws InterruptedException { + return doTryAcquireSharedNanos(nanosTimeout); + } + + private final boolean doTryAcquireSharedNanos(long nanosTimeout) throws InterruptedException { + WaitNode first = mLatchFirst; + if (first == null || first instanceof Shared) { + int trials = 0; + int state; + while ((state = mLatchState) >= 0) { + if (cStateHandle.compareAndSet(this, state, state + 1)) { + return true; + } + // Spin even if timeout is zero. The timeout applies to a blocking acquire. + trials = spin(trials); + } + } + + if (nanosTimeout == 0) { + return false; + } + + boolean result; + try { + result = acquire(new TimedShared(nanosTimeout)); + } catch (Throwable e) { + // Possibly an OutOfMemoryError. + if (nanosTimeout < 0) { + // Caller isn't expecting an exception, so spin. + doAcquireSharedSpin(); + return true; + } + return false; + } + + return checkTimedResult(result, nanosTimeout); + } + + private static boolean checkTimedResult(boolean result, long nanosTimeout) + throws InterruptedException + { + if (!result && (Thread.interrupted() || nanosTimeout < 0)) { + InterruptedException e; + try { + e = new InterruptedException(); + } catch (Throwable e2) { + // Possibly an OutOfMemoryError. + if (nanosTimeout < 0) { + throw e2; + } + return false; + } + throw e; + } + + return result; + } + + /** + * Like tryAcquireShared, except blocks if an exclusive latch is held. + * + * @return false if not acquired due to contention with other shared requests + */ + public boolean acquireSharedUncontended() { + WaitNode first = mLatchFirst; + if (first == null || first instanceof Shared) { + int state = mLatchState; + if (state >= 0) { + return cStateHandle.compareAndSet(this, state, state + 1); + } + } + + try { + acquire(new Shared()); + } catch (Throwable e) { + // Possibly an OutOfMemoryError. Caller isn't expecting an exception, so spin. + doAcquireSharedSpin(); + } + + return true; + } + + /** + * Like tryAcquireSharedNanos, except blocks if an exclusive latch is held. + * + * @param nanosTimeout pass negative for infinite timeout + * @return -1 if not acquired due to contention with other shared requests, 0 if timed out, + * or 1 if acquired + */ + public int acquireSharedUncontendedNanos(long nanosTimeout) throws InterruptedException { + WaitNode first = mLatchFirst; + if (first == null || first instanceof Shared) { + int state = mLatchState; + if (state >= 0) { + return cStateHandle.compareAndSet(this, state, state + 1) ? 1 : -1; + } + } + + boolean result; + try { + result = acquire(new TimedShared(nanosTimeout)); + } catch (Throwable e) { + // Possibly an OutOfMemoryError. + if (nanosTimeout < 0) { + // Caller isn't expecting an exception, so spin. + doAcquireSharedSpin(); + return 1; + } + return 0; + } + + return checkTimedResult(result, nanosTimeout) ? 1 : 0; + } + + /** + * Acquire a shared latch, barging ahead of any waiting threads if possible. + */ + public void acquireShared() { + if (!tryAcquireSharedSpin()) { + try { + acquire(new Shared()); + } catch (Throwable e) { + // Possibly an OutOfMemoryError. Caller isn't expecting an exception, so spin. + doAcquireSharedSpin(); + } + } + } + + private boolean tryAcquireSharedSpin() { + WaitNode first = mLatchFirst; + if (first == null || first instanceof Shared) { + int trials = 0; + int state; + while ((state = mLatchState) >= 0) { + if (cStateHandle.compareAndSet(this, state, state + 1)) { + return true; + } + trials = spin(trials); + } + } + return false; + } + + /** + * Acquire a shared latch, aborting if interrupted. + */ + public void acquireSharedInterruptibly() throws InterruptedException { + doTryAcquireSharedNanos(-1); + } + + /** + * Attempt to upgrade a held shared latch into an exclusive latch. Upgrade fails if shared + * latch is held by more than one thread. If successful, caller must later call + * releaseExclusive instead of releaseShared. + */ + public boolean tryUpgrade() { + return doTryUpgrade(); + } + + private boolean doTryUpgrade() { + while (true) { + if (mLatchState != 1) { + return false; + } + if (cStateHandle.compareAndSet(this, 1, EXCLUSIVE)) { + return true; + } + Thread.onSpinWait(); + } + } + + /** + * Release a held shared latch. + */ + public void releaseShared() { + int trials = 0; + while (true) { + int state = mLatchState; + + WaitNode last = mLatchLast; + if (last == null) { + // No waiters, so release the latch. + if (cStateHandle.compareAndSet(this, state, --state)) { + if (state == 0) { + // Need to check if any waiters again, due to race with enqueue. If + // cannot immediately re-acquire the latch, then let the new owner + // (which barged in) unpark the waiters when it releases the latch. + last = mLatchLast; + if (last != null && cStateHandle.compareAndSet(this, 0, EXCLUSIVE)) { + releaseExclusive(); + } + } + return; + } + } else if (state == 1) { + // Try to switch to exclusive, and then let releaseExclusive deal with + // unparking the waiters. + if (cStateHandle.compareAndSet(this, 1, EXCLUSIVE) || doTryUpgrade()) { + releaseExclusive(); + return; + } + } else if (cStateHandle.compareAndSet(this, state, --state)) { + return; + } + + trials = spin(trials); + } + } + + /** + * @return false if latch is held exclusive now + */ + private boolean doReleaseShared() { + // Note: Same as regular releaseShared, except doesn't recurse into the + // releaseExclusive method. + + int trials = 0; + while (true) { + int state = mLatchState; + + WaitNode last = mLatchLast; + if (last == null) { + if (cStateHandle.compareAndSet(this, state, --state)) { + if (state == 0) { + last = mLatchLast; + if (last != null && cStateHandle.compareAndSet(this, 0, EXCLUSIVE)) { + return false; + } + } + return true; + } + } else if (state == 1) { + if (cStateHandle.compareAndSet(this, 1, EXCLUSIVE) || doTryUpgrade()) { + return false; + } + } else if (cStateHandle.compareAndSet(this, state, --state)) { + return true; + } + + trials = spin(trials); + } + } + + private boolean acquire(final WaitNode node) { + node.mWaiter = Thread.currentThread(); + WaitNode prev = enqueue(node); + int acquireResult = node.tryAcquire(this); + + if (acquireResult < 0) { + int denied = 0; + while (true) { + boolean parkAbort = node.parkNow(this); + + acquireResult = node.tryAcquire(this); + + if (acquireResult >= 0) { + // Latch acquired after parking. + break; + } + + if (parkAbort) { + if (!cWaiterHandle.compareAndSet(node, Thread.currentThread(), null)) { + // Fair handoff just occurred. + return true; + } + + // Remove the node from the queue. If it's the first, it cannot be safely + // removed without the latch having been properly acquired. So let it + // linger around until the latch is released. + if (prev != null) { + remove(node, prev); + } + + return false; + } + + // Lost the race. Request fair handoff. + + if (denied++ == 0) { + node.mWaitState = WaitNode.SIGNALED; + } + } + } + + if (acquireResult == 0) { + // Remove the node now, releasing memory. + if (mLatchFirst != node) { + remove(node, prev); + } else { + removeFirst(node); + } + } + + return true; + } + + private void removeFirst(WaitNode node) { + // Removing the first node requires special attention. Because the latch is now held by + // the current thread, no other dequeues are in progress, but enqueues still are. + + while (true) { + WaitNode next = node.mNext; + if (next != null) { + mLatchFirst = next; + return; + } else { + // Queue is now empty, unless an enqueue is in progress. + WaitNode last = mLatchLast; + if (last == node && cLastHandle.compareAndSet(this, last, null)) { + cFirstHandle.compareAndSet(this, last, null); + return; + } + } + } + } + + private WaitNode enqueue(final WaitNode node) { + var prev = (WaitNode) cLastHandle.getAndSet(this, node); + + if (prev == null) { + mLatchFirst = node; + } else { + prev.mNext = node; + WaitNode pp = prev.mPrev; + if (pp != null) { + // The old last node was intended to be removed, but the last node cannot + // be removed unless it's also the first. Bypass it now that a new last + // node has been enqueued. + cNextHandle.setRelease(pp, node); + } + } + + return prev; + } + + /** + * @param node node to remove, not null + * @param prev previous node, not null + */ + private void remove(final WaitNode node, final WaitNode prev) { + WaitNode next = node.mNext; + + if (next == null) { + // Removing the last node creates race conditions with enqueues. Instead, stash a + // reference to the previous node and let the enqueue deal with it after a new node + // has been enqueued. + node.mPrev = prev; + next = node.mNext; + // Double check in case an enqueue just occurred that may have failed to notice the + // previous node assignment. + if (next == null) { + return; + } + } + + while (next.mWaiter == null) { + // Skip more nodes if possible. + WaitNode nextNext = next.mNext; + if (nextNext == null) { + break; + } + next = nextNext; + } + + // Bypass the removed node, allowing it to be released. + cNextHandle.setRelease(prev, next); + } + + private WaitNode first() { + int trials = 0; + while (true) { + WaitNode last = mLatchLast; + if (last == null) { + return null; + } + // Although the last waiter has been observed to exist, the first waiter field + // might not be set yet. + WaitNode first = mLatchFirst; + if (first != null) { + return first; + } + trials = spin(trials); + } + } + + public final boolean hasQueuedThreads() { + return mLatchLast != null; + } + + @Override + public String toString() { + var b = new StringBuilder(); + appendMiniString(b, this); + b.append(" {state="); + + int state = mLatchState; + if (state == 0) { + b.append("unlatched"); + } else if (state == EXCLUSIVE) { + b.append("exclusive"); + } else if (state >= 0) { + b.append("shared:").append(state); + } else { + b.append("illegal:").append(state); + } + + WaitNode last = mLatchLast; + + if (last != null) { + b.append(", "); + WaitNode first = mLatchFirst; + if (first == last) { + b.append("firstQueued=").append(last); + } else if (first == null) { + b.append("lastQueued=").append(last); + } else { + b.append("firstQueued=").append(first) + .append(", lastQueued=").append(last); + } + } + + return b.append('}').toString(); + } + + static void appendMiniString(StringBuilder b, Object obj) { + if (obj == null) { + b.append("null"); + return; + } + b.append(obj.getClass().getName()).append('@').append(Integer.toHexString(obj.hashCode())); + } + + /** + * @return new trials value + */ + static int spin(int trials) { + trials++; + if (trials >= SPIN_LIMIT) { + Thread.yield(); + trials = 0; + } else { + Thread.onSpinWait(); + } + return trials; + } + + static class WaitNode { + volatile Object mWaiter; + + static final int SIGNALED = 1, COND_WAIT = 2, COND_WAIT_SHARED = 3; + volatile int mWaitState; + + // Only set if node was deleted and must be bypassed when a new node is enqueued. + volatile WaitNode mPrev; + volatile WaitNode mNext; + + /** + * Constructor for latch wait. + */ + WaitNode() { + } + + /** + * Constructor for condition wait. Caller must hold exclusive latch. + */ + WaitNode(Object waiter, int waitState) { + cWaiterHandle.set(this, waiter); + cWaitStateHandle.set(this, waitState); + } + + /** + * @return true if timed out or interrupted + */ + boolean parkNow(Latch latch) { + LockSupport.park(latch); + return false; + } + + /** + * @return {@literal <0 if thread should park; 0 if acquired and node should also be + * removed; >0 if acquired and node should not be removed} + */ + int tryAcquire(Latch latch) { + int trials = 0; + while (true) { + for (int i=0; i= SPIN_LIMIT >> 1) { + return -1; + } + // Yield to avoid parking. + Thread.yield(); + } + } + + @Override + public String toString() { + var b = new StringBuilder(); + appendMiniString(b, this); + b.append(" {waiter=").append(mWaiter); + b.append(", state=").append(mWaitState); + b.append(", next="); appendMiniString(b, mNext); + b.append(", prev="); appendMiniString(b, mPrev); + return b.append('}').toString(); + } + } + + static class Timed extends WaitNode { + private long mNanosTimeout; + private long mEndNanos; + + Timed(long nanosTimeout) { + mNanosTimeout = nanosTimeout; + if (nanosTimeout >= 0) { + mEndNanos = System.nanoTime() + nanosTimeout; + } + } + + @Override + final boolean parkNow(Latch latch) { + if (mNanosTimeout < 0) { + LockSupport.park(latch); + return Thread.currentThread().isInterrupted(); + } else { + LockSupport.parkNanos(latch, mNanosTimeout); + if (Thread.currentThread().isInterrupted()) { + return true; + } + return (mNanosTimeout = mEndNanos - System.nanoTime()) <= 0; + } + } + } + + static class Shared extends WaitNode { + /** + * @return {@literal <0 if thread should park; 0 if acquired and node should also be + * removed; >0 if acquired and node should not be removed} + */ + @Override + final int tryAcquire(Latch latch) { + // Note: If mWaiter is null, then handoff was fair. The shared count should already + // be correct, and this node won't be in the queue anymore. + + WaitNode first = latch.mLatchFirst; + if (first != null && !(first instanceof Shared)) { + return mWaiter == null ? 1 : -1; + } + + int trials = 0; + while (true) { + if (mWaiter == null) { + return 1; + } + + int state = latch.mLatchState; + if (state < 0) { + return state; + } + + if (cStateHandle.compareAndSet(latch, state, state + 1)) { + // Acquired, so no need to reference the thread anymore. + Object waiter = mWaiter; + if (waiter == null || !cWaiterHandle.compareAndSet(this, waiter, null)) { + if (!cStateHandle.compareAndSet(latch, state + 1, state)) { + cStateHandle.getAndAdd(latch, -1); + } + return 1; + } + + // Only remove node if this thread is the first shared latch owner. This + // guarantees that no other thread will be concurrently removing nodes. + // Nodes for other threads will have their nodes removed later, as latches + // are released. Early removal is a garbage collection optimization. + return state; + } + + trials = spin(trials); + } + } + } + + static class TimedShared extends Shared { + private long mNanosTimeout; + private long mEndNanos; + + TimedShared(long nanosTimeout) { + mNanosTimeout = nanosTimeout; + if (nanosTimeout >= 0) { + mEndNanos = System.nanoTime() + nanosTimeout; + } + } + + @Override + final boolean parkNow(Latch latch) { + if (mNanosTimeout < 0) { + LockSupport.park(latch); + return Thread.currentThread().isInterrupted(); + } else { + LockSupport.parkNanos(latch, mNanosTimeout); + if (Thread.currentThread().isInterrupted()) { + return true; + } + return (mNanosTimeout = mEndNanos - System.nanoTime()) <= 0; + } + } + } +} diff --git a/memory/src/main/scala/filodb.memory/BlockManager.scala b/memory/src/main/scala/filodb.memory/BlockManager.scala index fbf5f9e624..dc1e57e503 100644 --- a/memory/src/main/scala/filodb.memory/BlockManager.scala +++ b/memory/src/main/scala/filodb.memory/BlockManager.scala @@ -4,7 +4,6 @@ import java.lang.{Long => jLong} import java.util import java.util.concurrent.TimeUnit import java.util.concurrent.locks.ReentrantLock -import java.util.concurrent.locks.StampedLock import com.kenai.jffi.{MemoryIO, PageManager} import com.typesafe.scalalogging.StrictLogging @@ -157,7 +156,7 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, protected val lock = new ReentrantLock() // Acquired when reclaiming on demand. Acquire shared lock to prevent block reclamation. - final val reclaimLock = new StampedLock + final val reclaimLock = new Latch override def blockSizeInBytes: Long = PageManager.getInstance().pageSize() * numPagesPerBlock @@ -247,13 +246,13 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, */ private def tryReclaimOnDemand(num: Int): Unit = { lock.unlock() - var stamp: Long = 0 + var acquired: Boolean = false try { val start = System.nanoTime() // Give up after waiting (in total) a little over 16 seconds. - stamp = tryExclusiveReclaimLock(8192) + acquired = tryExclusiveReclaimLock(8192) - if (stamp == 0) { + if (!acquired) { // Don't stall ingestion forever. Some queries might return invalid results because // the lock isn't held. If the lock state is broken, then ingestion is really stuck // and the node must be restarted. Queries should always release the lock. @@ -273,13 +272,13 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, tryReclaim(num) } } finally { - if (stamp != 0) { - reclaimLock.unlockWrite(stamp) + if (acquired) { + reclaimLock.releaseExclusive() } } } - private def tryExclusiveReclaimLock(finalTimeoutMillis: Int): Long = { + private def tryExclusiveReclaimLock(finalTimeoutMillis: Int): Boolean = { // Attempting to acquire the exclusive lock must wait for concurrent queries to finish, but // waiting will also stall new queries from starting. To protect against this, attempt with // a timeout to let any stalled queries through. To prevent starvation of the exclusive @@ -295,17 +294,17 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, var timeout = 1; while (true) { - val stamp = reclaimLock.tryWriteLock(timeout, TimeUnit.MILLISECONDS) - if (stamp != 0) { - return stamp + val acquired = reclaimLock.tryAcquireExclusiveNanos(TimeUnit.MILLISECONDS.toNanos(timeout)) + if (acquired) { + return true } timeout <<= 1 if (timeout > finalTimeoutMillis) { - return 0 + return false } Thread.`yield`() } - 0 // never reached, but scala compiler complains otherwise + false // never reached, but scala compiler complains otherwise } /** @@ -319,15 +318,15 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, var numFree: Int = 0 val start = System.nanoTime() // Give up after waiting (in total) a little over 2 seconds. - val stamp = tryExclusiveReclaimLock(1024) - if (stamp == 0) { + val acquired = tryExclusiveReclaimLock(1024) + if (!acquired) { logger.warn(s"Lock for BlockManager.ensureFreePercent timed out: ${reclaimLock}") numFree = numFreeBlocks } else { try { numFree = ensureFreePercent(pct) } finally { - reclaimLock.unlockWrite(stamp) + reclaimLock.releaseExclusive() } val numBytes = numFree * blockSizeInBytes logger.debug(s"BlockManager.ensureFreePercent numFree: $numFree ($numBytes bytes)") From 52e621ae90fc85e9ec8db1553812db030b745aed Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Tue, 9 Jun 2020 14:10:51 -0700 Subject: [PATCH 29/36] bug(core): Kamon needs a shutdown hook to drain metrics (#780) It is possible to miss metrics from FiloDB since any shutdown may miss metrics from the last time interval. This commit adds shutdown hook to drain kamon metrics to reporters. Added to both server and spark jobs. --- .../filodb.coordinator/KamonLogger.scala | 26 +++++++++++++++++++ .../memstore/TimeSeriesShard.scala | 1 + .../downsampler/chunk/DownsamplerMain.scala | 7 +++-- .../filodb/downsampler/index/DSIndexJob.scala | 2 +- .../downsampler/index/DSIndexJobMain.scala | 5 +++- .../scala/filodb.standalone/FiloServer.scala | 1 + 6 files changed, 38 insertions(+), 4 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/KamonLogger.scala b/coordinator/src/main/scala/filodb.coordinator/KamonLogger.scala index c86b873c13..6e919c09ba 100644 --- a/coordinator/src/main/scala/filodb.coordinator/KamonLogger.scala +++ b/coordinator/src/main/scala/filodb.coordinator/KamonLogger.scala @@ -1,7 +1,10 @@ package filodb.coordinator +import scala.concurrent.Await + import com.typesafe.config.Config import com.typesafe.scalalogging.StrictLogging +import kamon.Kamon import kamon.metric.{MeasurementUnit, MetricSnapshot, PeriodSnapshot} import kamon.metric.MeasurementUnit.{information, time} import kamon.metric.MeasurementUnit.Dimension.{Information, Time} @@ -119,3 +122,26 @@ object KamonLogger { new KamonSpanLogReporter } } + +object KamonShutdownHook extends StrictLogging { + + import scala.concurrent.duration._ + + private val shutdownHookAdded = new java.util.concurrent.atomic.AtomicBoolean(false) + def registerShutdownHook(): Unit = { + if (shutdownHookAdded.compareAndSet(false, true)) { + logger.info(s"Registering Kamon Shutdown Hook...") + Runtime.getRuntime.addShutdownHook(new Thread() { + override def run(): Unit = { + logger.info(s"Stopping Kamon modules - this will ensure that last few metrics are drained") + try { + Await.result(Kamon.stopModules(), 5.minutes) + logger.info(s"Finished stopping Kamon modules") + } catch { case e: Exception => + logger.error(s"Exception when stopping Kamon Modules", e) + } + } + }) + } + } +} \ No newline at end of file diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala index a40b90beb6..0db9c8b9f5 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala @@ -1518,5 +1518,6 @@ class TimeSeriesShard(val ref: DatasetRef, method to ensure that no threads are accessing the memory before it's freed. blockStore.releaseBlocks() */ + ingestSched.shutdown() } } diff --git a/spark-jobs/src/main/scala/filodb/downsampler/chunk/DownsamplerMain.scala b/spark-jobs/src/main/scala/filodb/downsampler/chunk/DownsamplerMain.scala index 8f5a5431ab..89c6497092 100644 --- a/spark-jobs/src/main/scala/filodb/downsampler/chunk/DownsamplerMain.scala +++ b/spark-jobs/src/main/scala/filodb/downsampler/chunk/DownsamplerMain.scala @@ -7,6 +7,7 @@ import kamon.Kamon import org.apache.spark.SparkConf import org.apache.spark.sql.SparkSession +import filodb.coordinator.KamonShutdownHook import filodb.downsampler.DownsamplerContext /** @@ -100,10 +101,12 @@ class Downsampler(settings: DownsamplerSettings, batchDownsampler: BatchDownsamp DownsamplerContext.dsLogger.info(s"Cassandra split size: ${splits.size}. We will have this many spark " + s"partitions. Tune splitsPerNode which was ${settings.splitsPerNode} if parallelism is low") + KamonShutdownHook.registerShutdownHook() + spark.sparkContext .makeRDD(splits) .mapPartitions { splitIter => - Kamon.init() // kamon init should be first thing in worker jvm + KamonShutdownHook.registerShutdownHook() import filodb.core.Iterators._ val rawDataSource = batchDownsampler.rawCassandraColStore val batchReadSpan = Kamon.spanBuilder("cassandra-raw-data-read-latency").start() @@ -117,7 +120,7 @@ class Downsampler(settings: DownsamplerSettings, batchDownsampler: BatchDownsamp batchIter // iterator of batches } .foreach { rawPartsBatch => - Kamon.init() // kamon init should be first thing in worker jvm + KamonShutdownHook.registerShutdownHook() batchDownsampler.downsampleBatch(rawPartsBatch, userTimeStart, userTimeEndExclusive) } 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 96b5e205a8..3aa7830e1c 100644 --- a/spark-jobs/src/main/scala/filodb/downsampler/index/DSIndexJob.scala +++ b/spark-jobs/src/main/scala/filodb/downsampler/index/DSIndexJob.scala @@ -76,7 +76,7 @@ class DSIndexJob(dsSettings: DownsamplerSettings, shard = shard.toInt, updateHour = epochHour) count += migrateWithDownsamplePartKeys(partKeys, shard) } - DownsamplerContext.dsLogger.info(s"Successfully Completed Full PartKey Migration for shard=$shard " + + DownsamplerContext.dsLogger.info(s"Successfully Completed Partial PartKey Migration for shard=$shard " + s"count=$count fromHour=$fromHour toHourExcl=$toHourExcl") } sparkForeachTasksCompleted.increment() diff --git a/spark-jobs/src/main/scala/filodb/downsampler/index/DSIndexJobMain.scala b/spark-jobs/src/main/scala/filodb/downsampler/index/DSIndexJobMain.scala index 1200dffe4e..3de7cb6141 100644 --- a/spark-jobs/src/main/scala/filodb/downsampler/index/DSIndexJobMain.scala +++ b/spark-jobs/src/main/scala/filodb/downsampler/index/DSIndexJobMain.scala @@ -7,6 +7,7 @@ import kamon.Kamon import org.apache.spark.SparkConf import org.apache.spark.sql.SparkSession +import filodb.coordinator.KamonShutdownHook import filodb.downsampler.DownsamplerContext import filodb.downsampler.chunk.DownsamplerSettings @@ -94,13 +95,15 @@ class IndexJobDriver(dsSettings: DownsamplerSettings, dsIndexJobSettings: DSInde val numShards = dsIndexJobSettings.numShards + KamonShutdownHook.registerShutdownHook() + DownsamplerContext.dsLogger.info(s"Spark Job Properties: ${spark.sparkContext.getConf.toDebugString}") val startHour = fromHour val endHourExcl = toHourExcl spark.sparkContext .makeRDD(0 until numShards) .foreach { shard => - Kamon.init() // kamon init should be first thing in worker jvm + KamonShutdownHook.registerShutdownHook() job.updateDSPartKeyIndex(shard, startHour, endHourExcl, doFullMigration) } DownsamplerContext.dsLogger.info(s"IndexUpdater Driver completed successfully") diff --git a/standalone/src/main/scala/filodb.standalone/FiloServer.scala b/standalone/src/main/scala/filodb.standalone/FiloServer.scala index 707c8819e4..975853bc65 100644 --- a/standalone/src/main/scala/filodb.standalone/FiloServer.scala +++ b/standalone/src/main/scala/filodb.standalone/FiloServer.scala @@ -68,6 +68,7 @@ class FiloServer(watcher: Option[ActorRef]) extends FilodbClusterNode { filoHttpServer.start(coordinatorActor, singleton, bootstrapper.getAkkaHttpRoute()) // Launch the profiler after startup, if configured. SimpleProfiler.launch(systemConfig.getConfig("filodb.profiler")) + KamonShutdownHook.registerShutdownHook() } catch { // if there is an error in the initialization, we need to fail fast so that the process can be rescheduled case NonFatal(e) => From 0d9c2958a224c67109df9c1f74464b65fed006c1 Mon Sep 17 00:00:00 2001 From: "Brian S. O'Neill" Date: Wed, 10 Jun 2020 09:55:22 -0700 Subject: [PATCH 30/36] fix(memory): Latch wasn't wasn't always waking up blocked shared waiters when an exclusive request timed out. --- memory/src/main/java/filodb/memory/Latch.java | 50 +++++++++++++++++-- 1 file changed, 45 insertions(+), 5 deletions(-) diff --git a/memory/src/main/java/filodb/memory/Latch.java b/memory/src/main/java/filodb/memory/Latch.java index 2a25e96874..296854c422 100644 --- a/memory/src/main/java/filodb/memory/Latch.java +++ b/memory/src/main/java/filodb/memory/Latch.java @@ -739,6 +739,17 @@ private boolean acquire(final WaitNode node) { return true; } + if (!(node instanceof Shared) && mLatchState >= 0) { + // Unpark any shared waiters that queued behind this exclusive request. + WaitNode wnode = node; + while ((wnode = wnode.mNext) instanceof Shared) { + Object waiter = wnode.mWaiter; + if (waiter instanceof Thread) { + LockSupport.unpark((Thread) waiter); + } + } + } + // Remove the node from the queue. If it's the first, it cannot be safely // removed without the latch having been properly acquired. So let it // linger around until the latch is released. @@ -802,6 +813,10 @@ private WaitNode enqueue(final WaitNode node) { // be removed unless it's also the first. Bypass it now that a new last // node has been enqueued. cNextHandle.setRelease(pp, node); + // Return a more correct previous node, although it might be stale. Node + // removal is somewhat lazy, and accurate removal is performed when the + // exclusive latch is released. + prev = pp; } } @@ -809,6 +824,12 @@ private WaitNode enqueue(final WaitNode node) { } /** + * Should only be called after clearing the mWaiter field. Ideally, multiple threads + * shouldn't be calling this method, because it can cause nodes to be resurrected and + * remain in the queue longer than necessary. They'll get cleaned out eventually. The + * problem is caused by the prev node reference, which might have changed or have been + * removed by the time this method is called. + * * @param node node to remove, not null * @param prev previous node, not null */ @@ -858,6 +879,23 @@ private WaitNode first() { } } + /** + * Returns the first waiter in the queue that's actually still waiting. + */ + private WaitNode firstWaiter() { + WaitNode first = mLatchFirst; + WaitNode next; + if (first == null || first.mWaiter != null || (next = first.mNext) == null) { + return first; + } + if (next.mWaiter != null) { + return next; + } + // Clean out some stale nodes. Note that removing the first node isn't safe. + remove(next, first); + return null; + } + public final boolean hasQueuedThreads() { return mLatchLast != null; } @@ -1032,7 +1070,7 @@ final int tryAcquire(Latch latch) { // Note: If mWaiter is null, then handoff was fair. The shared count should already // be correct, and this node won't be in the queue anymore. - WaitNode first = latch.mLatchFirst; + WaitNode first = latch.firstWaiter(); if (first != null && !(first instanceof Shared)) { return mWaiter == null ? 1 : -1; } @@ -1058,10 +1096,12 @@ final int tryAcquire(Latch latch) { return 1; } - // Only remove node if this thread is the first shared latch owner. This - // guarantees that no other thread will be concurrently removing nodes. - // Nodes for other threads will have their nodes removed later, as latches - // are released. Early removal is a garbage collection optimization. + // Only instruct the caller to remove this node if this is the first shared + // latch owner (the returned state value will be 0). This guarantees that + // no other thread will be concurrently calling removeFirst. The node will + // be removed after an exclusive latch is released, or when firstWaiter is + // called again. Note that it's possible to return 0 every time, but only + // if the caller is also instructed to never call removeFirst. return state; } From 5d37341f62166c1f30b97396e122e272cc6b9512 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Wed, 10 Jun 2020 13:43:35 -0700 Subject: [PATCH 31/36] feat(sparkJobs): Cardinality Buster can now take time range filters (#784) * Cardinality buster could not bust cardinality for a tag filter along with time range filter. This PR adds that capability. * Also add kamon init back for spark executors --- .../PerShardCardinalityBuster.scala | 60 ++++++++++++++---- .../downsampler/chunk/DownsamplerMain.scala | 2 + .../downsampler/index/DSIndexJobMain.scala | 1 + .../downsampler/DownsamplerMainSpec.scala | 63 +++++++++++++++++-- 4 files changed, 109 insertions(+), 17 deletions(-) diff --git a/spark-jobs/src/main/scala/filodb/cardbuster/PerShardCardinalityBuster.scala b/spark-jobs/src/main/scala/filodb/cardbuster/PerShardCardinalityBuster.scala index f3fb20d796..da32df6a53 100644 --- a/spark-jobs/src/main/scala/filodb/cardbuster/PerShardCardinalityBuster.scala +++ b/spark-jobs/src/main/scala/filodb/cardbuster/PerShardCardinalityBuster.scala @@ -1,5 +1,8 @@ package filodb.cardbuster +import java.time.Instant +import java.time.format.DateTimeFormatter + import scala.concurrent.Await import kamon.Kamon @@ -37,21 +40,54 @@ class PerShardCardinalityBuster(dsSettings: DownsamplerSettings, @transient lazy val deleteFilter = dsSettings.filodbConfig .as[Seq[Map[String, String]]]("cardbuster.delete-pk-filters").map(_.toSeq) + @transient lazy val startTimeGTE = dsSettings.filodbConfig + .as[Option[String]]("cardbuster.delete-startTimeGTE").map { str => + Instant.from(DateTimeFormatter.ISO_OFFSET_DATE_TIME.parse(str)).toEpochMilli + } + @transient lazy val startTimeLTE = dsSettings.filodbConfig + .as[Option[String]]("cardbuster.delete-startTimeLTE").map { str => + Instant.from(DateTimeFormatter.ISO_OFFSET_DATE_TIME.parse(str)).toEpochMilli + } + @transient lazy val endTimeGTE = dsSettings.filodbConfig + .as[Option[String]]("cardbuster.delete-endTimeGTE").map { str => + Instant.from(DateTimeFormatter.ISO_OFFSET_DATE_TIME.parse(str)).toEpochMilli + } + @transient lazy val endTimeLTE = dsSettings.filodbConfig + .as[Option[String]]("cardbuster.delete-endTimeLTE").map { str => + Instant.from(DateTimeFormatter.ISO_OFFSET_DATE_TIME.parse(str)).toEpochMilli + } def bustIndexRecords(shard: Int): Unit = { - BusterContext.log.info(s"Busting cardinality in shard=$shard with filter=$deleteFilter") - val toDelete = colStore.scanPartKeys(dataset, shard).map(_.partKey) - .filter { pk => - val rawSchemaId = RecordSchema.schemaID(pk, UnsafeUtils.arayOffset) - val schema = schemas(rawSchemaId) - val pkPairs = schema.partKeySchema.toStringPairs(pk, UnsafeUtils.arayOffset) - val willDelete = deleteFilter.exists(filter => filter.forall(pkPairs.contains)) - if (willDelete) { - BusterContext.log.debug(s"Deleting part key ${schema.partKeySchema.stringify(pk)}") - numPartKeysDeleting.increment() + BusterContext.log.info(s"Busting cardinality in shard=$shard with " + + s"filter=$deleteFilter " + + s"inDownsampleTables=$inDownsampleTables " + + s"startTimeGTE=$startTimeGTE " + + s"startTimeLTE=$startTimeLTE " + + s"endTimeGTE=$endTimeGTE " + + s"endTimeLTE=$endTimeLTE " + ) + val toDelete = colStore.scanPartKeys(dataset, shard) + .filter { pkr => + val timeOk = startTimeGTE.forall(pkr.startTime >= _) && + startTimeLTE.forall(pkr.startTime <= _) && + endTimeGTE.forall(pkr.endTime >= _) && + endTimeLTE.forall(pkr.endTime <= _) + + if (timeOk) { + val pk = pkr.partKey + val rawSchemaId = RecordSchema.schemaID(pk, UnsafeUtils.arayOffset) + val schema = schemas(rawSchemaId) + val pkPairs = schema.partKeySchema.toStringPairs(pk, UnsafeUtils.arayOffset) + val willDelete = deleteFilter.exists(filter => filter.forall(pkPairs.contains)) + if (willDelete) { + BusterContext.log.debug(s"Deleting part key ${schema.partKeySchema.stringify(pk)}") + numPartKeysDeleting.increment() + } + willDelete + } else { + false } - willDelete - } + }.map(_.partKey) val fut = colStore.deletePartKeys(dataset, shard, toDelete) val numKeysDeleted = Await.result(fut, dsSettings.cassWriteTimeout) BusterContext.log.info(s"Deleted keys from shard shard=$shard numKeysDeleted=$numKeysDeleted") diff --git a/spark-jobs/src/main/scala/filodb/downsampler/chunk/DownsamplerMain.scala b/spark-jobs/src/main/scala/filodb/downsampler/chunk/DownsamplerMain.scala index 89c6497092..f6e8632d49 100644 --- a/spark-jobs/src/main/scala/filodb/downsampler/chunk/DownsamplerMain.scala +++ b/spark-jobs/src/main/scala/filodb/downsampler/chunk/DownsamplerMain.scala @@ -106,6 +106,7 @@ class Downsampler(settings: DownsamplerSettings, batchDownsampler: BatchDownsamp spark.sparkContext .makeRDD(splits) .mapPartitions { splitIter => + Kamon.init() KamonShutdownHook.registerShutdownHook() import filodb.core.Iterators._ val rawDataSource = batchDownsampler.rawCassandraColStore @@ -120,6 +121,7 @@ class Downsampler(settings: DownsamplerSettings, batchDownsampler: BatchDownsamp batchIter // iterator of batches } .foreach { rawPartsBatch => + Kamon.init() KamonShutdownHook.registerShutdownHook() batchDownsampler.downsampleBatch(rawPartsBatch, userTimeStart, userTimeEndExclusive) } diff --git a/spark-jobs/src/main/scala/filodb/downsampler/index/DSIndexJobMain.scala b/spark-jobs/src/main/scala/filodb/downsampler/index/DSIndexJobMain.scala index 3de7cb6141..f889599255 100644 --- a/spark-jobs/src/main/scala/filodb/downsampler/index/DSIndexJobMain.scala +++ b/spark-jobs/src/main/scala/filodb/downsampler/index/DSIndexJobMain.scala @@ -103,6 +103,7 @@ class IndexJobDriver(dsSettings: DownsamplerSettings, dsIndexJobSettings: DSInde spark.sparkContext .makeRDD(0 until numShards) .foreach { shard => + Kamon.init() KamonShutdownHook.registerShutdownHook() job.updateDSPartKeyIndex(shard, startHour, endHourExcl, doFullMigration) } diff --git a/spark-jobs/src/test/scala/filodb/downsampler/DownsamplerMainSpec.scala b/spark-jobs/src/test/scala/filodb/downsampler/DownsamplerMainSpec.scala index b02f9d6840..4d768ebae9 100644 --- a/spark-jobs/src/test/scala/filodb/downsampler/DownsamplerMainSpec.scala +++ b/spark-jobs/src/test/scala/filodb/downsampler/DownsamplerMainSpec.scala @@ -321,15 +321,16 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w start until start + 6 } - it("should simulate bulk part key records being written for migration") { + it("should simulate bulk part key records being written into raw for migration") { val partBuilder = new RecordBuilder(offheapMem.nativeMemoryManager) val schemas = Seq(Schemas.promHistogram, Schemas.gauge, Schemas.promCounter) case class PkToWrite(pkr: PartKeyRecord, shard: Int, updateHour: Long) val pks = for { i <- 0 to 10000 } yield { val schema = schemas(i % schemas.size) - val partKey = partBuilder.partKeyFromObjects(schema, s"metric$i", bulkSeriesTags) + val partKey = partBuilder.partKeyFromObjects(schema, s"bulkmetric$i", bulkSeriesTags) val bytes = schema.partKeySchema.asByteArray(UnsafeUtils.ZeroPointer, partKey) - PkToWrite(PartKeyRecord(bytes, 0L, 1000L, Some(-i)), i % numShards, bulkPkUpdateHours(i % bulkPkUpdateHours.size)) + PkToWrite(PartKeyRecord(bytes, i, i + 500, Some(-i)), i % numShards, + bulkPkUpdateHours(i % bulkPkUpdateHours.size)) } val rawDataset = Dataset("prometheus", Schemas.promHistogram) @@ -391,7 +392,7 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w }.toSet // readKeys should not contain untyped part key - we dont downsample untyped - readKeys shouldEqual (0 to 10000).map(i => s"metric$i").toSet ++ (metricNames.toSet - untypedName) + readKeys shouldEqual (0 to 10000).map(i => s"bulkmetric$i").toSet ++ (metricNames.toSet - untypedName) } it("should read and verify gauge data in cassandra using PagedReadablePartition for 1-min downsampled data") { @@ -733,7 +734,56 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w .contains("No configuration setting found for key 'cardbuster'") shouldEqual true } - it ("should be able to bust cardinality in raw and downsample tables with spark job") { + it ("should be able to bust cardinality by time filter in downsample tables with spark job") { + val sparkConf = new SparkConf(loadDefaults = true) + sparkConf.setMaster("local[2]") + val deleteFilterConfig = ConfigFactory.parseString( + s""" + |filodb.cardbuster.delete-pk-filters = [ + | { + | _ns_ = "bulk_ns" + | _ws_ = "bulk_ws" + | } + |] + |filodb.cardbuster.delete-startTimeGTE = "${Instant.ofEpochMilli(0).toString}" + |filodb.cardbuster.delete-endTimeLTE = "${Instant.ofEpochMilli(600).toString}" + |""".stripMargin) + + val settings2 = new DownsamplerSettings(deleteFilterConfig.withFallback(conf)) + val dsIndexJobSettings2 = new DSIndexJobSettings(settings2) + val cardBuster = new CardinalityBuster(settings2, dsIndexJobSettings2) + cardBuster.run(sparkConf).close() + + sparkConf.set("spark.filodb.cardbuster.inDownsampleTables", "true") + cardBuster.run(sparkConf).close() + } + + it("should verify bulk part key records are absent after card busting by time filter in downsample tables") { + + def pkMetricName(pkr: PartKeyRecord): String = { + val strPairs = batchDownsampler.schemas.part.binSchema.toStringPairs(pkr.partKey, UnsafeUtils.arayOffset) + strPairs.find(p => p._1 == "_metric_").head._2 + } + + val readKeys = (0 until 4).flatMap { shard => + val partKeys = downsampleColStore.scanPartKeys(batchDownsampler.downsampleRefsByRes(FiniteDuration(5, "min")), + shard) + Await.result(partKeys.map(pkMetricName).toListL.runAsync, 1 minutes) + }.toSet + + // downsample set should not have a few bulk metrics + readKeys.size shouldEqual 9904 + + val readKeys2 = (0 until 4).flatMap { shard => + val partKeys = rawColStore.scanPartKeys(batchDownsampler.rawDatasetRef, shard) + Await.result(partKeys.map(pkMetricName).toListL.runAsync, 1 minutes) + }.toSet + + // raw set should remain same since inDownsampleTables=true in + readKeys2.size shouldEqual 10006 + } + + it ("should be able to bust cardinality in both raw and downsample tables with spark job") { val sparkConf = new SparkConf(loadDefaults = true) sparkConf.setMaster("local[2]") val deleteFilterConfig = ConfigFactory.parseString( """ @@ -747,8 +797,11 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w val settings2 = new DownsamplerSettings(deleteFilterConfig.withFallback(conf)) val dsIndexJobSettings2 = new DSIndexJobSettings(settings2) val cardBuster = new CardinalityBuster(settings2, dsIndexJobSettings2) + + // first run for downsample tables cardBuster.run(sparkConf).close() + // then run for raw tables sparkConf.set("spark.filodb.cardbuster.inDownsampleTables", "false") cardBuster.run(sparkConf).close() } From 6838120e75831be0982c02690da840544a753c91 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Wed, 10 Jun 2020 13:52:42 -0700 Subject: [PATCH 32/36] fix(jmh,query): Fix PlanDispatcher timeout handling + QueryInMemoryBenchmark (#783) --- .../scala/filodb.coordinator/QueryActor.scala | 3 ++- .../filodb.jmh/QueryInMemoryBenchmark.scala | 15 ++++++++++---- project/FiloBuild.scala | 2 +- .../filodb/query/exec/PlanDispatcher.scala | 20 ++++++++++--------- 4 files changed, 25 insertions(+), 15 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala b/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala index 5cfcc14a21..df0cadc5ff 100644 --- a/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala +++ b/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala @@ -171,7 +171,8 @@ final class QueryActor(memStore: MemStore, // 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("Actor mailbox timeout", QueryTimeoutException(queryTimeElapsed, this.getClass.getName)) + replyTo ! QueryError(s"Query timeout, $queryTimeElapsed ms > ${queryContext.queryTimeoutMillis}", + QueryTimeoutException(queryTimeElapsed, this.getClass.getName)) false } else true } diff --git a/jmh/src/main/scala/filodb.jmh/QueryInMemoryBenchmark.scala b/jmh/src/main/scala/filodb.jmh/QueryInMemoryBenchmark.scala index 727f1a8158..2018037351 100644 --- a/jmh/src/main/scala/filodb.jmh/QueryInMemoryBenchmark.scala +++ b/jmh/src/main/scala/filodb.jmh/QueryInMemoryBenchmark.scala @@ -42,14 +42,17 @@ class QueryInMemoryBenchmark extends StrictLogging { import client.QueryCommands._ import NodeClusterActor._ - val numShards = 64 + import filodb.standalone.SimpleProfiler + val prof = new SimpleProfiler(5, 60, 50) + + val numShards = 32 val numSamples = 720 // 2 hours * 3600 / 10 sec interval val numSeries = 100 val startTime = System.currentTimeMillis - (3600*1000) val numQueries = 500 // Please make sure this number matches the OperationsPerInvocation below val queryIntervalMin = 55 // # minutes between start and stop val queryStep = 150 // # of seconds between each query sample "step" - val spread = 6 + val spread = 5 // TODO: move setup and ingestion to another trait val system = ActorSystem("test", ConfigFactory.load("filodb-defaults.conf")) @@ -104,6 +107,7 @@ class QueryInMemoryBenchmark extends StrictLogging { Thread sleep 2000 cluster.memStore.asInstanceOf[TimeSeriesMemStore].refreshIndexForTesting(dataset.ref) // commit lucene index println(s"Ingestion ended") + prof.start() // Stuff for directly executing queries ourselves val engine = new SingleClusterPlanner(dataset.ref, Schemas(dataset.schema), shardMapper, 0, @@ -124,7 +128,7 @@ class QueryInMemoryBenchmark extends StrictLogging { val qParams = TimeStepParams(queryTime/1000, queryStep, (queryTime/1000) + queryIntervalMin*60) val logicalPlans = queries.map { q => Parser.queryRangeToLogicalPlan(q, qParams) } val queryCommands = logicalPlans.map { plan => - LogicalPlan2Query(dataset.ref, plan, QueryContext(Some(new StaticSpreadProvider(SpreadChange(0, 1))), 20000)) + LogicalPlan2Query(dataset.ref, plan, QueryContext(Some(new StaticSpreadProvider(SpreadChange(0, spread))), 20000)) } var queriesSucceeded = 0 @@ -134,6 +138,7 @@ class QueryInMemoryBenchmark extends StrictLogging { def shutdownFiloActors(): Unit = { cluster.shutdown() println(s"Succeeded: $queriesSucceeded Failed: $queriesFailed") + prof.stop() } // Window = 5 min and step=2.5 min, so 50% overlap @@ -143,7 +148,9 @@ class QueryInMemoryBenchmark extends StrictLogging { @OperationsPerInvocation(500) def someOverlapQueries(): Unit = { val futures = (0 until numQueries).map { n => - val f = asyncAsk(coordinator, queryCommands(n % queryCommands.length)) + val qCmd = queryCommands(n % queryCommands.length) + val time = System.currentTimeMillis + val f = asyncAsk(coordinator, qCmd.copy(qContext = qCmd.qContext.copy(queryId = n.toString, submitTime = time))) f.onSuccess { case q: QueryResult2 => queriesSucceeded += 1 case e: QError => queriesFailed += 1 diff --git a/project/FiloBuild.scala b/project/FiloBuild.scala index bee2872a20..27989aab7b 100644 --- a/project/FiloBuild.scala +++ b/project/FiloBuild.scala @@ -155,7 +155,7 @@ object Submodules { lazy val jmh = (project in file("jmh")) .enablePlugins(JmhPlugin) - .dependsOn(core % "compile->compile; compile->test", gateway) + .dependsOn(core % "compile->compile; compile->test", gateway, standalone) .settings( commonSettings, name := "filodb-jmh", diff --git a/query/src/main/scala/filodb/query/exec/PlanDispatcher.scala b/query/src/main/scala/filodb/query/exec/PlanDispatcher.scala index 263c17eaf7..9036959e19 100644 --- a/query/src/main/scala/filodb/query/exec/PlanDispatcher.scala +++ b/query/src/main/scala/filodb/query/exec/PlanDispatcher.scala @@ -29,16 +29,18 @@ trait PlanDispatcher extends java.io.Serializable { 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 queryTimeElapsed = System.currentTimeMillis() - plan.queryContext.submitTime + val remainingTime = plan.queryContext.queryTimeoutMillis - queryTimeElapsed // Don't send if time left is very small - if (remainingTime < 1) throw QueryTimeoutException(remainingTime, this.getClass.getName) - val t = Timeout(FiniteDuration(remainingTime, TimeUnit.MILLISECONDS)) - val fut = (target ? plan)(t).map { - case resp: QueryResponse => resp - case e => throw new IllegalStateException(s"Received bad response $e") + if (remainingTime < 1) { + Task.raiseError(QueryTimeoutException(remainingTime, this.getClass.getName)) + } else { + val t = Timeout(FiniteDuration(remainingTime, TimeUnit.MILLISECONDS)) + val fut = (target ? plan)(t).map { + case resp: QueryResponse => resp + case e => throw new IllegalStateException(s"Received bad response $e") + } + Task.fromFuture(fut) } - Task.fromFuture(fut) } } From eff84f65eede3983fa9fc593fb3b3a5b4b8aa8ea Mon Sep 17 00:00:00 2001 From: "Brian S. O'Neill" Date: Tue, 9 Jun 2020 09:48:50 -0700 Subject: [PATCH 33/36] fix(memory,core) Add a Latch class and use it instead of StampedLock. --- .../memstore/TimeSeriesShard.scala | 5 +- memory/src/main/java/filodb/memory/Latch.java | 1098 +++++++++++++++++ .../scala/filodb.memory/BlockManager.scala | 31 +- 3 files changed, 1115 insertions(+), 19 deletions(-) create mode 100644 memory/src/main/java/filodb/memory/Latch.java diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala index 0db9c8b9f5..aad7a5fe2d 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala @@ -1417,9 +1417,8 @@ class TimeSeriesShard(val ref: DatasetRef, def lookupPartitions(partMethod: PartitionScanMethod, chunkMethod: ChunkScanMethod, querySession: QuerySession): PartLookupResult = { - val reclaimReadLock = reclaimLock.asReadLock() - querySession.lock = Some(reclaimReadLock) - reclaimReadLock.lock() + querySession.lock = Some(reclaimLock) + reclaimLock.lock() // any exceptions thrown here should be caught by a wrapped Task. // At the end, MultiSchemaPartitionsExec.execute releases the lock when the task is complete partMethod match { diff --git a/memory/src/main/java/filodb/memory/Latch.java b/memory/src/main/java/filodb/memory/Latch.java new file mode 100644 index 0000000000..2a25e96874 --- /dev/null +++ b/memory/src/main/java/filodb/memory/Latch.java @@ -0,0 +1,1098 @@ +/* + * Copyright (C) 2011-2017 Cojen.org + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * This class is copied from the Tupl project and modified slightly. In particular, it doesn't + * support LatchCondition, and it doesn't use Parker. Also, modified to implement the Lock + * interface for acquiring shared locks. + */ + +package filodb.memory; + +import java.lang.invoke.MethodHandles; +import java.lang.invoke.VarHandle; + +import java.util.concurrent.TimeUnit; + +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.LockSupport; + +/** + * Non-reentrant read-write latch, designed for throughout over fairness. Implementation + * doesn't track thread ownership or check for illegal usage. As a result, it typically + * outperforms ReentrantLock and built-in Java synchronization. Although latch acquisition is + * typically unfair, waiting threads aren't starved indefinitely. + * + * @author Brian S O'Neill + */ +public class Latch implements Lock { + public static final int UNLATCHED = 0, EXCLUSIVE = 0x80000000, SHARED = 1; + + static final int SPIN_LIMIT = Runtime.getRuntime().availableProcessors() > 1 ? 1 << 10 : 1; + + static final VarHandle cStateHandle, cFirstHandle, cLastHandle, + cWaiterHandle, cWaitStateHandle, cPrevHandle, cNextHandle; + + static { + try { + cStateHandle = + MethodHandles.lookup().findVarHandle + (Latch.class, "mLatchState", int.class); + + cFirstHandle = + MethodHandles.lookup().findVarHandle + (Latch.class, "mLatchFirst", WaitNode.class); + + cLastHandle = + MethodHandles.lookup().findVarHandle + (Latch.class, "mLatchLast", WaitNode.class); + + cWaiterHandle = + MethodHandles.lookup().findVarHandle + (WaitNode.class, "mWaiter", Object.class); + + cWaitStateHandle = + MethodHandles.lookup().findVarHandle + (WaitNode.class, "mWaitState", int.class); + + cPrevHandle = + MethodHandles.lookup().findVarHandle + (WaitNode.class, "mPrev", WaitNode.class); + + cNextHandle = + MethodHandles.lookup().findVarHandle + (WaitNode.class, "mNext", WaitNode.class); + } catch (Throwable e) { + throw new Error(e); + } + } + + private static void uncaught(Throwable e) { + Thread t = Thread.currentThread(); + t.getUncaughtExceptionHandler().uncaughtException(t, e); + } + + /* + unlatched: 0 latch is available + shared: 1..0x7fffffff latch is held shared + exclusive: 0x80000000 latch is held exclusively + illegal: 0x80000001..0xffffffff illegal exclusive state + */ + volatile int mLatchState; + + // Queue of waiting threads. + private volatile WaitNode mLatchFirst; + private volatile WaitNode mLatchLast; + + public Latch() { + } + + /** + * @param initialState UNLATCHED, EXCLUSIVE, or SHARED + */ + public Latch(int initialState) { + // Assume that this latch instance is published to other threads safely, and so a + // volatile store isn't required. + cStateHandle.set(this, initialState); + } + + @Override + public void lock() { + acquireShared(); + } + + @Override + public void lockInterruptibly() throws InterruptedException { + acquireSharedInterruptibly(); + } + + @Override + public boolean tryLock() { + return tryAcquireShared(); + } + + @Override + public boolean tryLock(long time, TimeUnit unit) throws InterruptedException { + return tryAcquireSharedNanos(unit.toNanos(time)); + } + + @Override + public void unlock() { + releaseShared(); + } + + @Override + public Condition newCondition() { + throw new UnsupportedOperationException(); + } + + boolean isHeldExclusive() { + return mLatchState == EXCLUSIVE; + } + + /** + * Try to acquire the exclusive latch, barging ahead of any waiting threads if possible. + */ + public boolean tryAcquireExclusive() { + return doTryAcquireExclusive(); + } + + private boolean doTryAcquireExclusive() { + return mLatchState == 0 && cStateHandle.compareAndSet(this, 0, EXCLUSIVE); + } + + private void doAcquireExclusiveSpin() { + while (!doTryAcquireExclusive()) { + Thread.onSpinWait(); + } + } + + /** + * Attempt to acquire the exclusive latch, aborting if interrupted. + * + * @param nanosTimeout pass negative for infinite timeout + */ + public boolean tryAcquireExclusiveNanos(long nanosTimeout) throws InterruptedException { + return doTryAcquireExclusiveNanos(nanosTimeout); + } + + private boolean doTryAcquireExclusiveNanos(long nanosTimeout) throws InterruptedException { + if (doTryAcquireExclusive()) { + return true; + } + + if (nanosTimeout == 0) { + return false; + } + + boolean result; + try { + result = acquire(new Timed(nanosTimeout)); + } catch (Throwable e) { + // Possibly an OutOfMemoryError. + if (nanosTimeout < 0) { + // Caller isn't expecting an exception, so spin. + doAcquireExclusiveSpin(); + return true; + } + return false; + } + + return checkTimedResult(result, nanosTimeout); + } + + /** + * Acquire the exclusive latch, barging ahead of any waiting threads if possible. + */ + public void acquireExclusive() { + if (!doTryAcquireExclusive()) { + doAcquireExclusive(); + } + } + + /** + * Caller should have already called tryAcquireExclusive. + */ + private void doAcquireExclusive() { + try { + acquire(new WaitNode()); + } catch (Throwable e) { + // Possibly an OutOfMemoryError. Caller isn't expecting an exception, so spin. + doAcquireExclusiveSpin(); + } + } + + /** + * Acquire the exclusive latch, aborting if interrupted. + */ + public void acquireExclusiveInterruptibly() throws InterruptedException { + doTryAcquireExclusiveNanos(-1); + } + + /** + * Invokes the given continuation upon the latch being acquired exclusively. When acquired, + * the continuation is run by the current thread, or it's enqueued to be run by a thread + * which releases the latch. The releasing thread actually retains the latch and runs the + * continuation, effectively transferring latch ownership. The continuation must not + * explicitly release the latch, although it can downgrade the latch. Any exception thrown + * by the continuation is passed to the uncaught exception handler of the running thread, + * and then the latch is released. + * + * @param cont called with latch held + */ + public void uponExclusive(Runnable cont) { + if (!doTryAcquireExclusive()) enqueue: { + WaitNode node; + try { + node = new WaitNode(cont, WaitNode.SIGNALED); + } catch (Throwable e) { + // Possibly an OutOfMemoryError. Caller isn't expecting an exception, so spin. + doAcquireExclusiveSpin(); + break enqueue; + } + + WaitNode prev = enqueue(node); + + boolean acquired = doTryAcquireExclusive(); + + if (node.mWaiter == null) { + // Continuation already ran or is running right now. + if (acquired) { + releaseExclusive(); + } + return; + } + + if (!acquired) { + return; + } + + cWaiterHandle.setOpaque(node, null); + + // Acquired while still in the queue. Remove the node now, releasing memory. + if (mLatchFirst != node) { + remove(node, prev); + } else { + removeFirst(node); + } + } + + try { + cont.run(); + } catch (Throwable e) { + uncaught(e); + } + + releaseExclusive(); + } + + /** + * Downgrade the held exclusive latch into a shared latch. Caller must later call + * releaseShared instead of releaseExclusive. + */ + public final void downgrade() { + mLatchState = 1; + + while (true) { + // Sweep through the queue, waking up a contiguous run of shared waiters. + final WaitNode first = first(); + if (first == null) { + return; + } + + WaitNode node = first; + while (true) { + Object waiter = node.mWaiter; + if (waiter != null) { + if (node instanceof Shared) { + cStateHandle.getAndAdd(this, 1); + if (cWaiterHandle.compareAndSet(node, waiter, null)) { + LockSupport.unpark((Thread) waiter); + } else { + // Already unparked, so fix the share count. + cStateHandle.getAndAdd(this, -1); + } + } else { + if (node != first) { + // Advance the queue past any shared waiters that were encountered. + mLatchFirst = node; + } + return; + } + } + + WaitNode next = node.mNext; + + if (next == null) { + // Queue is now empty, unless an enqueue is in progress. + if (cLastHandle.compareAndSet(this, node, null)) { + cFirstHandle.compareAndSet(this, first, null); + return; + } + // Sweep from the start again. + break; + } + + node = next; + } + } + } + + /** + * Release the held exclusive latch. + */ + public final void releaseExclusive() { + int trials = 0; + while (true) { + WaitNode last = mLatchLast; + + if (last == null) { + // No waiters, so release the latch. + mLatchState = 0; + + // Need to check if any waiters again, due to race with enqueue. If cannot + // immediately re-acquire the latch, then let the new owner (which barged in) + // unpark the waiters when it releases the latch. + last = mLatchLast; + if (last == null || !cStateHandle.compareAndSet(this, 0, EXCLUSIVE)) { + return; + } + } + + // Although the last waiter has been observed to exist, the first waiter field + // might not be set yet. + WaitNode first = mLatchFirst; + + unpark: if (first != null) { + Object waiter = first.mWaiter; + + if (waiter != null) { + if (first instanceof Shared) { + // TODO: can this be combined into one downgrade step? + downgrade(); + if (doReleaseShared()) { + return; + } + trials = 0; + continue; + } + + if (first.mWaitState != WaitNode.SIGNALED) { + // Unpark the waiter, but allow another thread to barge in. + mLatchState = 0; + LockSupport.unpark((Thread) waiter); + return; + } + } + + // Remove first from the queue. + { + WaitNode next = first.mNext; + if (next != null) { + mLatchFirst = next; + } else { + // Queue is now empty, unless an enqueue is in progress. + if (last != first || !cLastHandle.compareAndSet(this, last, null)) { + break unpark; + } + cFirstHandle.compareAndSet(this, last, null); + } + } + + if (waiter != null && cWaiterHandle.compareAndSet(first, waiter, null)) { + // Fair handoff to waiting thread or continuation. + if (waiter instanceof Thread) { + LockSupport.unpark((Thread) waiter); + return; + } + try { + ((Runnable) waiter).run(); + } catch (Throwable e) { + uncaught(e); + } + if (!isHeldExclusive()) { + if (mLatchState <= 0) { + throw new IllegalStateException + ("Illegal latch state: " + mLatchState + ", caused by " + waiter); + } + if (doReleaseShared()) { + return; + } + } + trials = 0; + continue; + } + } + + trials = spin(trials); + } + } + + /** + * Convenience method, which releases the held exclusive or shared latch. + * + * @param exclusive call releaseExclusive if true, else call releaseShared + */ + public final void release(boolean exclusive) { + if (exclusive) { + releaseExclusive(); + } else { + releaseShared(); + } + } + + /** + * Releases an exclusive or shared latch. + */ + public final void releaseEither() { + if (((int) cStateHandle.get(this)) == EXCLUSIVE) { + releaseExclusive(); + } else { + releaseShared(); + } + } + + /** + * Try to acquire a shared latch, barging ahead of any waiting threads if possible. + */ + public boolean tryAcquireShared() { + return doTryAcquireShared(); + } + + private boolean doTryAcquireShared() { + WaitNode first = mLatchFirst; + if (first != null && !(first instanceof Shared)) { + return false; + } + int state = mLatchState; + return state >= 0 && cStateHandle.compareAndSet(this, state, state + 1); + } + + private void doAcquireSharedSpin() { + while (!doTryAcquireShared()) { + Thread.onSpinWait(); + } + } + + /** + * Attempt to acquire a shared latch, aborting if interrupted. + * + * @param nanosTimeout pass negative for infinite timeout + */ + public boolean tryAcquireSharedNanos(long nanosTimeout) throws InterruptedException { + return doTryAcquireSharedNanos(nanosTimeout); + } + + private final boolean doTryAcquireSharedNanos(long nanosTimeout) throws InterruptedException { + WaitNode first = mLatchFirst; + if (first == null || first instanceof Shared) { + int trials = 0; + int state; + while ((state = mLatchState) >= 0) { + if (cStateHandle.compareAndSet(this, state, state + 1)) { + return true; + } + // Spin even if timeout is zero. The timeout applies to a blocking acquire. + trials = spin(trials); + } + } + + if (nanosTimeout == 0) { + return false; + } + + boolean result; + try { + result = acquire(new TimedShared(nanosTimeout)); + } catch (Throwable e) { + // Possibly an OutOfMemoryError. + if (nanosTimeout < 0) { + // Caller isn't expecting an exception, so spin. + doAcquireSharedSpin(); + return true; + } + return false; + } + + return checkTimedResult(result, nanosTimeout); + } + + private static boolean checkTimedResult(boolean result, long nanosTimeout) + throws InterruptedException + { + if (!result && (Thread.interrupted() || nanosTimeout < 0)) { + InterruptedException e; + try { + e = new InterruptedException(); + } catch (Throwable e2) { + // Possibly an OutOfMemoryError. + if (nanosTimeout < 0) { + throw e2; + } + return false; + } + throw e; + } + + return result; + } + + /** + * Like tryAcquireShared, except blocks if an exclusive latch is held. + * + * @return false if not acquired due to contention with other shared requests + */ + public boolean acquireSharedUncontended() { + WaitNode first = mLatchFirst; + if (first == null || first instanceof Shared) { + int state = mLatchState; + if (state >= 0) { + return cStateHandle.compareAndSet(this, state, state + 1); + } + } + + try { + acquire(new Shared()); + } catch (Throwable e) { + // Possibly an OutOfMemoryError. Caller isn't expecting an exception, so spin. + doAcquireSharedSpin(); + } + + return true; + } + + /** + * Like tryAcquireSharedNanos, except blocks if an exclusive latch is held. + * + * @param nanosTimeout pass negative for infinite timeout + * @return -1 if not acquired due to contention with other shared requests, 0 if timed out, + * or 1 if acquired + */ + public int acquireSharedUncontendedNanos(long nanosTimeout) throws InterruptedException { + WaitNode first = mLatchFirst; + if (first == null || first instanceof Shared) { + int state = mLatchState; + if (state >= 0) { + return cStateHandle.compareAndSet(this, state, state + 1) ? 1 : -1; + } + } + + boolean result; + try { + result = acquire(new TimedShared(nanosTimeout)); + } catch (Throwable e) { + // Possibly an OutOfMemoryError. + if (nanosTimeout < 0) { + // Caller isn't expecting an exception, so spin. + doAcquireSharedSpin(); + return 1; + } + return 0; + } + + return checkTimedResult(result, nanosTimeout) ? 1 : 0; + } + + /** + * Acquire a shared latch, barging ahead of any waiting threads if possible. + */ + public void acquireShared() { + if (!tryAcquireSharedSpin()) { + try { + acquire(new Shared()); + } catch (Throwable e) { + // Possibly an OutOfMemoryError. Caller isn't expecting an exception, so spin. + doAcquireSharedSpin(); + } + } + } + + private boolean tryAcquireSharedSpin() { + WaitNode first = mLatchFirst; + if (first == null || first instanceof Shared) { + int trials = 0; + int state; + while ((state = mLatchState) >= 0) { + if (cStateHandle.compareAndSet(this, state, state + 1)) { + return true; + } + trials = spin(trials); + } + } + return false; + } + + /** + * Acquire a shared latch, aborting if interrupted. + */ + public void acquireSharedInterruptibly() throws InterruptedException { + doTryAcquireSharedNanos(-1); + } + + /** + * Attempt to upgrade a held shared latch into an exclusive latch. Upgrade fails if shared + * latch is held by more than one thread. If successful, caller must later call + * releaseExclusive instead of releaseShared. + */ + public boolean tryUpgrade() { + return doTryUpgrade(); + } + + private boolean doTryUpgrade() { + while (true) { + if (mLatchState != 1) { + return false; + } + if (cStateHandle.compareAndSet(this, 1, EXCLUSIVE)) { + return true; + } + Thread.onSpinWait(); + } + } + + /** + * Release a held shared latch. + */ + public void releaseShared() { + int trials = 0; + while (true) { + int state = mLatchState; + + WaitNode last = mLatchLast; + if (last == null) { + // No waiters, so release the latch. + if (cStateHandle.compareAndSet(this, state, --state)) { + if (state == 0) { + // Need to check if any waiters again, due to race with enqueue. If + // cannot immediately re-acquire the latch, then let the new owner + // (which barged in) unpark the waiters when it releases the latch. + last = mLatchLast; + if (last != null && cStateHandle.compareAndSet(this, 0, EXCLUSIVE)) { + releaseExclusive(); + } + } + return; + } + } else if (state == 1) { + // Try to switch to exclusive, and then let releaseExclusive deal with + // unparking the waiters. + if (cStateHandle.compareAndSet(this, 1, EXCLUSIVE) || doTryUpgrade()) { + releaseExclusive(); + return; + } + } else if (cStateHandle.compareAndSet(this, state, --state)) { + return; + } + + trials = spin(trials); + } + } + + /** + * @return false if latch is held exclusive now + */ + private boolean doReleaseShared() { + // Note: Same as regular releaseShared, except doesn't recurse into the + // releaseExclusive method. + + int trials = 0; + while (true) { + int state = mLatchState; + + WaitNode last = mLatchLast; + if (last == null) { + if (cStateHandle.compareAndSet(this, state, --state)) { + if (state == 0) { + last = mLatchLast; + if (last != null && cStateHandle.compareAndSet(this, 0, EXCLUSIVE)) { + return false; + } + } + return true; + } + } else if (state == 1) { + if (cStateHandle.compareAndSet(this, 1, EXCLUSIVE) || doTryUpgrade()) { + return false; + } + } else if (cStateHandle.compareAndSet(this, state, --state)) { + return true; + } + + trials = spin(trials); + } + } + + private boolean acquire(final WaitNode node) { + node.mWaiter = Thread.currentThread(); + WaitNode prev = enqueue(node); + int acquireResult = node.tryAcquire(this); + + if (acquireResult < 0) { + int denied = 0; + while (true) { + boolean parkAbort = node.parkNow(this); + + acquireResult = node.tryAcquire(this); + + if (acquireResult >= 0) { + // Latch acquired after parking. + break; + } + + if (parkAbort) { + if (!cWaiterHandle.compareAndSet(node, Thread.currentThread(), null)) { + // Fair handoff just occurred. + return true; + } + + // Remove the node from the queue. If it's the first, it cannot be safely + // removed without the latch having been properly acquired. So let it + // linger around until the latch is released. + if (prev != null) { + remove(node, prev); + } + + return false; + } + + // Lost the race. Request fair handoff. + + if (denied++ == 0) { + node.mWaitState = WaitNode.SIGNALED; + } + } + } + + if (acquireResult == 0) { + // Remove the node now, releasing memory. + if (mLatchFirst != node) { + remove(node, prev); + } else { + removeFirst(node); + } + } + + return true; + } + + private void removeFirst(WaitNode node) { + // Removing the first node requires special attention. Because the latch is now held by + // the current thread, no other dequeues are in progress, but enqueues still are. + + while (true) { + WaitNode next = node.mNext; + if (next != null) { + mLatchFirst = next; + return; + } else { + // Queue is now empty, unless an enqueue is in progress. + WaitNode last = mLatchLast; + if (last == node && cLastHandle.compareAndSet(this, last, null)) { + cFirstHandle.compareAndSet(this, last, null); + return; + } + } + } + } + + private WaitNode enqueue(final WaitNode node) { + var prev = (WaitNode) cLastHandle.getAndSet(this, node); + + if (prev == null) { + mLatchFirst = node; + } else { + prev.mNext = node; + WaitNode pp = prev.mPrev; + if (pp != null) { + // The old last node was intended to be removed, but the last node cannot + // be removed unless it's also the first. Bypass it now that a new last + // node has been enqueued. + cNextHandle.setRelease(pp, node); + } + } + + return prev; + } + + /** + * @param node node to remove, not null + * @param prev previous node, not null + */ + private void remove(final WaitNode node, final WaitNode prev) { + WaitNode next = node.mNext; + + if (next == null) { + // Removing the last node creates race conditions with enqueues. Instead, stash a + // reference to the previous node and let the enqueue deal with it after a new node + // has been enqueued. + node.mPrev = prev; + next = node.mNext; + // Double check in case an enqueue just occurred that may have failed to notice the + // previous node assignment. + if (next == null) { + return; + } + } + + while (next.mWaiter == null) { + // Skip more nodes if possible. + WaitNode nextNext = next.mNext; + if (nextNext == null) { + break; + } + next = nextNext; + } + + // Bypass the removed node, allowing it to be released. + cNextHandle.setRelease(prev, next); + } + + private WaitNode first() { + int trials = 0; + while (true) { + WaitNode last = mLatchLast; + if (last == null) { + return null; + } + // Although the last waiter has been observed to exist, the first waiter field + // might not be set yet. + WaitNode first = mLatchFirst; + if (first != null) { + return first; + } + trials = spin(trials); + } + } + + public final boolean hasQueuedThreads() { + return mLatchLast != null; + } + + @Override + public String toString() { + var b = new StringBuilder(); + appendMiniString(b, this); + b.append(" {state="); + + int state = mLatchState; + if (state == 0) { + b.append("unlatched"); + } else if (state == EXCLUSIVE) { + b.append("exclusive"); + } else if (state >= 0) { + b.append("shared:").append(state); + } else { + b.append("illegal:").append(state); + } + + WaitNode last = mLatchLast; + + if (last != null) { + b.append(", "); + WaitNode first = mLatchFirst; + if (first == last) { + b.append("firstQueued=").append(last); + } else if (first == null) { + b.append("lastQueued=").append(last); + } else { + b.append("firstQueued=").append(first) + .append(", lastQueued=").append(last); + } + } + + return b.append('}').toString(); + } + + static void appendMiniString(StringBuilder b, Object obj) { + if (obj == null) { + b.append("null"); + return; + } + b.append(obj.getClass().getName()).append('@').append(Integer.toHexString(obj.hashCode())); + } + + /** + * @return new trials value + */ + static int spin(int trials) { + trials++; + if (trials >= SPIN_LIMIT) { + Thread.yield(); + trials = 0; + } else { + Thread.onSpinWait(); + } + return trials; + } + + static class WaitNode { + volatile Object mWaiter; + + static final int SIGNALED = 1, COND_WAIT = 2, COND_WAIT_SHARED = 3; + volatile int mWaitState; + + // Only set if node was deleted and must be bypassed when a new node is enqueued. + volatile WaitNode mPrev; + volatile WaitNode mNext; + + /** + * Constructor for latch wait. + */ + WaitNode() { + } + + /** + * Constructor for condition wait. Caller must hold exclusive latch. + */ + WaitNode(Object waiter, int waitState) { + cWaiterHandle.set(this, waiter); + cWaitStateHandle.set(this, waitState); + } + + /** + * @return true if timed out or interrupted + */ + boolean parkNow(Latch latch) { + LockSupport.park(latch); + return false; + } + + /** + * @return {@literal <0 if thread should park; 0 if acquired and node should also be + * removed; >0 if acquired and node should not be removed} + */ + int tryAcquire(Latch latch) { + int trials = 0; + while (true) { + for (int i=0; i= SPIN_LIMIT >> 1) { + return -1; + } + // Yield to avoid parking. + Thread.yield(); + } + } + + @Override + public String toString() { + var b = new StringBuilder(); + appendMiniString(b, this); + b.append(" {waiter=").append(mWaiter); + b.append(", state=").append(mWaitState); + b.append(", next="); appendMiniString(b, mNext); + b.append(", prev="); appendMiniString(b, mPrev); + return b.append('}').toString(); + } + } + + static class Timed extends WaitNode { + private long mNanosTimeout; + private long mEndNanos; + + Timed(long nanosTimeout) { + mNanosTimeout = nanosTimeout; + if (nanosTimeout >= 0) { + mEndNanos = System.nanoTime() + nanosTimeout; + } + } + + @Override + final boolean parkNow(Latch latch) { + if (mNanosTimeout < 0) { + LockSupport.park(latch); + return Thread.currentThread().isInterrupted(); + } else { + LockSupport.parkNanos(latch, mNanosTimeout); + if (Thread.currentThread().isInterrupted()) { + return true; + } + return (mNanosTimeout = mEndNanos - System.nanoTime()) <= 0; + } + } + } + + static class Shared extends WaitNode { + /** + * @return {@literal <0 if thread should park; 0 if acquired and node should also be + * removed; >0 if acquired and node should not be removed} + */ + @Override + final int tryAcquire(Latch latch) { + // Note: If mWaiter is null, then handoff was fair. The shared count should already + // be correct, and this node won't be in the queue anymore. + + WaitNode first = latch.mLatchFirst; + if (first != null && !(first instanceof Shared)) { + return mWaiter == null ? 1 : -1; + } + + int trials = 0; + while (true) { + if (mWaiter == null) { + return 1; + } + + int state = latch.mLatchState; + if (state < 0) { + return state; + } + + if (cStateHandle.compareAndSet(latch, state, state + 1)) { + // Acquired, so no need to reference the thread anymore. + Object waiter = mWaiter; + if (waiter == null || !cWaiterHandle.compareAndSet(this, waiter, null)) { + if (!cStateHandle.compareAndSet(latch, state + 1, state)) { + cStateHandle.getAndAdd(latch, -1); + } + return 1; + } + + // Only remove node if this thread is the first shared latch owner. This + // guarantees that no other thread will be concurrently removing nodes. + // Nodes for other threads will have their nodes removed later, as latches + // are released. Early removal is a garbage collection optimization. + return state; + } + + trials = spin(trials); + } + } + } + + static class TimedShared extends Shared { + private long mNanosTimeout; + private long mEndNanos; + + TimedShared(long nanosTimeout) { + mNanosTimeout = nanosTimeout; + if (nanosTimeout >= 0) { + mEndNanos = System.nanoTime() + nanosTimeout; + } + } + + @Override + final boolean parkNow(Latch latch) { + if (mNanosTimeout < 0) { + LockSupport.park(latch); + return Thread.currentThread().isInterrupted(); + } else { + LockSupport.parkNanos(latch, mNanosTimeout); + if (Thread.currentThread().isInterrupted()) { + return true; + } + return (mNanosTimeout = mEndNanos - System.nanoTime()) <= 0; + } + } + } +} diff --git a/memory/src/main/scala/filodb.memory/BlockManager.scala b/memory/src/main/scala/filodb.memory/BlockManager.scala index fbf5f9e624..dc1e57e503 100644 --- a/memory/src/main/scala/filodb.memory/BlockManager.scala +++ b/memory/src/main/scala/filodb.memory/BlockManager.scala @@ -4,7 +4,6 @@ import java.lang.{Long => jLong} import java.util import java.util.concurrent.TimeUnit import java.util.concurrent.locks.ReentrantLock -import java.util.concurrent.locks.StampedLock import com.kenai.jffi.{MemoryIO, PageManager} import com.typesafe.scalalogging.StrictLogging @@ -157,7 +156,7 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, protected val lock = new ReentrantLock() // Acquired when reclaiming on demand. Acquire shared lock to prevent block reclamation. - final val reclaimLock = new StampedLock + final val reclaimLock = new Latch override def blockSizeInBytes: Long = PageManager.getInstance().pageSize() * numPagesPerBlock @@ -247,13 +246,13 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, */ private def tryReclaimOnDemand(num: Int): Unit = { lock.unlock() - var stamp: Long = 0 + var acquired: Boolean = false try { val start = System.nanoTime() // Give up after waiting (in total) a little over 16 seconds. - stamp = tryExclusiveReclaimLock(8192) + acquired = tryExclusiveReclaimLock(8192) - if (stamp == 0) { + if (!acquired) { // Don't stall ingestion forever. Some queries might return invalid results because // the lock isn't held. If the lock state is broken, then ingestion is really stuck // and the node must be restarted. Queries should always release the lock. @@ -273,13 +272,13 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, tryReclaim(num) } } finally { - if (stamp != 0) { - reclaimLock.unlockWrite(stamp) + if (acquired) { + reclaimLock.releaseExclusive() } } } - private def tryExclusiveReclaimLock(finalTimeoutMillis: Int): Long = { + private def tryExclusiveReclaimLock(finalTimeoutMillis: Int): Boolean = { // Attempting to acquire the exclusive lock must wait for concurrent queries to finish, but // waiting will also stall new queries from starting. To protect against this, attempt with // a timeout to let any stalled queries through. To prevent starvation of the exclusive @@ -295,17 +294,17 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, var timeout = 1; while (true) { - val stamp = reclaimLock.tryWriteLock(timeout, TimeUnit.MILLISECONDS) - if (stamp != 0) { - return stamp + val acquired = reclaimLock.tryAcquireExclusiveNanos(TimeUnit.MILLISECONDS.toNanos(timeout)) + if (acquired) { + return true } timeout <<= 1 if (timeout > finalTimeoutMillis) { - return 0 + return false } Thread.`yield`() } - 0 // never reached, but scala compiler complains otherwise + false // never reached, but scala compiler complains otherwise } /** @@ -319,15 +318,15 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, var numFree: Int = 0 val start = System.nanoTime() // Give up after waiting (in total) a little over 2 seconds. - val stamp = tryExclusiveReclaimLock(1024) - if (stamp == 0) { + val acquired = tryExclusiveReclaimLock(1024) + if (!acquired) { logger.warn(s"Lock for BlockManager.ensureFreePercent timed out: ${reclaimLock}") numFree = numFreeBlocks } else { try { numFree = ensureFreePercent(pct) } finally { - reclaimLock.unlockWrite(stamp) + reclaimLock.releaseExclusive() } val numBytes = numFree * blockSizeInBytes logger.debug(s"BlockManager.ensureFreePercent numFree: $numFree ($numBytes bytes)") From be97868b05aa7b2be421a39d6da99c952c61c716 Mon Sep 17 00:00:00 2001 From: "Brian S. O'Neill" Date: Wed, 10 Jun 2020 09:55:22 -0700 Subject: [PATCH 34/36] fix(memory): Latch wasn't wasn't always waking up blocked shared waiters when an exclusive request timed out. --- memory/src/main/java/filodb/memory/Latch.java | 50 +++++++++++++++++-- 1 file changed, 45 insertions(+), 5 deletions(-) diff --git a/memory/src/main/java/filodb/memory/Latch.java b/memory/src/main/java/filodb/memory/Latch.java index 2a25e96874..296854c422 100644 --- a/memory/src/main/java/filodb/memory/Latch.java +++ b/memory/src/main/java/filodb/memory/Latch.java @@ -739,6 +739,17 @@ private boolean acquire(final WaitNode node) { return true; } + if (!(node instanceof Shared) && mLatchState >= 0) { + // Unpark any shared waiters that queued behind this exclusive request. + WaitNode wnode = node; + while ((wnode = wnode.mNext) instanceof Shared) { + Object waiter = wnode.mWaiter; + if (waiter instanceof Thread) { + LockSupport.unpark((Thread) waiter); + } + } + } + // Remove the node from the queue. If it's the first, it cannot be safely // removed without the latch having been properly acquired. So let it // linger around until the latch is released. @@ -802,6 +813,10 @@ private WaitNode enqueue(final WaitNode node) { // be removed unless it's also the first. Bypass it now that a new last // node has been enqueued. cNextHandle.setRelease(pp, node); + // Return a more correct previous node, although it might be stale. Node + // removal is somewhat lazy, and accurate removal is performed when the + // exclusive latch is released. + prev = pp; } } @@ -809,6 +824,12 @@ private WaitNode enqueue(final WaitNode node) { } /** + * Should only be called after clearing the mWaiter field. Ideally, multiple threads + * shouldn't be calling this method, because it can cause nodes to be resurrected and + * remain in the queue longer than necessary. They'll get cleaned out eventually. The + * problem is caused by the prev node reference, which might have changed or have been + * removed by the time this method is called. + * * @param node node to remove, not null * @param prev previous node, not null */ @@ -858,6 +879,23 @@ private WaitNode first() { } } + /** + * Returns the first waiter in the queue that's actually still waiting. + */ + private WaitNode firstWaiter() { + WaitNode first = mLatchFirst; + WaitNode next; + if (first == null || first.mWaiter != null || (next = first.mNext) == null) { + return first; + } + if (next.mWaiter != null) { + return next; + } + // Clean out some stale nodes. Note that removing the first node isn't safe. + remove(next, first); + return null; + } + public final boolean hasQueuedThreads() { return mLatchLast != null; } @@ -1032,7 +1070,7 @@ final int tryAcquire(Latch latch) { // Note: If mWaiter is null, then handoff was fair. The shared count should already // be correct, and this node won't be in the queue anymore. - WaitNode first = latch.mLatchFirst; + WaitNode first = latch.firstWaiter(); if (first != null && !(first instanceof Shared)) { return mWaiter == null ? 1 : -1; } @@ -1058,10 +1096,12 @@ final int tryAcquire(Latch latch) { return 1; } - // Only remove node if this thread is the first shared latch owner. This - // guarantees that no other thread will be concurrently removing nodes. - // Nodes for other threads will have their nodes removed later, as latches - // are released. Early removal is a garbage collection optimization. + // Only instruct the caller to remove this node if this is the first shared + // latch owner (the returned state value will be 0). This guarantees that + // no other thread will be concurrently calling removeFirst. The node will + // be removed after an exclusive latch is released, or when firstWaiter is + // called again. Note that it's possible to return 0 every time, but only + // if the caller is also instructed to never call removeFirst. return state; } From 8311c724bd54279e7e4e7022b88fc1611bb28fb2 Mon Sep 17 00:00:00 2001 From: TanviBhavsar Date: Fri, 12 Jun 2020 10:23:52 -0700 Subject: [PATCH 35/36] fix(query): throw exception when shardMapper has no of shards 0 and return execPlan head in materialize when only one execPlan is present (#778) --- .../queryplanner/SingleClusterPlanner.scala | 1 + .../queryplanner/SinglePartitionPlanner.scala | 6 ++++-- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala index 680ffc9b50..b75bbaaaae 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala @@ -73,6 +73,7 @@ class SingleClusterPlanner(dsRef: DatasetRef, def materialize(logicalPlan: LogicalPlan, qContext: QueryContext): ExecPlan = { + if (shardMapperFunc.numShards == 0) throw new IllegalStateException("No shards available") val materialized = walkLogicalPlanTree(logicalPlan, qContext) match { case PlanResult(Seq(justOne), stitch) => diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SinglePartitionPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SinglePartitionPlanner.scala index 0d2c8f1cdf..b00f7904f2 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SinglePartitionPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SinglePartitionPlanner.scala @@ -65,12 +65,14 @@ class SinglePartitionPlanner(planners: Map[String, QueryPlanner], plannerSelecto private def materializeLabelValues(logicalPlan: LogicalPlan, qContext: QueryContext) = { val execPlans = planners.values.toList.distinct.map(_.materialize(logicalPlan, qContext)) - LabelValuesDistConcatExec(qContext, InProcessPlanDispatcher, execPlans) + if (execPlans.size == 1) execPlans.head + else LabelValuesDistConcatExec(qContext, InProcessPlanDispatcher, execPlans) } private def materializeSeriesKeysFilters(logicalPlan: LogicalPlan, qContext: QueryContext) = { val execPlans = planners.values.toList.distinct.map(_.materialize(logicalPlan, qContext)) - PartKeysDistConcatExec(qContext, InProcessPlanDispatcher, execPlans) + if (execPlans.size == 1) execPlans.head + else PartKeysDistConcatExec(qContext, InProcessPlanDispatcher, execPlans) } } From d377b6f851ce73c4df53154251b1fdfbb398fe84 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Fri, 12 Jun 2020 12:26:00 -0700 Subject: [PATCH 36/36] feat(query): Instrument query scheduler to get metrics (#787) * Removed additional monix task due to querySession.close and folded it into QueryActor inline. * Kamon instrumentation of query scheduler for metrics --- .../scala/filodb.coordinator/QueryActor.scala | 53 ++++++++++++++++--- .../exec/MultiSchemaPartitionsExec.scala | 14 ----- 2 files changed, 47 insertions(+), 20 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala b/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala index df0cadc5ff..400a7fc0ab 100644 --- a/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala +++ b/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala @@ -1,5 +1,7 @@ package filodb.coordinator +import java.lang.Thread.UncaughtExceptionHandler +import java.util.concurrent.{ScheduledThreadPoolExecutor, ThreadFactory} import java.util.concurrent.atomic.AtomicLong import scala.util.control.NonFatal @@ -8,8 +10,10 @@ import akka.actor.{ActorRef, ActorSystem, Props} import akka.dispatch.{Envelope, UnboundedStablePriorityMailbox} import com.typesafe.config.Config import kamon.Kamon +import kamon.instrumentation.executor.ExecutorInstrumentation import kamon.tag.TagSet import monix.execution.Scheduler +import monix.execution.schedulers.SchedulerService import net.ceedubs.ficus.Ficus._ import net.ceedubs.ficus.readers.ValueReader @@ -88,8 +92,7 @@ final class QueryActor(memStore: MemStore, val queryConfig = new QueryConfig(config.getConfig("filodb.query")) val queryPlanner = new SingleClusterPlanner(dsRef, schemas, shardMapFunc, earliestRawTimestampFn, queryConfig, functionalSpreadProvider) - val numSchedThreads = Math.ceil(config.getDouble("filodb.query.threads-factor") * sys.runtime.availableProcessors) - val queryScheduler = Scheduler.fixedPool(s"$QuerySchedName-$dsRef", numSchedThreads.toInt) + val queryScheduler = createInstrumentedQueryScheduler() private val tags = Map("dataset" -> dsRef.toString) private val lpRequests = Kamon.counter("queryactor-logicalPlan-requests").withTags(TagSet.from(tags)) @@ -97,6 +100,42 @@ final class QueryActor(memStore: MemStore, private val resultVectors = Kamon.histogram("queryactor-result-num-rvs").withTags(TagSet.from(tags)) private val queryErrors = Kamon.counter("queryactor-query-errors").withTags(TagSet.from(tags)) + /** + * Instrumentation adds following metrics on the Query Scheduler + * + * # Counter + * executor_tasks_submitted_total{type="ThreadPoolExecutor",name="query-sched-prometheus"} + * # Counter + * executor_tasks_completed_total{type="ThreadPoolExecutor",name="query-sched-prometheus"} + * # Histogram + * executor_threads_active{type="ThreadPoolExecutor",name="query-sched-prometheus"} + * # Histogram + * executor_queue_size_count{type="ThreadPoolExecutor",name="query-sched-prometheus"} + * + */ + private def createInstrumentedQueryScheduler(): SchedulerService = { + val numSchedThreads = Math.ceil(config.getDouble("filodb.query.threads-factor") + * sys.runtime.availableProcessors).toInt + val schedName = s"$QuerySchedName-$dsRef" + + val thFactory = new ThreadFactory { + def newThread(r: Runnable) = { + val thread = new Thread(r) + thread.setName(s"$schedName-${thread.getId}") + thread.setDaemon(true) + thread.setUncaughtExceptionHandler( + new UncaughtExceptionHandler { + override def uncaughtException(t: Thread, e: Throwable): Unit = + logger.error("Uncaught Exception in Query Scheduler", e) + }) + thread + } + } + // TODO retaining old fixed size pool for now - later change to fork join pool. + val executor = new ScheduledThreadPoolExecutor(numSchedThreads, thFactory) + Scheduler.apply(ExecutorInstrumentation.instrument(executor, schedName)) + } + def execPhysicalPlan2(q: ExecPlan, replyTo: ActorRef): Unit = { if (checkTimeout(q.queryContext, replyTo)) { epRequests.increment() @@ -106,6 +145,7 @@ final class QueryActor(memStore: MemStore, q.execute(memStore, querySession)(queryScheduler) .foreach { res => FiloSchedulers.assertThreadName(QuerySchedName) + querySession.close() replyTo ! res res match { case QueryResult(_, _, vectors) => resultVectors.record(vectors.length) @@ -118,10 +158,11 @@ final class QueryActor(memStore: MemStore, } } }(queryScheduler).recover { case ex => - // 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) + 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) } } diff --git a/query/src/main/scala/filodb/query/exec/MultiSchemaPartitionsExec.scala b/query/src/main/scala/filodb/query/exec/MultiSchemaPartitionsExec.scala index 748fcdac18..5b5fb12be0 100644 --- a/query/src/main/scala/filodb/query/exec/MultiSchemaPartitionsExec.scala +++ b/query/src/main/scala/filodb/query/exec/MultiSchemaPartitionsExec.scala @@ -1,7 +1,6 @@ package filodb.query.exec import kamon.Kamon -import monix.eval.Task import monix.execution.Scheduler import filodb.core.{DatasetRef, QueryTimeoutException} @@ -9,7 +8,6 @@ import filodb.core.metadata.Schemas import filodb.core.query.{ColumnFilter, QueryContext, QuerySession} import filodb.core.store._ import filodb.query.Query.qLogger -import filodb.query.QueryResponse final case class UnknownSchemaQueryErr(id: Int) extends Exception(s"Unknown schema ID $id during query. This likely means a schema config change happened and " + @@ -84,18 +82,6 @@ final case class MultiSchemaPartitionsExec(queryContext: QueryContext, } } - /** - * Overridden to close the session and release locks after query executes - */ - override def execute(source: ChunkSource, - querySession: QuerySession) - (implicit sched: Scheduler): Task[QueryResponse] = { - val child = super.execute(source, querySession) - // doOnFinish handles the success and exception case. It does not handle the canceled case - child.doOnCancel(Task.now(querySession.close())) - child.doOnFinish(_ => Task.now(querySession.close())) - } - def doExecute(source: ChunkSource, querySession: QuerySession) (implicit sched: Scheduler): ExecResult = {