diff --git a/cassandra/src/main/scala/filodb.cassandra/columnstore/CassandraColumnStore.scala b/cassandra/src/main/scala/filodb.cassandra/columnstore/CassandraColumnStore.scala index d7f78128b8..d49ae65242 100644 --- a/cassandra/src/main/scala/filodb.cassandra/columnstore/CassandraColumnStore.scala +++ b/cassandra/src/main/scala/filodb.cassandra/columnstore/CassandraColumnStore.scala @@ -353,25 +353,24 @@ extends ColumnStore with CassandraChunkSource with StrictLogging { } } - private def hour(millis: Long = System.currentTimeMillis()) = millis / 1000 / 60 / 60 - def writePartKeys(ref: DatasetRef, shard: Int, partKeys: Observable[PartKeyRecord], - diskTTLSeconds: Int, + diskTTLSeconds: Int, updateHour: Long, writeToPkUTTable: Boolean = true): Future[Response] = { val pkTable = getOrCreatePartitionKeysTable(ref, shard) val pkByUTTable = getOrCreatePartitionKeysByUpdateTimeTable(ref) - val updateHour = hour() val span = Kamon.spanBuilder("write-part-keys").asChildOf(Kamon.currentSpan()).start() val ret = partKeys.mapAsync(writeParallelism) { pk => val ttl = if (pk.endTime == Long.MaxValue) -1 else diskTTLSeconds - val split = pk.hash.get % pkByUTNumSplits // caller needs to supply hash for partKey - cannot be None + // caller needs to supply hash for partKey - cannot be None + // Logical & MaxValue needed to make split positive by zeroing sign bit + val split = (pk.hash.get & Int.MaxValue) % pkByUTNumSplits val writePkFut = pkTable.writePartKey(pk, ttl).flatMap { - case resp if resp == Success - && writeToPkUTTable => pkByUTTable.writePartKey(shard, updateHour, split, pk, pkByUTTtlSeconds) - - case resp => Future.successful(resp) + case resp if resp == Success && writeToPkUTTable => + pkByUTTable.writePartKey(shard, updateHour, split, pk, pkByUTTtlSeconds) + case resp => + Future.successful(resp) } Task.fromFuture(writePkFut).map{ resp => sinkStats.partKeysWrite(1) @@ -386,11 +385,16 @@ extends ColumnStore with CassandraChunkSource with StrictLogging { shard: Int, updateHour: Long): Observable[PartKeyRecord] = { val pkByUTTable = getOrCreatePartitionKeysByUpdateTimeTable(ref) - Observable.fromIterable(0 to pkByUTNumSplits) + Observable.fromIterable(0 until pkByUTNumSplits) .flatMap { split => pkByUTTable.scanPartKeys(shard, updateHour, split) } } } +/** + * FIXME this works only for Murmur3Partitioner because it generates + * Long based tokens. If other partitioners are used, this can potentially break. + * Correct way is to pass Token objects so CQL stmts can bind tokens with stmt.bind().setPartitionKeyToken(token) + */ case class CassandraTokenRangeSplit(tokens: Seq[(String, String)], replicas: Set[InetSocketAddress]) extends ScanSplit { // NOTE: You need both the host string and the IP address for Spark's locality to work @@ -477,14 +481,14 @@ trait CassandraChunkSource extends RawChunkSource with StrictLogging { def getOrCreateIngestionTimeIndexTable(dataset: DatasetRef): IngestionTimeIndexTable = { indexTableCache.getOrElseUpdate(dataset, - { dataset: DatasetRef => - new IngestionTimeIndexTable(dataset, clusterConnector)(readEc) }) - } + { dataset: DatasetRef => + new IngestionTimeIndexTable(dataset, clusterConnector, ingestionConsistencyLevel)(readEc) }) +} def getOrCreatePartitionKeysByUpdateTimeTable(dataset: DatasetRef): PartitionKeysByUpdateTimeTable = { partKeysByUTTableCache.getOrElseUpdate(dataset, { dataset: DatasetRef => - new PartitionKeysByUpdateTimeTable(dataset, clusterConnector)(readEc) }) + new PartitionKeysByUpdateTimeTable(dataset, clusterConnector, ingestionConsistencyLevel)(readEc) }) } def getOrCreatePartitionKeysTable(dataset: DatasetRef, shard: Int): PartitionKeysTable = { diff --git a/cassandra/src/main/scala/filodb.cassandra/columnstore/IngestionTimeIndexTable.scala b/cassandra/src/main/scala/filodb.cassandra/columnstore/IngestionTimeIndexTable.scala index 4eab2b1569..a764ea9d69 100644 --- a/cassandra/src/main/scala/filodb.cassandra/columnstore/IngestionTimeIndexTable.scala +++ b/cassandra/src/main/scala/filodb.cassandra/columnstore/IngestionTimeIndexTable.scala @@ -19,7 +19,9 @@ import filodb.core.store.ChunkSinkStats * using TimeSeriesChunksTable. This is because the chunks are likely to be fetched from * Cassandra due to locality when using TimeSeriesChunksTable, and the chunks table is smaller. */ -sealed class IngestionTimeIndexTable(val dataset: DatasetRef, val connector: FiloCassandraConnector) +sealed class IngestionTimeIndexTable(val dataset: DatasetRef, + val connector: FiloCassandraConnector, + writeConsistencyLevel: ConsistencyLevel) (implicit ec: ExecutionContext) extends BaseDatasetTable { import scala.collection.JavaConverters._ @@ -36,9 +38,26 @@ sealed class IngestionTimeIndexTable(val dataset: DatasetRef, val connector: Fil |) WITH compression = { 'sstable_compression': '$sstableCompression'}""".stripMargin - lazy val allCql = session.prepare( - s"SELECT ingestion_time, start_time, info FROM $tableString " + - s" WHERE partition = ?") + private lazy val writeIndexCql = session.prepare( + s"INSERT INTO $tableString (partition, ingestion_time, start_time, info) " + + s"VALUES (?, ?, ?, ?) USING TTL ?") + .setConsistencyLevel(writeConsistencyLevel) + + private lazy val allCql = session.prepare( + s"SELECT ingestion_time, start_time, info FROM $tableString " + + s"WHERE partition = ?") + .setConsistencyLevel(ConsistencyLevel.ONE) + + private lazy val scanCql1 = session.prepare( + s"SELECT partition, ingestion_time, start_time, info FROM $tableString " + + s"WHERE TOKEN(partition) >= ? AND TOKEN(partition) < ? AND ingestion_time >= ? AND ingestion_time <= ? " + + s"ALLOW FILTERING") + .setConsistencyLevel(ConsistencyLevel.ONE) + + private lazy val scanCql2 = session.prepare( + s"SELECT partition FROM $tableString " + + s"WHERE TOKEN(partition) >= ? AND TOKEN(partition) < ? AND ingestion_time >= ? AND ingestion_time <= ? " + + s"ALLOW FILTERING") .setConsistencyLevel(ConsistencyLevel.ONE) /** @@ -63,36 +82,40 @@ sealed class IngestionTimeIndexTable(val dataset: DatasetRef, val connector: Fil def scanRowsByIngestionTimeNoAsync(tokens: Seq[(String, String)], ingestionTimeStart: Long, ingestionTimeEnd: Long): Iterator[Row] = { - def cql(start: String, end: String): String = - s"SELECT partition, ingestion_time, start_time, info FROM $tableString " + - s"WHERE TOKEN(partition) >= $start AND TOKEN(partition) < $end " + - s"AND ingestion_time >= $ingestionTimeStart AND ingestion_time <= $ingestionTimeEnd " + - s"ALLOW FILTERING" tokens.iterator.flatMap { case (start, end) => - session.execute(cql(start, end)).iterator.asScala + /* + * FIXME conversion of tokens to Long works only for Murmur3Partitioner because it generates + * Long based tokens. If other partitioners are used, this can potentially break. + * Correct way is to pass Token objects around and bind tokens with stmt.bind().setPartitionKeyToken(token) + */ + val stmt = scanCql1.bind(start.toLong: java.lang.Long, + end.toLong: java.lang.Long, + ingestionTimeStart: java.lang.Long, + ingestionTimeEnd: java.lang.Long) + session.execute(stmt).iterator.asScala } } def scanPartKeysByIngestionTime(tokens: Seq[(String, String)], ingestionTimeStart: Long, ingestionTimeEnd: Long): Observable[ByteBuffer] = { - def cql(start: String, end: String): String = - s"SELECT partition FROM $tableString WHERE TOKEN(partition) >= $start AND TOKEN(partition) < $end " + - s"AND ingestion_time >= $ingestionTimeStart AND ingestion_time <= $ingestionTimeEnd " + - s"ALLOW FILTERING" val it = tokens.iterator.flatMap { case (start, end) => - session.execute(cql(start, end)).iterator.asScala + /* + * FIXME conversion of tokens to Long works only for Murmur3Partitioner because it generates + * Long based tokens. If other partitioners are used, this can potentially break. + * Correct way is to pass Token objects around and bind tokens with stmt.bind().setPartitionKeyToken(token) + */ + val stmt = scanCql2.bind(start.toLong: java.lang.Long, + end.toLong: java.lang.Long, + ingestionTimeStart: java.lang.Long, + ingestionTimeEnd: java.lang.Long) + session.execute(stmt).iterator.asScala .map { row => row.getBytes("partition") } } Observable.fromIterator(it).handleObservableErrors } - lazy val writeIndexCql = session.prepare( - s"INSERT INTO $tableString (partition, ingestion_time, start_time, info) " + - "VALUES (?, ?, ?, ?) USING TTL ?") - .setConsistencyLevel(ConsistencyLevel.ONE) - /** * Writes new records to the ingestion table. * diff --git a/cassandra/src/main/scala/filodb.cassandra/columnstore/PartitionKeysByUpdateTimeTable.scala b/cassandra/src/main/scala/filodb.cassandra/columnstore/PartitionKeysByUpdateTimeTable.scala index 4a0940c5b1..5a42ec38c4 100644 --- a/cassandra/src/main/scala/filodb.cassandra/columnstore/PartitionKeysByUpdateTimeTable.scala +++ b/cassandra/src/main/scala/filodb.cassandra/columnstore/PartitionKeysByUpdateTimeTable.scala @@ -12,7 +12,8 @@ import filodb.core.{DatasetRef, Response} import filodb.core.store.PartKeyRecord sealed class PartitionKeysByUpdateTimeTable(val dataset: DatasetRef, - val connector: FiloCassandraConnector) + val connector: FiloCassandraConnector, + writeConsistencyLevel: ConsistencyLevel) (implicit ec: ExecutionContext) extends BaseDatasetTable { import filodb.cassandra.Util._ @@ -32,11 +33,16 @@ sealed class PartitionKeysByUpdateTimeTable(val dataset: DatasetRef, | WITH compression = {'chunk_length_in_kb': '16', 'sstable_compression': '$sstableCompression'}""".stripMargin // TODO time window compaction since we have time component in the primary key - lazy val writePartitionKeyCql = - session.prepare( - s"INSERT INTO ${tableString} (shard, epochHour, split, partKey, startTime, endTime) " + - s"VALUES (?, ?, ?, ?, ?, ?) USING TTL ?") - .setConsistencyLevel(ConsistencyLevel.ONE) + private lazy val writePartitionKeyCql = session.prepare( + s"INSERT INTO $tableString (shard, epochHour, split, partKey, startTime, endTime) " + + s"VALUES (?, ?, ?, ?, ?, ?) USING TTL ?") + .setConsistencyLevel(writeConsistencyLevel) + + private lazy val readCql = session.prepare( + s"SELECT * FROM $tableString " + + s"WHERE shard = ? AND epochHour = ? AND split = ? ") + .setConsistencyLevel(ConsistencyLevel.ONE) + def writePartKey(shard: Int, updateHour: Long, split: Int, pk: PartKeyRecord, ttlSeconds: Int): Future[Response] = { @@ -45,8 +51,6 @@ sealed class PartitionKeysByUpdateTimeTable(val dataset: DatasetRef, toBuffer(pk.partKey), pk.startTime: JLong, pk.endTime: JLong, ttlSeconds: JInt)) } - lazy val readCql = session.prepare(s"SELECT * FROM $tableString " + - s"WHERE shard = ? AND epochHour = ? AND split = ? ") def scanPartKeys(shard: Int, updateHour: Long, split: Int): Observable[PartKeyRecord] = { session.executeAsync(readCql.bind(shard: JInt, updateHour: JLong, split: JInt)) .toObservable.handleObservableErrors diff --git a/cassandra/src/main/scala/filodb.cassandra/columnstore/PartitionKeysTable.scala b/cassandra/src/main/scala/filodb.cassandra/columnstore/PartitionKeysTable.scala index 783b56c180..0846df621a 100644 --- a/cassandra/src/main/scala/filodb.cassandra/columnstore/PartitionKeysTable.scala +++ b/cassandra/src/main/scala/filodb.cassandra/columnstore/PartitionKeysTable.scala @@ -30,16 +30,21 @@ sealed class PartitionKeysTable(val dataset: DatasetRef, | PRIMARY KEY (partKey) |) WITH compression = {'chunk_length_in_kb': '16', 'sstable_compression': '$sstableCompression'}""".stripMargin - lazy val writePartitionCql = - session.prepare( - s"INSERT INTO ${tableString} (partKey, startTime, endTime) VALUES (?, ?, ?) USING TTL ?") + private lazy val writePartitionCql = session.prepare( + s"INSERT INTO ${tableString} (partKey, startTime, endTime) " + + s"VALUES (?, ?, ?) USING TTL ?") .setConsistencyLevel(writeConsistencyLevel) - lazy val writePartitionCqlNoTtl = - session.prepare( - s"INSERT INTO ${tableString} (partKey, startTime, endTime) VALUES (?, ?, ?)") + private lazy val writePartitionCqlNoTtl = session.prepare( + s"INSERT INTO ${tableString} (partKey, startTime, endTime) " + + s"VALUES (?, ?, ?)") .setConsistencyLevel(writeConsistencyLevel) + private lazy val scanCql = session.prepare( + s"SELECT * FROM $tableString " + + s"WHERE TOKEN(partKey) >= ? AND TOKEN(partKey) < ?") + .setConsistencyLevel(ConsistencyLevel.ONE) + def writePartKey(pk: PartKeyRecord, diskTimeToLiveSeconds: Int): Future[Response] = { if (diskTimeToLiveSeconds <= 0) { connector.execStmtWithRetries(writePartitionCqlNoTtl.bind( @@ -50,8 +55,6 @@ sealed class PartitionKeysTable(val dataset: DatasetRef, } } - lazy val scanCql = session.prepare(s"SELECT * FROM $tableString " + - s"WHERE TOKEN(partKey) >= ? AND TOKEN(partKey) < ?") def scanPartKeys(tokens: Seq[(String, String)], shard: Int): Observable[PartKeyRecord] = { val res: Observable[Iterator[PartKeyRecord]] = Observable.fromIterable(tokens) .mapAsync { range => diff --git a/cassandra/src/main/scala/filodb.cassandra/columnstore/TimeSeriesChunksTable.scala b/cassandra/src/main/scala/filodb.cassandra/columnstore/TimeSeriesChunksTable.scala index d6a474adf1..cb5d0236f6 100644 --- a/cassandra/src/main/scala/filodb.cassandra/columnstore/TimeSeriesChunksTable.scala +++ b/cassandra/src/main/scala/filodb.cassandra/columnstore/TimeSeriesChunksTable.scala @@ -40,10 +40,36 @@ sealed class TimeSeriesChunksTable(val dataset: DatasetRef, |) WITH compression = { 'sstable_compression': '$sstableCompression'}""".stripMargin - lazy val writeChunksCql = session.prepare( - s"""INSERT INTO $tableString (partition, chunkid, info, chunks - |) VALUES (?, ?, ?, ?) USING TTL ?""".stripMargin - ).setConsistencyLevel(writeConsistencyLevel) + private lazy val writeChunksCql = session.prepare( + s"INSERT INTO $tableString (partition, chunkid, info, chunks) " + + s"VALUES (?, ?, ?, ?) USING TTL ?") + .setConsistencyLevel(writeConsistencyLevel) + + private lazy val readChunkInCql = session.prepare( + s"SELECT info, chunks FROM $tableString " + + s"WHERE partition = ? AND chunkid IN ?") + .setConsistencyLevel(ConsistencyLevel.ONE) + + private lazy val readChunksCql = session.prepare( + s"SELECT chunkid, info, chunks FROM $tableString " + + s"WHERE partition = ? AND chunkid IN ?") + .setConsistencyLevel(ConsistencyLevel.ONE) + + private lazy val readAllChunksCql = session.prepare( + s"SELECT chunkid, info, chunks FROM $tableString " + + s"WHERE partition = ?") + .setConsistencyLevel(ConsistencyLevel.ONE) + + private lazy val scanBySplit = session.prepare( + s"SELECT partition, info, chunks FROM $tableString " + + s"WHERE TOKEN(partition) >= ? AND TOKEN(partition) < ?") + .setConsistencyLevel(ConsistencyLevel.ONE) + + private lazy val readChunkRangeCql = session.prepare( + s"SELECT partition, info, chunks FROM $tableString " + + s"WHERE partition IN ? AND chunkid >= ? AND chunkid < ?") + .setConsistencyLevel(ConsistencyLevel.ONE) + def writeChunks(partition: Array[Byte], chunkInfo: ChunkSetInfo, @@ -81,12 +107,6 @@ sealed class TimeSeriesChunksTable(val dataset: DatasetRef, ) } - lazy val readChunkInCql = session.prepare( - s"""SELECT info, chunks FROM $tableString WHERE - | partition = ? - | AND chunkid IN ?""".stripMargin) - .setConsistencyLevel(ConsistencyLevel.ONE) - /** * Reads and returns a single RawPartData, raw data for a single partition/time series */ @@ -109,10 +129,6 @@ sealed class TimeSeriesChunksTable(val dataset: DatasetRef, RawChunkSet(row.getBytes(infoIndex).array, chunks) } - lazy val readChunksCql = session.prepare( - s"SELECT chunkid, info, chunks FROM $tableString WHERE partition = ? AND chunkid IN ?") - .setConsistencyLevel(ConsistencyLevel.ONE) - /** * Reads raw chunk set Rows consisting of: * @@ -129,10 +145,6 @@ sealed class TimeSeriesChunksTable(val dataset: DatasetRef, session.execute(query) } - lazy val readAllChunksCql = session.prepare( - s"SELECT chunkid, info, chunks FROM $tableString WHERE partition = ?") - .setConsistencyLevel(ConsistencyLevel.ONE) - /** * Test method which returns the same results as the readChunks method. Not async-friendly. */ @@ -143,12 +155,6 @@ sealed class TimeSeriesChunksTable(val dataset: DatasetRef, /** * Reads and returns a stream of RawPartDatas given a range of chunkIDs from multiple partitions */ - lazy val readChunkRangeCql = session.prepare( - s"""SELECT partition, info, chunks FROM $tableString WHERE - | partition IN ? - | AND chunkid >= ? AND chunkid < ?""".stripMargin) - .setConsistencyLevel(ConsistencyLevel.ONE) - def readRawPartitionRange(partitions: Seq[Array[Byte]], startTime: Long, endTimeExclusive: Long): Observable[RawPartData] = { @@ -177,10 +183,15 @@ sealed class TimeSeriesChunksTable(val dataset: DatasetRef, } def scanPartitionsBySplit(tokens: Seq[(String, String)]): Observable[RawPartData] = { - def cql(start: String, end: String): String = - s"SELECT partition, info, chunks FROM $tableString WHERE TOKEN(partition) >= $start AND TOKEN(partition) < $end " + val res: Observable[Future[Iterator[RawPartData]]] = Observable.fromIterable(tokens).map { case (start, end) => - session.executeAsync(cql(start, end)).toIterator.handleErrors + /* + * FIXME conversion of tokens to Long works only for Murmur3Partitioner because it generates + * Long based tokens. If other partitioners are used, this can potentially break. + * Correct way to bind tokens is to do stmt.bind().setPartitionKeyToken(token) + */ + val stmt = scanBySplit.bind(start.toLong: java.lang.Long, end.toLong: java.lang.Long) + session.executeAsync(stmt).toIterator.handleErrors .map { rowIt => rowIt.map { row => (row.getBytes(0), chunkSetFromRow(row, 1)) } .sortedGroupBy(_._1) diff --git a/cassandra/src/test/scala/filodb.cassandra/columnstore/CassandraColumnStoreSpec.scala b/cassandra/src/test/scala/filodb.cassandra/columnstore/CassandraColumnStoreSpec.scala index 2e41e4f007..8bb8fe4935 100644 --- a/cassandra/src/test/scala/filodb.cassandra/columnstore/CassandraColumnStoreSpec.scala +++ b/cassandra/src/test/scala/filodb.cassandra/columnstore/CassandraColumnStoreSpec.scala @@ -1,25 +1,24 @@ package filodb.cassandra.columnstore -import scala.collection.mutable.ArrayBuffer -import scala.concurrent.duration._ - import java.lang.ref.Reference import java.nio.ByteBuffer +import scala.collection.mutable.ArrayBuffer +import scala.concurrent.duration._ + import com.datastax.driver.core.Row import com.typesafe.config.ConfigFactory import monix.reactive.Observable +import filodb.cassandra.DefaultFiloSessionProvider +import filodb.cassandra.metastore.CassandraMetaStore import filodb.core._ import filodb.core.binaryrecord2.RecordBuilder +import filodb.core.metadata.{Dataset, Schemas} +import filodb.core.store.{ChunkSet, ChunkSetInfo, ColumnStoreSpec, PartKeyRecord} import filodb.memory.BinaryRegionLarge import filodb.memory.format.UnsafeUtils import filodb.memory.format.ZeroCopyUTF8String._ -import filodb.core.metadata.{Dataset, Schemas} -import filodb.core.store.{ChunkSet, ChunkSetInfo} -import filodb.core.store.ColumnStoreSpec -import filodb.cassandra.DefaultFiloSessionProvider -import filodb.cassandra.metastore.CassandraMetaStore class CassandraColumnStoreSpec extends ColumnStoreSpec { import NamesTestData._ @@ -46,6 +45,29 @@ class CassandraColumnStoreSpec extends ColumnStoreSpec { } } + "PartKey Reads and Writes" should "work" in { + val dataset = Dataset("prometheus", Schemas.gauge).ref + + colStore.initialize(dataset, 1).futureValue + colStore.truncate(dataset, 1).futureValue + + val pks = (10000 to 30000).map(_.toString.getBytes) + .zipWithIndex.map { case (pk, i) => PartKeyRecord(pk, 5, 10, Some(i))}.toSet + + val updateHour = 10 + colStore.writePartKeys(dataset, 0, Observable.fromIterable(pks), 1.hour.toSeconds.toInt, 10, true ) + .futureValue shouldEqual Success + + val expectedKeys = pks.map(pk => new String(pk.partKey).toInt) + + val readData = colStore.getPartKeysByUpdateHour(dataset, 0, updateHour).toListL.runAsync.futureValue.toSet + readData.map(pk => new String(pk.partKey).toInt) shouldEqual expectedKeys + + val readData2 = colStore.scanPartKeys(dataset, 0).toListL.runAsync.futureValue.toSet + readData2.map(pk => new String(pk.partKey).toInt) shouldEqual expectedKeys + + } + "copyChunksByIngestionTimeRange" should "actually work" in { val dataset = Dataset("source", Schemas.gauge) val targetDataset = Dataset("target", Schemas.gauge) diff --git a/cli/src/main/scala/filodb.cli/CliMain.scala b/cli/src/main/scala/filodb.cli/CliMain.scala index 093e7e2148..340ecd77b4 100644 --- a/cli/src/main/scala/filodb.cli/CliMain.scala +++ b/cli/src/main/scala/filodb.cli/CliMain.scala @@ -18,6 +18,7 @@ import filodb.coordinator.client.QueryCommands.StaticSpreadProvider import filodb.core._ import filodb.core.binaryrecord2.RecordBuilder import filodb.core.metadata.{Column, Schemas} +import filodb.core.query.{PromQlQueryParams, QueryContext, TsdbQueryParams, UnavailablePromQlQueryParams} import filodb.core.store.ChunkSetInfoOnHeap import filodb.memory.MemFactory import filodb.memory.format.{BinaryVector, Classes, MemoryReader, RowReader} @@ -271,7 +272,7 @@ object CliMain extends ArgMain[Arguments] with FilodbClusterNode { timeParams: TimeRangeParams, options: QOptions): Unit = { val logicalPlan = Parser.queryRangeToLogicalPlan(query, timeParams) - executeQuery2(client, dataset, logicalPlan, options, PromQlQueryParams(query,timeParams.start, timeParams.step, + executeQuery2(client, dataset, logicalPlan, options, PromQlQueryParams(systemConfig.getConfig("routing"), query,timeParams.start, timeParams.step, timeParams.end)) } @@ -332,7 +333,7 @@ object CliMain extends ArgMain[Arguments] with FilodbClusterNode { val ref = DatasetRef(dataset) val spreadProvider: Option[SpreadProvider] = options.spread.map(s => StaticSpreadProvider(SpreadChange(0, s))) val qOpts = QueryContext(tsdbQueryParams, spreadProvider, options.sampleLimit) - .copy(queryTimeoutSecs = options.timeout.toSeconds.toInt, + .copy(queryTimeoutMillis = options.timeout.toMillis.toInt, shardOverrides = options.shardOverrides) println(s"Sending query command to server for $ref with options $qOpts...") println(s"Query Plan:\n$plan") diff --git a/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala b/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala index 32faadc946..323022fc94 100644 --- a/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala +++ b/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala @@ -17,6 +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.store.CorruptVectorException import filodb.query._ import filodb.query.exec.ExecPlan @@ -99,16 +100,16 @@ final class QueryActor(memStore: MemStore, def execPhysicalPlan2(q: ExecPlan, replyTo: ActorRef): Unit = { epRequests.increment() Kamon.currentSpan().tag("query", q.getClass.getSimpleName) - Kamon.currentSpan().tag("query-id", q.id) - q.execute(memStore, queryConfig)(queryScheduler, queryConfig.askTimeout) - .foreach { res => + Kamon.currentSpan().tag("query-id", q.queryContext.queryId) + q.execute(memStore, queryConfig)(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.id} Normal QueryError returned from query execution: $e") + 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 => @@ -116,8 +117,8 @@ final class QueryActor(memStore: MemStore, } }(queryScheduler).recover { case ex => // Unhandled exception in query, should be rare - logger.error(s"queryId ${q.id} Unhandled Query Error: ", ex) - replyTo ! QueryError(q.id, ex) + logger.error(s"queryId ${q.queryContext.queryId} Unhandled Query Error: ", ex) + replyTo ! QueryError(q.queryContext.queryId, ex) }(queryScheduler) } diff --git a/coordinator/src/main/scala/filodb.coordinator/client/FiloKryoSerializers.scala b/coordinator/src/main/scala/filodb.coordinator/client/FiloKryoSerializers.scala index 8b23d24ecd..125e340ed1 100644 --- a/coordinator/src/main/scala/filodb.coordinator/client/FiloKryoSerializers.scala +++ b/coordinator/src/main/scala/filodb.coordinator/client/FiloKryoSerializers.scala @@ -2,10 +2,11 @@ package com.esotericsoftware.kryo.io import com.esotericsoftware.kryo.{Serializer => KryoSerializer} import com.esotericsoftware.kryo.Kryo +import com.typesafe.config.{ ConfigFactory, ConfigRenderOptions} import com.typesafe.scalalogging.StrictLogging import filodb.core.binaryrecord2.{RecordSchema => RecordSchema2} -import filodb.core.query.{ColumnInfo, PartitionInfo, PartitionRangeVectorKey} +import filodb.core.query.{ColumnInfo, PartitionInfo, PartitionRangeVectorKey, PromQlQueryParams} import filodb.memory.format._ // NOTE: This file has to be in the kryo namespace so we can use the require() method @@ -76,4 +77,27 @@ class PartitionInfoSerializer extends KryoSerializer[PartitionInfo] { BinaryRegionUtils.writeLargeRegion(info.base, info.offset, output) output.writeInt(info.shardNo) } +} + +class PromQlQueryParamsSerializer extends KryoSerializer[PromQlQueryParams] { + override def read(kryo: Kryo, input: Input, typ: Class[PromQlQueryParams]): PromQlQueryParams = { + val config = ConfigFactory.parseString(input.readString()) + val promQl = input.readString() + val start = input.readLong() + val step = input.readLong() + val end = input.readLong() + val spreadInt = input.readInt() + val spread = if (spreadInt == -1) None else Some(spreadInt) + val procFailure = input.readBoolean() + PromQlQueryParams(config, promQl, start, step, end, spread, procFailure) + } + override def write(kryo: Kryo, output: Output, promParam: PromQlQueryParams): Unit = { + output.writeString(promParam.config.root().render(ConfigRenderOptions.concise())) + output.writeString(promParam.promQl) + output.writeLong(promParam.startSecs) + output.writeLong(promParam.stepSecs) + output.writeLong(promParam.endSecs) + output.writeInt(promParam.spread.getOrElse(-1)) + output.writeBoolean(promParam.processFailure) + } } \ No newline at end of file diff --git a/coordinator/src/main/scala/filodb.coordinator/client/QueryCommands.scala b/coordinator/src/main/scala/filodb.coordinator/client/QueryCommands.scala index 0a966bdc68..3504af2e5e 100644 --- a/coordinator/src/main/scala/filodb.coordinator/client/QueryCommands.scala +++ b/coordinator/src/main/scala/filodb.coordinator/client/QueryCommands.scala @@ -1,7 +1,7 @@ package filodb.coordinator.client -import filodb.core.query.ColumnFilter -import filodb.query.{LogicalPlan => LogicalPlan2, QueryCommand, QueryContext} +import filodb.core.query.{ColumnFilter, QueryContext} +import filodb.query.{LogicalPlan => LogicalPlan2, QueryCommand} object QueryCommands { import filodb.core._ diff --git a/coordinator/src/main/scala/filodb.coordinator/client/QueryOps.scala b/coordinator/src/main/scala/filodb.coordinator/client/QueryOps.scala index 101379f523..cd29d6d54e 100644 --- a/coordinator/src/main/scala/filodb.coordinator/client/QueryOps.scala +++ b/coordinator/src/main/scala/filodb.coordinator/client/QueryOps.scala @@ -5,7 +5,8 @@ import scala.concurrent.duration._ import com.typesafe.scalalogging.StrictLogging import filodb.core._ -import filodb.query.{LogicalPlan => LogicalPlan2, QueryContext, QueryResponse => QueryResponse2} +import filodb.core.query.QueryContext +import filodb.query.{LogicalPlan => LogicalPlan2, QueryResponse => QueryResponse2} trait QueryOps extends ClientBase with StrictLogging { import QueryCommands._ @@ -54,10 +55,10 @@ trait QueryOps extends ClientBase with StrictLogging { plan: LogicalPlan2, qContext: QueryContext = QueryContext()): QueryResponse2 = { val qCmd = LogicalPlan2Query(dataset, plan, qContext) - // NOTE: It's very important to extend the query timeout for the ask itself, because the queryTimeoutSecs is + // NOTE: It's very important to extend the query timeout for the ask itself, because the queryTimeoutMillis is // the internal FiloDB scatter-gather timeout. We need additional time for the proper error to get transmitted // back in case of internal timeouts. - askCoordinator(qCmd, (qContext.queryTimeoutSecs + 10).seconds) { case r: QueryResponse2 => r } + askCoordinator(qCmd, (qContext.queryTimeoutMillis + 10000).millis) { case r: QueryResponse2 => r } } } \ No newline at end of file diff --git a/coordinator/src/main/scala/filodb.coordinator/client/Serializer.scala b/coordinator/src/main/scala/filodb.coordinator/client/Serializer.scala index 67394bc31f..3dbfdde856 100644 --- a/coordinator/src/main/scala/filodb.coordinator/client/Serializer.scala +++ b/coordinator/src/main/scala/filodb.coordinator/client/Serializer.scala @@ -12,7 +12,6 @@ import filodb.core.binaryrecord2.{RecordSchema => RecordSchema2} import filodb.core.metadata.{Column, PartitionSchema, Schema, Schemas} import filodb.core.query.ColumnInfo import filodb.memory.format.ZeroCopyUTF8String -import filodb.query.QueryContext /** * Register commonly used classes for efficient Kryo serialization. If this is not done then Kryo might have to @@ -99,6 +98,7 @@ class KryoInit { kryo.register(classOf[QueryCommands.BadQuery]) kryo.register(classOf[QueryContext]) kryo.register(classOf[QueryCommands.FilteredPartitionQuery]) + kryo.register(classOf[PromQlQueryParams], new PromQlQueryParamsSerializer) } } diff --git a/coordinator/src/main/scala/filodb.coordinator/queryengine/Utils.scala b/coordinator/src/main/scala/filodb.coordinator/queryengine/Utils.scala index a8c2a6b7ef..082607322e 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryengine/Utils.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryengine/Utils.scala @@ -15,9 +15,8 @@ import filodb.coordinator.ShardMapper import filodb.core.{ErrorResponse, SpreadProvider} import filodb.core.binaryrecord2.RecordBuilder import filodb.core.metadata.Dataset -import filodb.core.query.{ColumnFilter, Filter} +import filodb.core.query.{ColumnFilter, Filter, QueryContext} import filodb.core.store._ -import filodb.query.QueryContext final case class ChildErrorResponse(source: ActorRef, resp: ErrorResponse) extends Exception(s"From [$source] - $resp") diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/CompositePlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/CompositePlanner.scala index 664d5e91f1..f162033379 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/CompositePlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/CompositePlanner.scala @@ -7,6 +7,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.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 b6564f664d..6cca313f2d 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/HighAvailabilityPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/HighAvailabilityPlanner.scala @@ -4,7 +4,8 @@ import com.typesafe.config.{Config, ConfigFactory} import com.typesafe.scalalogging.StrictLogging import filodb.core.DatasetRef -import filodb.query.{LogicalPlan, PromQlInvocationParams, PromQlQueryParams, QueryContext} +import filodb.core.query.{PromQlQueryParams, QueryContext} +import filodb.query.LogicalPlan import filodb.query.exec.{ExecPlan, InProcessPlanDispatcher, PromQlExec, StitchRvsExec} /** @@ -45,16 +46,16 @@ class HighAvailabilityPlanner(dsRef: DatasetRef, val queryParams = qContext.origQueryParams.asInstanceOf[PromQlQueryParams] val routingConfig = queryEngineConfig.getConfig("routing") // Divide by 1000 to convert millis to seconds. PromQL params are in seconds. - val promQlInvocationParams = PromQlInvocationParams(routingConfig, queryParams.promQl, + val promQlParams = PromQlQueryParams(routingConfig, queryParams.promQl, timeRange.startMs / 1000, queryParams.stepSecs, timeRange.endMs / 1000, queryParams.spread, processFailure = false) - logger.debug("PromQlExec params:" + promQlInvocationParams) - PromQlExec(qContext.queryId, InProcessPlanDispatcher, dsRef, promQlInvocationParams, qContext.submitTime) + logger.debug("PromQlExec params:" + promQlParams) + PromQlExec(qContext, InProcessPlanDispatcher, dsRef, promQlParams) } } if (execPlans.size == 1) execPlans.head - else StitchRvsExec(qContext.queryId, + else StitchRvsExec(qContext, InProcessPlanDispatcher, execPlans.sortWith((x, y) => !x.isInstanceOf[PromQlExec])) // ^^ Stitch RemoteExec plan results with local using InProcessPlanDispatcher diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala index a7d658d0ff..d3b822b7e1 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LogicalPlanUtils.scala @@ -33,7 +33,7 @@ object LogicalPlanUtils { case lp: ApplyMiscellaneousFunction => getPeriodicSeriesTimeFromLogicalPlan(lp.vectors) case lp: ApplySortFunction => getPeriodicSeriesTimeFromLogicalPlan(lp.vectors) case lp: ScalarVaryingDoublePlan => getPeriodicSeriesTimeFromLogicalPlan(lp.vectors) - case lp: ScalarTimeBasedPlan => TimeRange(lp.rangeParams.start, lp.rangeParams.end) + case lp: ScalarTimeBasedPlan => TimeRange(lp.rangeParams.startSecs, lp.rangeParams.endSecs) case lp: VectorPlan => getPeriodicSeriesTimeFromLogicalPlan(lp.scalars) case lp: ApplyAbsentFunction => getPeriodicSeriesTimeFromLogicalPlan(lp.vectors) case _ => throw new BadQueryException(s"Invalid logical plan") diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LongTimeRangePlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LongTimeRangePlanner.scala index 22a49087d3..26500c5eb8 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/LongTimeRangePlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/LongTimeRangePlanner.scala @@ -1,7 +1,8 @@ package filodb.coordinator.queryplanner import filodb.coordinator.queryplanner.LogicalPlanUtils._ -import filodb.query.{LogicalPlan, PeriodicSeriesPlan, QueryContext} +import filodb.core.query.QueryContext +import filodb.query.{LogicalPlan, PeriodicSeriesPlan} import filodb.query.exec.{ExecPlan, PlanDispatcher, StitchRvsExec} /** @@ -44,7 +45,7 @@ class LongTimeRangePlanner(rawClusterPlanner: QueryPlanner, val rawLp = copyWithUpdatedTimeRange(logicalPlan, TimeRange(firstInstantInRaw, p.endMs), lookbackMs) val rawEp = rawClusterPlanner.materialize(rawLp, qContext) - StitchRvsExec(qContext.queryId, stitchDispatcher, Seq(rawEp, downsampleEp)) + StitchRvsExec(qContext, stitchDispatcher, Seq(rawEp, downsampleEp)) } case _ => // for now send everything else to raw cluster. Metadata queries are TODO diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiClusterPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiClusterPlanner.scala index db21c1f12c..54cef39f8e 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiClusterPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/MultiClusterPlanner.scala @@ -1,6 +1,7 @@ package filodb.coordinator.queryplanner -import filodb.query.{LogicalPlan, QueryContext} +import filodb.core.query.QueryContext +import filodb.query.LogicalPlan import filodb.query.exec.ExecPlan /** diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/QueryPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/QueryPlanner.scala index 61974bb9b6..3182059c01 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryplanner/QueryPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryplanner/QueryPlanner.scala @@ -6,7 +6,8 @@ import kamon.Kamon import monix.eval.Task import monix.execution.Scheduler -import filodb.query.{LogicalPlan, QueryContext, QueryResponse} +import filodb.core.query.QueryContext +import filodb.query.{LogicalPlan, QueryResponse} import filodb.query.exec.ExecPlan /** diff --git a/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryplanner/SingleClusterPlanner.scala index c593de2db2..0908670171 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, RangeParams} +import filodb.core.query.{ColumnFilter, Filter, QueryContext, RangeParams} import filodb.core.store.{AllChunkScan, ChunkScanMethod, InMemoryChunkScan, TimeRangeChunkScan, WriteBufferChunkScan} import filodb.prometheus.ast.Vectors.{PromMetricLabel, TypeLabel} import filodb.prometheus.ast.WindowConstants @@ -80,10 +80,10 @@ class SingleClusterPlanner(dsRef: DatasetRef, case PlanResult(many, stitch) => val targetActor = pickDispatcher(many) many.head match { - case lve: LabelValuesExec => LabelValuesDistConcatExec(qContext.queryId, targetActor, many) - case ske: PartKeysExec => PartKeysDistConcatExec(qContext.queryId, targetActor, many) + case lve: LabelValuesExec => LabelValuesDistConcatExec(qContext, targetActor, many) + case ske: PartKeysExec => PartKeysDistConcatExec(qContext, targetActor, many) case ep: ExecPlan => - val topPlan = DistConcatExec(qContext.queryId, targetActor, many) + val topPlan = DistConcatExec(qContext, targetActor, many) if (stitch) topPlan.addRangeVectorTransformer(StitchRvsMapper()) topPlan } @@ -96,14 +96,14 @@ class SingleClusterPlanner(dsRef: DatasetRef, private def shardsFromFilters(filters: Seq[ColumnFilter], - options: QueryContext): Seq[Int] = { + qContext: QueryContext): Seq[Int] = { - val spreadProvToUse = options.spreadOverride.getOrElse(spreadProvider) + val spreadProvToUse = qContext.spreadOverride.getOrElse(spreadProvider) - require(shardColumns.nonEmpty || options.shardOverrides.nonEmpty, + require(shardColumns.nonEmpty || qContext.shardOverrides.nonEmpty, s"Dataset $dsRef does not have shard columns defined, and shard overrides were not mentioned") - options.shardOverrides.getOrElse { + qContext.shardOverrides.getOrElse { val shardVals = shardColumns.map { shardCol => // So to compute the shard hash we need shardCol == value filter (exact equals) for each shardColumn filters.find(f => f.column == shardCol) match { @@ -158,45 +158,45 @@ class SingleClusterPlanner(dsRef: DatasetRef, * @return ExecPlans that answer the logical plan provided */ private def walkLogicalPlanTree(logicalPlan: LogicalPlan, - options: QueryContext): PlanResult = { + qContext: QueryContext): PlanResult = { logicalPlan match { - case lp: RawSeries => materializeRawSeries(options, lp) - case lp: RawChunkMeta => materializeRawChunkMeta(options, lp) - case lp: PeriodicSeries => materializePeriodicSeries(options, lp) - case lp: PeriodicSeriesWithWindowing => materializePeriodicSeriesWithWindowing(options, lp) - case lp: ApplyInstantFunction => materializeApplyInstantFunction(options, lp) - case lp: ApplyInstantFunctionRaw => materializeApplyInstantFunctionRaw(options, lp) - case lp: Aggregate => materializeAggregate(options, lp) - case lp: BinaryJoin => materializeBinaryJoin(options, lp) - case lp: ScalarVectorBinaryOperation => materializeScalarVectorBinOp(options, lp) - case lp: LabelValues => materializeLabelValues(options, lp) - case lp: SeriesKeysByFilters => materializeSeriesKeysByFilters(options, lp) - case lp: ApplyMiscellaneousFunction => materializeApplyMiscellaneousFunction(options, lp) - case lp: ApplySortFunction => materializeApplySortFunction(options, lp) - case lp: ScalarVaryingDoublePlan => materializeScalarPlan(options, lp) - case lp: ScalarTimeBasedPlan => materializeScalarTimeBased(options, lp) - case lp: VectorPlan => materializeVectorPlan(options, lp) - case lp: ScalarFixedDoublePlan => materializeFixedScalar(options, lp) - case lp: ApplyAbsentFunction => materializeAbsentFunction(options, lp) + case lp: RawSeries => materializeRawSeries(qContext, lp) + case lp: RawChunkMeta => materializeRawChunkMeta(qContext, lp) + case lp: PeriodicSeries => materializePeriodicSeries(qContext, lp) + case lp: PeriodicSeriesWithWindowing => materializePeriodicSeriesWithWindowing(qContext, lp) + case lp: ApplyInstantFunction => materializeApplyInstantFunction(qContext, lp) + case lp: ApplyInstantFunctionRaw => materializeApplyInstantFunctionRaw(qContext, lp) + case lp: Aggregate => materializeAggregate(qContext, lp) + case lp: BinaryJoin => materializeBinaryJoin(qContext, lp) + case lp: ScalarVectorBinaryOperation => materializeScalarVectorBinOp(qContext, lp) + case lp: LabelValues => materializeLabelValues(qContext, lp) + case lp: SeriesKeysByFilters => materializeSeriesKeysByFilters(qContext, lp) + case lp: ApplyMiscellaneousFunction => materializeApplyMiscellaneousFunction(qContext, lp) + case lp: ApplySortFunction => materializeApplySortFunction(qContext, lp) + case lp: ScalarVaryingDoublePlan => materializeScalarPlan(qContext, lp) + case lp: ScalarTimeBasedPlan => materializeScalarTimeBased(qContext, lp) + case lp: VectorPlan => materializeVectorPlan(qContext, lp) + case lp: ScalarFixedDoublePlan => materializeFixedScalar(qContext, lp) + case lp: ApplyAbsentFunction => materializeAbsentFunction(qContext, lp) case _ => throw new BadQueryException("Invalid logical plan") } } - private def materializeScalarVectorBinOp(options: QueryContext, + private def materializeScalarVectorBinOp(qContext: QueryContext, lp: ScalarVectorBinaryOperation): PlanResult = { - val vectors = walkLogicalPlanTree(lp.vector, options) - val funcArg = materializeFunctionArgs(Seq(lp.scalarArg), options) + val vectors = walkLogicalPlanTree(lp.vector, qContext) + val funcArg = materializeFunctionArgs(Seq(lp.scalarArg), qContext) vectors.plans.foreach(_.addRangeVectorTransformer(ScalarOperationMapper(lp.operator, lp.scalarIsLhs, funcArg))) vectors } - private def materializeBinaryJoin(options: QueryContext, + private def materializeBinaryJoin(qContext: QueryContext, lp: BinaryJoin): PlanResult = { - val lhs = walkLogicalPlanTree(lp.lhs, options) - val stitchedLhs = if (lhs.needsStitch) Seq(StitchRvsExec(options.queryId, pickDispatcher(lhs.plans), lhs.plans)) + val lhs = walkLogicalPlanTree(lp.lhs, qContext) + val stitchedLhs = if (lhs.needsStitch) Seq(StitchRvsExec(qContext, pickDispatcher(lhs.plans), lhs.plans)) else lhs.plans - val rhs = walkLogicalPlanTree(lp.rhs, options) - val stitchedRhs = if (rhs.needsStitch) Seq(StitchRvsExec(options.queryId, pickDispatcher(rhs.plans), rhs.plans)) + val rhs = walkLogicalPlanTree(lp.rhs, qContext) + val stitchedRhs = if (rhs.needsStitch) Seq(StitchRvsExec(qContext, pickDispatcher(rhs.plans), rhs.plans)) else rhs.plans // TODO Currently we create separate exec plan node for stitching. // Ideally, we can go one step further and add capability to NonLeafNode plans to pre-process @@ -206,17 +206,17 @@ class SingleClusterPlanner(dsRef: DatasetRef, val targetActor = pickDispatcher(stitchedLhs ++ stitchedRhs) val joined = if (lp.operator.isInstanceOf[SetOperator]) - Seq(exec.SetOperatorExec(options.queryId, targetActor, stitchedLhs, stitchedRhs, lp.operator, + Seq(exec.SetOperatorExec(qContext, targetActor, stitchedLhs, stitchedRhs, lp.operator, lp.on, lp.ignoring, dsOptions.metricColumn)) else - Seq(BinaryJoinExec(options.queryId, targetActor, stitchedLhs, stitchedRhs, lp.operator, lp.cardinality, + Seq(BinaryJoinExec(qContext, targetActor, stitchedLhs, stitchedRhs, lp.operator, lp.cardinality, lp.on, lp.ignoring, lp.include, dsOptions.metricColumn)) PlanResult(joined, false) } - private def materializeAggregate(options: QueryContext, + private def materializeAggregate(qContext: QueryContext, lp: Aggregate): PlanResult = { - val toReduceLevel1 = walkLogicalPlanTree(lp.vectors, options) + val toReduceLevel1 = walkLogicalPlanTree(lp.vectors, qContext) // Now we have one exec plan per shard /* * Note that in order for same overlapping RVs to not be double counted when spread is increased, @@ -239,58 +239,74 @@ class SingleClusterPlanner(dsRef: DatasetRef, val groupSize = Math.sqrt(toReduceLevel1.plans.size).ceil.toInt toReduceLevel1.plans.grouped(groupSize).map { nodePlans => val reduceDispatcher = nodePlans.head.dispatcher - ReduceAggregateExec(options.queryId, reduceDispatcher, nodePlans, lp.operator, lp.params) + ReduceAggregateExec(qContext, reduceDispatcher, nodePlans, lp.operator, lp.params) }.toList } else toReduceLevel1.plans val reduceDispatcher = pickDispatcher(toReduceLevel2) - val reducer = ReduceAggregateExec(options.queryId, reduceDispatcher, toReduceLevel2, lp.operator, lp.params) + val reducer = ReduceAggregateExec(qContext, reduceDispatcher, toReduceLevel2, lp.operator, lp.params) reducer.addRangeVectorTransformer(AggregatePresenter(lp.operator, lp.params)) PlanResult(Seq(reducer), false) // since we have aggregated, no stitching } - private def materializeApplyInstantFunction(options: QueryContext, + private def materializeApplyInstantFunction(qContext: QueryContext, lp: ApplyInstantFunction): PlanResult = { - val vectors = walkLogicalPlanTree(lp.vectors, options) - val paramsExec = materializeFunctionArgs(lp.functionArgs, options) + val vectors = walkLogicalPlanTree(lp.vectors, qContext) + val paramsExec = materializeFunctionArgs(lp.functionArgs, qContext) vectors.plans.foreach(_.addRangeVectorTransformer(InstantVectorFunctionMapper(lp.function, paramsExec))) vectors } - private def materializeApplyInstantFunctionRaw(options: QueryContext, + private def materializeApplyInstantFunctionRaw(qContext: QueryContext, lp: ApplyInstantFunctionRaw): PlanResult = { - val vectors = walkLogicalPlanTree(lp.vectors, options) - val paramsExec = materializeFunctionArgs(lp.functionArgs, options) + val vectors = walkLogicalPlanTree(lp.vectors, qContext) + val paramsExec = materializeFunctionArgs(lp.functionArgs, qContext) vectors.plans.foreach(_.addRangeVectorTransformer(InstantVectorFunctionMapper(lp.function, paramsExec))) vectors } - private def materializePeriodicSeriesWithWindowing(options: QueryContext, + private def materializePeriodicSeriesWithWindowing(qContext: QueryContext, lp: PeriodicSeriesWithWindowing): PlanResult = { - val series = walkLogicalPlanTree(lp.series, options) + val series = walkLogicalPlanTree(lp.series, qContext) val rawSource = lp.series.isRaw val execRangeFn = InternalRangeFunction.lpToInternalFunc(lp.function) - val paramsExec = materializeFunctionArgs(lp.functionArgs, options) - - val newStartMs = boundStartInstant(lp.startMs, lp.stepMs, lp.endMs, lp.window, lp.offset.getOrElse(0)) - val window = if (execRangeFn == InternalRangeFunction.Timestamp) None else Some(lp.window) - series.plans.foreach(_.addRangeVectorTransformer(PeriodicSamplesMapper(newStartMs, lp.stepMs, - lp.endMs, window, Some(execRangeFn), paramsExec, lp.offset, rawSource))) - series + val paramsExec = materializeFunctionArgs(lp.functionArgs, qContext) + + val newStartMs = boundToStartTimeToEarliestRetained(lp.startMs, lp.stepMs, lp.window, lp.offset.getOrElse(0)) + if (newStartMs <= lp.endMs) { + val window = if (execRangeFn == InternalRangeFunction.Timestamp) None else Some(lp.window) + series.plans.foreach(_.addRangeVectorTransformer(PeriodicSamplesMapper(newStartMs, lp.stepMs, + lp.endMs, window, Some(execRangeFn), qContext, paramsExec, lp.offset, rawSource))) + series + } else { // query is outside retention period, simply return empty result + PlanResult(Seq(EmptyResultExec(qContext, dsRef))) + } } - private def materializePeriodicSeries(options: QueryContext, + private def materializePeriodicSeries(qContext: QueryContext, lp: PeriodicSeries): PlanResult = { - val rawSeries = walkLogicalPlanTree(lp.rawSeries, options) - val newStartMs = boundStartInstant(lp.startMs, lp.stepMs, lp.endMs, + val rawSeries = walkLogicalPlanTree(lp.rawSeries, qContext) + val newStartMs = boundToStartTimeToEarliestRetained(lp.startMs, lp.stepMs, WindowConstants.staleDataLookbackSeconds * 1000, lp.offset.getOrElse(0)) - rawSeries.plans.foreach(_.addRangeVectorTransformer(PeriodicSamplesMapper(newStartMs, lp.stepMs, lp.endMs, - None, None, Nil, lp.offset))) - rawSeries + if (newStartMs <= lp.endMs) { + rawSeries.plans.foreach(_.addRangeVectorTransformer(PeriodicSamplesMapper(newStartMs, lp.stepMs, lp.endMs, + None, None, qContext, Nil, lp.offset))) + rawSeries + } else { // query is outside retention period, simply return empty result + PlanResult(Seq(EmptyResultExec(qContext, dsRef))) + } } - private def boundStartInstant(startMs: Long, stepMs: Long, endMs: Long, - windowMs: Long, offsetMs: Long): Long = { + /** + * Calculates the earliest startTime possible given the query's start/window/step/offset. + * This is used to bound the startTime of queries so we dont create possibility of aggregating + * partially expired data and return incomplete results. + * + * @return new startTime to be used for query in ms. If original startTime is within retention + * period, returns it as is. + */ + private def boundToStartTimeToEarliestRetained(startMs: Long, stepMs: Long, + windowMs: Long, offsetMs: Long): Long = { // In case query is earlier than earliestRetainedTimestamp then we need to drop the first few instants // to prevent inaccurate results being served. Inaccuracy creeps in because data can be in memory for which // equivalent data may not be in cassandra. Aggregations cannot be guaranteed to be complete. @@ -326,10 +342,10 @@ class SingleClusterPlanner(dsRef: DatasetRef, (newFilters, schemaOpt) } - private def materializeRawSeries(options: QueryContext, + private def materializeRawSeries(qContext: QueryContext, lp: RawSeries): PlanResult = { - val spreadProvToUse = options.spreadOverride.getOrElse(spreadProvider) + val spreadProvToUse = qContext.spreadOverride.getOrElse(spreadProvider) val colName = lp.columns.headOption val (renamedFilters, schemaOpt) = extractSchemaFilter(renameMetricFilter(lp.filters)) @@ -338,15 +354,15 @@ class SingleClusterPlanner(dsRef: DatasetRef, case IntervalSelector(from, to) => spreadChanges.exists(c => c.time >= from && c.time <= to) case _ => false } - val execPlans = shardsFromFilters(renamedFilters, options).map { shard => + val execPlans = shardsFromFilters(renamedFilters, qContext).map { shard => val dispatcher = dispatcherForShard(shard) - MultiSchemaPartitionsExec(options.queryId, options.submitTime, options.sampleLimit, dispatcher, dsRef, shard, - renamedFilters, toChunkScanMethod(lp.rangeSelector), schemaOpt, colName) + MultiSchemaPartitionsExec(qContext, dispatcher, dsRef, shard, renamedFilters, toChunkScanMethod(lp.rangeSelector), + schemaOpt, colName) } PlanResult(execPlans, needsStitch) } - private def materializeLabelValues(options: QueryContext, + private def materializeLabelValues(qContext: QueryContext, lp: LabelValues): PlanResult = { val filters = lp.labelConstraints.map { case (k, v) => ColumnFilter(k, Filter.Equals(v)) }.toSeq // If the label is PromMetricLabel and is different than dataset's metric name, @@ -356,54 +372,52 @@ class SingleClusterPlanner(dsRef: DatasetRef, lp.labelNames.updated(metricLabelIndex, dsOptions.metricColumn) else lp.labelNames val shardsToHit = if (shardColumns.toSet.subsetOf(lp.labelConstraints.keySet)) { - shardsFromFilters(filters, options) + shardsFromFilters(filters, qContext) } else { mdNoShardKeyFilterRequests.increment() shardMapperFunc.assignedShards } val metaExec = shardsToHit.map { shard => val dispatcher = dispatcherForShard(shard) - exec.LabelValuesExec(options.queryId, options.submitTime, options.sampleLimit, dispatcher, dsRef, shard, - filters, labelNames, lp.lookbackTimeMs) + exec.LabelValuesExec(qContext, dispatcher, dsRef, shard, filters, labelNames, lp.lookbackTimeMs) } PlanResult(metaExec, false) } - private def materializeSeriesKeysByFilters(options: QueryContext, + private def materializeSeriesKeysByFilters(qContext: QueryContext, lp: SeriesKeysByFilters): PlanResult = { // NOTE: _type_ filter support currently isn't there in series keys queries val (renamedFilters, schemaOpt) = extractSchemaFilter(renameMetricFilter(lp.filters)) val filterCols = lp.filters.map(_.column).toSet val shardsToHit = if (shardColumns.toSet.subsetOf(filterCols)) { - shardsFromFilters(lp.filters, options) + shardsFromFilters(lp.filters, qContext) } else { mdNoShardKeyFilterRequests.increment() shardMapperFunc.assignedShards } val metaExec = shardsToHit.map { shard => val dispatcher = dispatcherForShard(shard) - PartKeysExec(options.queryId, options.submitTime, options.sampleLimit, dispatcher, dsRef, shard, - schemas.part, renamedFilters, lp.startMs, lp.endMs) + PartKeysExec(qContext, dispatcher, dsRef, shard, schemas.part, renamedFilters, lp.startMs, lp.endMs) } PlanResult(metaExec, false) } - private def materializeRawChunkMeta(options: QueryContext, + private def materializeRawChunkMeta(qContext: QueryContext, lp: RawChunkMeta): PlanResult = { // Translate column name to ID and validate here val colName = if (lp.column.isEmpty) None else Some(lp.column) val (renamedFilters, schemaOpt) = extractSchemaFilter(renameMetricFilter(lp.filters)) - val metaExec = shardsFromFilters(renamedFilters, options).map { shard => + val metaExec = shardsFromFilters(renamedFilters, qContext).map { shard => val dispatcher = dispatcherForShard(shard) - SelectChunkInfosExec(options.queryId, options.submitTime, options.sampleLimit, dispatcher, dsRef, shard, - renamedFilters, toChunkScanMethod(lp.rangeSelector), schemaOpt, colName) + SelectChunkInfosExec(qContext, dispatcher, dsRef, shard, renamedFilters, toChunkScanMethod(lp.rangeSelector), + schemaOpt, colName) } PlanResult(metaExec, false) } - private def materializeApplyMiscellaneousFunction(options: QueryContext, + private def materializeApplyMiscellaneousFunction(qContext: QueryContext, lp: ApplyMiscellaneousFunction): PlanResult = { - val vectors = walkLogicalPlanTree(lp.vectors, options) + val vectors = walkLogicalPlanTree(lp.vectors, qContext) if (lp.function == MiscellaneousFunctionId.HistToPromVectors) vectors.plans.foreach(_.addRangeVectorTransformer(HistToPromSeriesMapper(schemas.part))) else @@ -412,14 +426,14 @@ class SingleClusterPlanner(dsRef: DatasetRef, } private def materializeFunctionArgs(functionParams: Seq[FunctionArgsPlan], - options: QueryContext): Seq[FuncArgs] = { + qContext: QueryContext): Seq[FuncArgs] = { if (functionParams.isEmpty){ Nil } else { functionParams.map { param => param match { case num: ScalarFixedDoublePlan => StaticFuncArgs(num.scalar, num.timeStepParams) - case s: ScalarVaryingDoublePlan => ExecPlanFuncArgs(materialize(s, options), + case s: ScalarVaryingDoublePlan => ExecPlanFuncArgs(materialize(s, qContext), RangeParams(s.startMs, s.stepMs, s.endMs)) case t: ScalarTimeBasedPlan => TimeFuncArgs(t.rangeParams) case _ => throw new UnsupportedOperationException("Invalid logical plan") @@ -428,12 +442,12 @@ class SingleClusterPlanner(dsRef: DatasetRef, } } - private def materializeScalarPlan(options: QueryContext, + private def materializeScalarPlan(qContext: QueryContext, lp: ScalarVaryingDoublePlan): PlanResult = { - val vectors = walkLogicalPlanTree(lp.vectors, options) + val vectors = walkLogicalPlanTree(lp.vectors, qContext) if (vectors.plans.length > 1) { val targetActor = pickDispatcher(vectors.plans) - val topPlan = DistConcatExec(options.queryId, targetActor, vectors.plans) + val topPlan = DistConcatExec(qContext, targetActor, vectors.plans) topPlan.addRangeVectorTransformer(ScalarFunctionMapper(lp.function, RangeParams(lp.startMs, lp.stepMs, lp.endMs))) PlanResult(Seq(topPlan), vectors.needsStitch) } else { @@ -443,12 +457,12 @@ class SingleClusterPlanner(dsRef: DatasetRef, } } - private def materializeApplySortFunction(options: QueryContext, + private def materializeApplySortFunction(qContext: QueryContext, lp: ApplySortFunction): PlanResult = { - val vectors = walkLogicalPlanTree(lp.vectors, options) + val vectors = walkLogicalPlanTree(lp.vectors, qContext) if(vectors.plans.length > 1) { val targetActor = pickDispatcher(vectors.plans) - val topPlan = DistConcatExec(options.queryId, targetActor, vectors.plans) + val topPlan = DistConcatExec(qContext, targetActor, vectors.plans) topPlan.addRangeVectorTransformer(SortFunctionMapper(lp.function)) PlanResult(Seq(topPlan), vectors.needsStitch) } else { @@ -457,12 +471,12 @@ class SingleClusterPlanner(dsRef: DatasetRef, } } - private def materializeAbsentFunction(options: QueryContext, + private def materializeAbsentFunction(qContext: QueryContext, lp: ApplyAbsentFunction): PlanResult = { - val vectors = walkLogicalPlanTree(lp.vectors, options) + val vectors = walkLogicalPlanTree(lp.vectors, qContext) if (vectors.plans.length > 1) { val targetActor = pickDispatcher(vectors.plans) - val topPlan = DistConcatExec(options.queryId, targetActor, vectors.plans) + val topPlan = DistConcatExec(qContext, targetActor, vectors.plans) topPlan.addRangeVectorTransformer(AbsentFunctionMapper(lp.columnFilters, lp.rangeParams, PromMetricLabel)) PlanResult(Seq(topPlan), vectors.needsStitch) @@ -473,24 +487,22 @@ class SingleClusterPlanner(dsRef: DatasetRef, } } - private def materializeScalarTimeBased(options: QueryContext, + private def materializeScalarTimeBased(qContext: QueryContext, lp: ScalarTimeBasedPlan): PlanResult = { - val scalarTimeBasedExec = TimeScalarGeneratorExec(options.queryId, dsRef, lp.rangeParams, lp.function, - options.sampleLimit) + val scalarTimeBasedExec = TimeScalarGeneratorExec(qContext, dsRef, lp.rangeParams, lp.function) PlanResult(Seq(scalarTimeBasedExec), false) } - private def materializeVectorPlan(options: QueryContext, + private def materializeVectorPlan(qContext: QueryContext, lp: VectorPlan): PlanResult = { - val vectors = walkLogicalPlanTree(lp.scalars, options) + val vectors = walkLogicalPlanTree(lp.scalars, qContext) vectors.plans.foreach(_.addRangeVectorTransformer(VectorFunctionMapper())) vectors } - private def materializeFixedScalar(options: QueryContext, + private def materializeFixedScalar(qContext: QueryContext, lp: ScalarFixedDoublePlan): PlanResult = { - val scalarFixedDoubleExec = ScalarFixedDoubleExec(options.queryId, dsRef, lp.timeStepParams, lp.scalar, - options.sampleLimit) + val scalarFixedDoubleExec = ScalarFixedDoubleExec(qContext, dsRef, lp.timeStepParams, lp.scalar) PlanResult(Seq(scalarFixedDoubleExec), false) } diff --git a/coordinator/src/multi-jvm/scala/filodb.coordinator/ClusterRecoverySpec.scala b/coordinator/src/multi-jvm/scala/filodb.coordinator/ClusterRecoverySpec.scala index eb0378191a..08ae64030a 100644 --- a/coordinator/src/multi-jvm/scala/filodb.coordinator/ClusterRecoverySpec.scala +++ b/coordinator/src/multi-jvm/scala/filodb.coordinator/ClusterRecoverySpec.scala @@ -2,17 +2,15 @@ package filodb.coordinator import scala.concurrent.Future import scala.concurrent.duration._ - import akka.actor.ActorRef import akka.pattern.ask import akka.remote.testkit.MultiNodeConfig import akka.util.Timeout import com.typesafe.config.ConfigFactory import org.scalatest.time.{Millis, Seconds, Span} - import filodb.core._ import filodb.core.metadata.Column.ColumnType -import filodb.core.query.ColumnInfo +import filodb.core.query.{ColumnInfo, QueryContext} object ClusterRecoverySpecConfig extends MultiNodeConfig { // register the named roles (nodes) of the test diff --git a/coordinator/src/test/scala/filodb.coordinator/NodeCoordinatorActorSpec.scala b/coordinator/src/test/scala/filodb.coordinator/NodeCoordinatorActorSpec.scala index e107e940da..933b89af8f 100644 --- a/coordinator/src/test/scala/filodb.coordinator/NodeCoordinatorActorSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/NodeCoordinatorActorSpec.scala @@ -119,9 +119,9 @@ class NodeCoordinatorActorSpec extends ActorTest(NodeCoordinatorActorSpec.getNew } } - val timeMinSchema = ResultSchema(Seq(ColumnInfo("timestamp", LongColumn), ColumnInfo("min", DoubleColumn)), 1) - val countSchema = ResultSchema(Seq(ColumnInfo("timestamp", LongColumn), ColumnInfo("count", DoubleColumn)), 1) - val valueSchema = ResultSchema(Seq(ColumnInfo("timestamp", LongColumn), ColumnInfo("value", DoubleColumn)), 1) + val timeMinSchema = ResultSchema(Seq(ColumnInfo("timestamp", TimestampColumn), ColumnInfo("min", DoubleColumn)), 1) + val countSchema = ResultSchema(Seq(ColumnInfo("timestamp", TimestampColumn), ColumnInfo("count", DoubleColumn)), 1) + val valueSchema = ResultSchema(Seq(ColumnInfo("timestamp", TimestampColumn), ColumnInfo("value", DoubleColumn)), 1) val qOpt = QueryContext(shardOverrides = Some(Seq(0))) describe("QueryActor commands and responses") { diff --git a/coordinator/src/test/scala/filodb.coordinator/client/SerializationSpec.scala b/coordinator/src/test/scala/filodb.coordinator/client/SerializationSpec.scala index cc9e1d7173..372be4e05b 100644 --- a/coordinator/src/test/scala/filodb.coordinator/client/SerializationSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/client/SerializationSpec.scala @@ -3,11 +3,11 @@ package filodb.coordinator.client import akka.actor.ActorRef 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.{query, MachineMetricsData, SpreadChange} +import filodb.core.{MachineMetricsData, SpreadChange, query} import filodb.core.binaryrecord2.BinaryRecordRowReader import filodb.core.metadata.{Dataset, Schemas} import filodb.core.metadata.Column.ColumnType @@ -131,7 +131,7 @@ class SerializationSpec extends ActorTest(SerializationSpecConfig.getNewSystem) }.toBuffer - val cols = Array(new ColumnInfo("timestamp", ColumnType.LongColumn), + val cols = Array(new ColumnInfo("timestamp", ColumnType.TimestampColumn), new ColumnInfo("value", ColumnType.DoubleColumn)) val srvs = for { i <- 0 to 9 } yield { val rv = new RangeVector { @@ -312,4 +312,22 @@ class SerializationSpec extends ActorTest(SerializationSpecConfig.getNewSystem) actual.toList shouldEqual expected } + it ("should serialize and deserialize serialize ExecPlan with config") { + val node0 = TestProbe().ref + val mapper = new ShardMapper(1) + def mapperRef: ShardMapper = mapper + mapper.registerNode(Seq(0), node0) + val to = System.currentTimeMillis() / 1000 + val from = to - 50 + val qParams = TimeStepParams(from, 10, to) + val engine = new SingleClusterPlanner(dataset.ref, Schemas.global, mapperRef, 0) + + val logicalPlan = Parser.queryRangeToLogicalPlan( + s"""http_request_duration_seconds_bucket{job="prometheus",$shardKeyStr}""", + qParams) + val param = PromQlQueryParams(ConfigFactory.empty(), "test", 1000, 200, 5000) + val execPlan = engine.materialize(logicalPlan, QueryContext(origQueryParams = param, + spreadOverride = Some(new StaticSpreadProvider(SpreadChange(0, 0))))) + roundTrip(execPlan) shouldEqual execPlan + } } diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/FailureProviderSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/FailureProviderSpec.scala index 7ccff28afc..22e11d24d1 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/FailureProviderSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/FailureProviderSpec.scala @@ -1,14 +1,10 @@ package filodb.coordinator.queryplanner -import monix.eval.Task -import monix.execution.Scheduler import org.scalatest.{FunSpec, Matchers} -import scala.concurrent.duration.FiniteDuration import filodb.core.DatasetRef import filodb.core.query.{ColumnFilter, Filter} import filodb.query._ -import filodb.query.exec.{ExecPlan, PlanDispatcher} class FailureProviderSpec extends FunSpec with Matchers { @@ -33,12 +29,6 @@ class FailureProviderSpec extends FunSpec with Matchers { val windowed2 = PeriodicSeriesWithWindowing(raw2, from + 1000, 1000, to, 5000, RangeFunctionId.Rate) val summed2 = Aggregate(AggregationOperator.Sum, windowed2, Nil, Seq("job")) - val dummyDispatcher = new PlanDispatcher { - override def dispatch(plan: ExecPlan) - (implicit sched: Scheduler, - timeout: FiniteDuration): Task[QueryResponse] = ??? - } - val datasetRef = DatasetRef("dataset", Some("cassandra")) it("should check for PeriodicSeries plan") { diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/HighAvailabilityPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/HighAvailabilityPlannerSpec.scala index cda9932311..f77f721f87 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/HighAvailabilityPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/HighAvailabilityPlannerSpec.scala @@ -4,11 +4,10 @@ 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} +import filodb.core.query.{ColumnFilter, Filter, PromQlQueryParams, QueryContext} import filodb.core.store.TimeRangeChunkScan import filodb.query._ import filodb.query.exec._ @@ -41,7 +40,7 @@ class HighAvailabilityPlannerSpec extends FunSpec with Matchers { ColumnFilter("job", Filter.Equals("myService")), ColumnFilter("le", Filter.Equals("0.3"))) - private val promQlQueryParams = PromQlQueryParams("sum(heap_usage)", 100, 1, 1000, None) + private val promQlQueryParams = PromQlQueryParams(ConfigFactory.empty,"sum(heap_usage)", 100, 1, 1000, None) val localPlanner = new SingleClusterPlanner(dsRef, schemas, mapperRef, earliestRetainedTimestampFn = 0) @@ -115,7 +114,7 @@ class HighAvailabilityPlannerSpec extends FunSpec with Matchers { val raw = RawSeries(rangeSelector = intervalSelector, filters = f1, columns = Seq("value")) val windowed = PeriodicSeriesWithWindowing(raw, from, 100, to, 5000, RangeFunctionId.Rate) val summed = Aggregate(AggregationOperator.Sum, windowed, Nil, Seq("job")) - val promQlQueryParams = PromQlQueryParams("", from/1000, 1, to/1000, None) + val promQlQueryParams = PromQlQueryParams(ConfigFactory.empty, "", from/1000, 1, to/1000, None) val failureProvider = new FailureProvider { override def getFailures(datasetRef: DatasetRef, queryTimeRange: TimeRange): Seq[FailureTimeRange] = { @@ -240,7 +239,7 @@ class HighAvailabilityPlannerSpec extends FunSpec with Matchers { val raw = RawSeries(rangeSelector = intervalSelector, filters = f1, columns = Seq("value")) val windowed = PeriodicSeriesWithWindowing(raw, from * 1000, step * 1000, to * 1000, 5000, RangeFunctionId.Rate) val summed = Aggregate(AggregationOperator.Sum, windowed, Nil, Seq("job")) - val promQlQueryParams = PromQlQueryParams("dummy query", from, step, to, None) + val promQlQueryParams = PromQlQueryParams(ConfigFactory.empty, "dummy query", from, step, to, None) val failureProvider = new FailureProvider { override def getFailures(datasetRef: DatasetRef, queryTimeRange: TimeRange): Seq[FailureTimeRange] = { @@ -295,7 +294,7 @@ class HighAvailabilityPlannerSpec extends FunSpec with Matchers { val raw = RawSeries(rangeSelector = intervalSelector, filters = f1, columns = Seq("value")) val windowed = PeriodicSeriesWithWindowing(raw, from * 1000, step * 1000, to * 1000, 5000, RangeFunctionId.Rate) val summed = Aggregate(AggregationOperator.Sum, windowed, Nil, Seq("job")) - val promQlQueryParams = PromQlQueryParams("dummy query", from, step, to, None) + val promQlQueryParams = PromQlQueryParams(ConfigFactory.empty, "dummy query", from, step, to, None) val failureProvider = new FailureProvider { override def getFailures(datasetRef: DatasetRef, queryTimeRange: TimeRange): Seq[FailureTimeRange] = { @@ -328,7 +327,7 @@ class HighAvailabilityPlannerSpec extends FunSpec with Matchers { val raw = RawSeries(rangeSelector = intervalSelector, filters = f1, columns = Seq("value")) val windowed = PeriodicSeriesWithWindowing(raw, from * 1000, step * 1000, to * 1000, 5000, RangeFunctionId.Rate) val summed = Aggregate(AggregationOperator.Sum, windowed, Nil, Seq("job")) - val promQlQueryParams = PromQlQueryParams("dummy query", from, step, to, None) + val promQlQueryParams = PromQlQueryParams(ConfigFactory.empty, "dummy query", from, step, to, None) val failureProvider = new FailureProvider { override def getFailures(datasetRef: DatasetRef, queryTimeRange: TimeRange): Seq[FailureTimeRange] = { @@ -367,7 +366,7 @@ class HighAvailabilityPlannerSpec extends FunSpec with Matchers { val raw = RawSeries(rangeSelector = intervalSelector, filters = f1, columns = Seq("value")) val windowed = PeriodicSeriesWithWindowing(raw, from * 1000, step * 1000, to * 1000, 5000, RangeFunctionId.Rate) val summed = Aggregate(AggregationOperator.Sum, windowed, Nil, Seq("job")) - val promQlQueryParams = PromQlQueryParams("dummy query", from, step, to, None) + val promQlQueryParams = PromQlQueryParams(ConfigFactory.empty, "dummy query", from, step, to, None) val failureProvider = new FailureProvider { override def getFailures(datasetRef: DatasetRef, queryTimeRange: TimeRange): Seq[FailureTimeRange] = { diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/LongTimeRangePlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/LongTimeRangePlannerSpec.scala index 2756b6cb06..4628924196 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/LongTimeRangePlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/LongTimeRangePlannerSpec.scala @@ -1,40 +1,38 @@ 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.store.ChunkSource import filodb.prometheus.ast.TimeStepParams import filodb.prometheus.parse.Parser -import filodb.query.{LogicalPlan, PeriodicSeriesPlan, QueryConfig, QueryContext} +import filodb.query.{LogicalPlan, PeriodicSeriesPlan, QueryConfig} import filodb.query.exec._ class LongTimeRangePlannerSpec extends FunSpec with Matchers { - class MockExecPlan(val name: String, val lp: LogicalPlan, val qContext: QueryContext) extends ExecPlan { - override def id: String = ??? + class MockExecPlan(val name: String, val lp: LogicalPlan) extends ExecPlan { + override def queryContext: QueryContext = QueryContext() override def children: Seq[ExecPlan] = ??? override def submitTime: Long = ??? - override def limit: Int = ??? override def dataset: DatasetRef = ??? override def dispatcher: PlanDispatcher = ??? override def doExecute(source: ChunkSource, queryConfig: QueryConfig) - (implicit sched: Scheduler, timeout: FiniteDuration): ExecResult = ??? + (implicit sched: Scheduler): ExecResult = ??? override protected def args: String = ??? } val rawPlanner = new QueryPlanner { override def materialize(logicalPlan: LogicalPlan, qContext: QueryContext): ExecPlan = { - new MockExecPlan("raw", logicalPlan, qContext) + new MockExecPlan("raw", logicalPlan) } } val downsamplePlanner = new QueryPlanner { override def materialize(logicalPlan: LogicalPlan, qContext: QueryContext): ExecPlan = { - new MockExecPlan("downsample", logicalPlan, qContext) + new MockExecPlan("downsample", logicalPlan) } } @@ -107,4 +105,13 @@ class LongTimeRangePlannerSpec extends FunSpec with Matchers { ep.lp shouldEqual logicalPlan } } + + it("should direct raw-cluster-only queries to raw planner for scalar vector queries") { + val logicalPlan = Parser.queryRangeToLogicalPlan("scalar(vector(1)) * 10", + TimeStepParams(now/1000 - 7.minutes.toSeconds, 1.minute.toSeconds, now/1000 - 1.minutes.toSeconds)) + + val ep = longTermPlanner.materialize(logicalPlan, QueryContext()).asInstanceOf[MockExecPlan] + ep.name shouldEqual "raw" + ep.lp shouldEqual logicalPlan + } } diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/ScalarQueriesSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/ScalarQueriesSpec.scala index 7926d606d7..c638fe62bc 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/ScalarQueriesSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/ScalarQueriesSpec.scala @@ -1,20 +1,15 @@ package filodb.coordinator.queryplanner -import scala.concurrent.duration.FiniteDuration - import akka.actor.ActorSystem import akka.testkit.TestProbe import com.typesafe.config.ConfigFactory -import monix.eval.Task -import monix.execution.Scheduler import org.scalatest.{FunSpec, Matchers} import filodb.coordinator.ShardMapper import filodb.core.MetricsTestData import filodb.core.metadata.Schemas -import filodb.core.query.{ColumnFilter, Filter, RangeParams} +import filodb.core.query.{ColumnFilter, Filter, PromQlQueryParams, QueryContext, RangeParams} import filodb.prometheus.parse.Parser -import filodb.query import filodb.query._ import filodb.query.ScalarFunctionId.Time import filodb.query.exec._ @@ -33,18 +28,12 @@ class ScalarQueriesSpec extends FunSpec with Matchers { val dsRef = dataset.ref val schemas = Schemas(dataset.schema) - val emptyDispatcher = new PlanDispatcher { - override def dispatch(plan: ExecPlan)(implicit sched: Scheduler, - timeout: FiniteDuration): Task[query.QueryResponse] = ??? - } - val engine = new SingleClusterPlanner(dsRef, schemas, mapperRef, earliestRetainedTimestampFn = 0) - val queryEngineConfigString = "routing {\n buddy {\n http {\n timeout = 10.seconds\n }\n }\n}" val queryEngineConfig = ConfigFactory.parseString(queryEngineConfigString) - val promQlQueryParams = PromQlQueryParams("sum(heap_usage)", 100, 1, 1000, None) + val promQlQueryParams = PromQlQueryParams(ConfigFactory.empty, "sum(heap_usage)", 100, 1, 1000, None) val f1 = Seq(ColumnFilter("__name__", Filter.Equals("http_request_duration_seconds_bucket")), diff --git a/coordinator/src/test/scala/filodb.coordinator/queryplanner/SingleClusterPlannerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryplanner/SingleClusterPlannerSpec.scala index 1fe8397161..ed810cc559 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryplanner/SingleClusterPlannerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryplanner/SingleClusterPlannerSpec.scala @@ -4,19 +4,21 @@ import scala.concurrent.duration._ import akka.actor.ActorSystem import akka.testkit.TestProbe +import com.typesafe.config.ConfigFactory import org.scalatest.{FunSpec, Matchers} +import org.scalatest.concurrent.ScalaFutures import filodb.coordinator.ShardMapper import filodb.coordinator.client.QueryCommands.{FunctionalSpreadProvider, StaticSpreadProvider} -import filodb.core.{MetricsTestData, SpreadChange} +import filodb.core.{GlobalScheduler, MetricsTestData, SpreadChange} import filodb.core.metadata.Schemas -import filodb.core.query.{ColumnFilter, Filter} +import filodb.core.query.{ColumnFilter, Filter, PromQlQueryParams, QueryContext} import filodb.prometheus.ast.{TimeStepParams, WindowConstants} import filodb.prometheus.parse.Parser import filodb.query._ import filodb.query.exec._ -class SingleClusterPlannerSpec extends FunSpec with Matchers { +class SingleClusterPlannerSpec extends FunSpec with Matchers with ScalaFutures { implicit val system = ActorSystem() private val node = TestProbe().ref @@ -58,7 +60,7 @@ class SingleClusterPlannerSpec extends FunSpec with Matchers { val raw2 = RawSeries(rangeSelector = intervalSelector, filters= f2, columns = Seq("value")) val windowed2 = PeriodicSeriesWithWindowing(raw2, from, 1000, to, 5000, RangeFunctionId.Rate) val summed2 = Aggregate(AggregationOperator.Sum, windowed2, Nil, Seq("job")) - val promQlQueryParams = PromQlQueryParams("sum(heap_usage)", 100, 1, 1000, None) + val promQlQueryParams = PromQlQueryParams(ConfigFactory.empty, "sum(heap_usage)", 100, 1, 1000, None) it ("should generate ExecPlan for LogicalPlan") { // final logical plan @@ -256,10 +258,9 @@ class SingleClusterPlannerSpec extends FunSpec with Matchers { } it("should bound queries until retention period and drop instants outside retention period") { - val planner = new SingleClusterPlanner(dsRef, schemas, mapperRef, - earliestRetainedTimestampFn = System.currentTimeMillis - 3.days.toMillis) - val nowSeconds = System.currentTimeMillis() / 1000 + val planner = new SingleClusterPlanner(dsRef, schemas, mapperRef, + earliestRetainedTimestampFn = nowSeconds * 1000 - 3.days.toMillis) // Case 1: no offset or window val logicalPlan1 = Parser.queryRangeToLogicalPlan("""foo{job="bar"}""", @@ -285,8 +286,7 @@ class SingleClusterPlannerSpec extends FunSpec with Matchers { + 1.minute.toMillis // step + 20.minutes.toMillis) // window - - // Case 2: offset and some window + // Case 3: offset and some window val logicalPlan3 = Parser.queryRangeToLogicalPlan("""rate(foo{job="bar"}[20m] offset 15m)""", TimeStepParams(nowSeconds - 4.days.toSeconds, 1.minute.toSeconds, nowSeconds)) @@ -299,6 +299,15 @@ class SingleClusterPlannerSpec extends FunSpec with Matchers { + 20.minutes.toMillis // window + 15.minutes.toMillis) // offset + // Case 4: outside retention + val logicalPlan4 = Parser.queryRangeToLogicalPlan("""foo{job="bar"}""", + TimeStepParams(nowSeconds - 10.days.toSeconds, 1.minute.toSeconds, nowSeconds - 5.days.toSeconds)) + val ep4 = planner.materialize(logicalPlan4, QueryContext()) + ep4.isInstanceOf[EmptyResultExec] shouldEqual true + import GlobalScheduler._ + val res = ep4.dispatcher.dispatch(ep4).runAsync.futureValue.asInstanceOf[QueryResult] + res.result.isEmpty shouldEqual true } -} + + } diff --git a/core/src/main/resources/filodb-defaults.conf b/core/src/main/resources/filodb-defaults.conf index c21b697f75..7fe98d5dea 100644 --- a/core/src/main/resources/filodb-defaults.conf +++ b/core/src/main/resources/filodb-defaults.conf @@ -37,7 +37,7 @@ filodb { copyTags = { "_ns_" = [ "_ns", "exporter", "job" ] } - ignoreShardKeyColumnSuffixes = { "__name__" = ["_bucket", "_count", "_sum"] } + ignoreShardKeyColumnSuffixes = { "_metric_" = ["_bucket", "_count", "_sum"] } ignoreTagsOnPartitionKeyHash = ["le"] metricColumn = "_metric_" shardKeyColumns = [ "_metric_", "_ws_", "_ns_" ] @@ -73,8 +73,7 @@ filodb { untyped { columns = ["timestamp:ts", "number:double"] value-column = "number" - downsamplers = [ "tTime(0)", "dMin(1)", "dMax(1)", "dSum(1)", "dCount(1)", "dAvg(1)" ] - downsample-schema = "ds-gauge" + downsamplers = [] } prom-counter { @@ -304,12 +303,6 @@ filodb { ds-index-job { - # config to run one time complete partkey migration from raw cluster to downsampler cluster. - # This is required in the following scenarios - # 1. Initial refresh of partkey index to downsampler cluster - # 2. For fixing corrupt downsampler index - migrate-full-index = false - # Name of the dataset from which to downsample # raw-dataset-name = "prometheus" diff --git a/core/src/main/scala/filodb.core/Messages.scala b/core/src/main/scala/filodb.core/Messages.scala index dc28de84f5..7efe5085f2 100644 --- a/core/src/main/scala/filodb.core/Messages.scala +++ b/core/src/main/scala/filodb.core/Messages.scala @@ -29,3 +29,5 @@ final case class NotFoundError(what: String) extends Exception(what) final case class StorageEngineException(t: Throwable) extends Exception(t) final case class MetadataException(t: Throwable) extends Exception(t) final case class SystemLimitsReachedException(msg: String) extends Exception(msg) +final case class QueryTimeoutException(queryTime: Long, timedOutAt: String) extends + Exception (s"Query timeout in $timedOutAt after ${queryTime/1000} seconds") \ No newline at end of file diff --git a/core/src/main/scala/filodb.core/binaryrecord2/RecordBuilder.scala b/core/src/main/scala/filodb.core/binaryrecord2/RecordBuilder.scala index ad49b7972f..c3a76a04b4 100644 --- a/core/src/main/scala/filodb.core/binaryrecord2/RecordBuilder.scala +++ b/core/src/main/scala/filodb.core/binaryrecord2/RecordBuilder.scala @@ -4,7 +4,8 @@ import com.typesafe.scalalogging.StrictLogging import org.agrona.DirectBuffer import scalaxy.loops._ -import filodb.core.metadata.{Column, DatasetOptions, PartitionSchema, Schema} +import filodb.core.binaryrecord2.RecordSchema.schemaID +import filodb.core.metadata.{Column, DatasetOptions, PartitionSchema, Schema, Schemas} import filodb.core.metadata.Column.ColumnType.{DoubleColumn, LongColumn, MapColumn, StringColumn} import filodb.core.query.ColumnInfo import filodb.memory._ @@ -688,4 +689,25 @@ object RecordBuilder { case _ => shardKeyColValue } } + + /** + * mutate dataschema of the partitionKey for downsampling, only when downsample dataschema is different + * than raw schema (e.g. Guages) + */ + final def updateSchema(partKeyBase: Any, partKeyOffset: Long, schema: Schema): Unit = { + UnsafeUtils.setShort(partKeyBase, partKeyOffset + 4, schema.schemaHash.toShort) + } + + /** + * Build a partkey from the source partkey and change the downsample schema. + * Useful during downsampling as dataschema may differ. + */ + final def buildDownsamplePartKey(pkBytes: Array[Byte], schemas: Schemas): Option[Array[Byte]] = { + val rawSchema = schemas(schemaID(pkBytes, UnsafeUtils.arayOffset)) + rawSchema.downsample.map { downSch => + val dsPkeyBytes = pkBytes.clone + updateSchema(dsPkeyBytes, UnsafeUtils.arayOffset, downSch) + dsPkeyBytes + } + } } diff --git a/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesShard.scala b/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesShard.scala index 881f90f102..f0b11dfc48 100644 --- a/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesShard.scala @@ -101,11 +101,15 @@ class DownsampledTimeSeriesShard(rawDatasetRef: DatasetRef, private def hour(millis: Long = System.currentTimeMillis()) = millis / 1000 / 60 / 60 def recoverIndex(): Future[Unit] = { - indexUpdatedHour.set(hour() - 1) indexBootstrapper.bootstrapIndex(partKeyIndex, shardNum, indexDataset){ _ => createPartitionID() } .map { count => logger.info(s"Bootstrapped index for dataset=$indexDataset shard=$shardNum with $count records") }.map { _ => + // need to start recovering 6 hours prior to now since last index migration could have run 6 hours ago + // and we'd be missing entries that would be migrated in the last 6 hours. + // Hence indexUpdatedHour should be: currentHour - 6 + val indexJobIntervalInHours = (downsampleStoreConfig.maxChunkTime.toMinutes + 59) / 60 // for ceil division + indexUpdatedHour.set(hour() - indexJobIntervalInHours - 1) startHousekeepingTask() startStatsUpdateTask() }.runAsync(housekeepingSched) @@ -155,7 +159,8 @@ class DownsampledTimeSeriesShard(rawDatasetRef: DatasetRef, // before refresh happens because we will not revist the hour again. val toHour = hour() - 2 val fromHour = indexUpdatedHour.get() + 1 - indexRefresher.refreshIndex(partKeyIndex, shardNum, rawDatasetRef, fromHour, toHour)(lookupOrCreatePartId) + indexRefresher.refreshWithDownsamplePartKeys(partKeyIndex, shardNum, rawDatasetRef, + fromHour, toHour, schemas)(lookupOrCreatePartId) .map { count => indexUpdatedHour.set(toHour) stats.indexEntriesRefreshed.increment(count) @@ -170,8 +175,7 @@ class DownsampledTimeSeriesShard(rawDatasetRef: DatasetRef, } private def startStatsUpdateTask(): Unit = { - logger.info(s"Starting Index Refresh task from raw dataset=$rawDatasetRef shard=$shardNum " + - s"every ${rawStoreConfig.flushInterval}") + logger.info(s"Starting Stats Update task from raw dataset=$rawDatasetRef shard=$shardNum every 1 minute") gaugeUpdateFuture = Observable.intervalWithFixedDelay(1.minute).map { _ => updateGauges() }.onErrorRestartUnlimited.completedL.runAsync(housekeepingSched) @@ -182,8 +186,8 @@ class DownsampledTimeSeriesShard(rawDatasetRef: DatasetRef, stats.indexRamBytes.update(partKeyIndex.indexRamBytes) } - private def lookupOrCreatePartId(pk: PartKeyRecord): Int = { - partKeyIndex.partIdFromPartKeySlow(pk.partKey, UnsafeUtils.arayOffset).getOrElse(createPartitionID()) + private def lookupOrCreatePartId(pk: Array[Byte]): Int = { + partKeyIndex.partIdFromPartKeySlow(pk, UnsafeUtils.arayOffset).getOrElse(createPartitionID()) } /** @@ -270,7 +274,7 @@ class DownsampledTimeSeriesShard(rawDatasetRef: DatasetRef, partKeyIndex.partKeyFromPartId(partID).map { pkBytesRef => val unsafeKeyOffset = PartKeyLuceneIndex.bytesRefToUnsafeOffset(pkBytesRef.offset) RecordSchema.schemaID(pkBytesRef.bytes, unsafeKeyOffset) - }.getOrElse(throw new IllegalStateException("PartId returned by lucene, but partKey not found")) + }.getOrElse(throw new IllegalStateException(s"PartId $partID returned by lucene, but partKey not found")) } private def chooseDownsampleResolution(chunkScanMethod: ChunkScanMethod): DatasetRef = { @@ -285,9 +289,9 @@ class DownsampledTimeSeriesShard(rawDatasetRef: DatasetRef, private def makePagedPartition(part: RawPartData, firstSchemaId: Int): ReadablePartition = { val schemaId = RecordSchema.schemaID(part.partitionKey, UnsafeUtils.arayOffset) - if (schemaId != firstSchemaId) - throw new IllegalArgumentException("Query involves results with multiple schema. " + - "Use type tag to provide narrower query") + 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) } @@ -339,8 +343,7 @@ class DownsampledTimeSeriesShard(rawDatasetRef: DatasetRef, // TODO small optimization for some other day util.Arrays.copyOfRange(partKeyBytes.get.bytes, partKeyBytes.get.offset, partKeyBytes.get.offset + partKeyBytes.get.length) - else throw new IllegalStateException("This is not an expected behavior." + - " PartId should always have a corresponding PartKey!") + else throw new IllegalStateException(s"Could not find partKey or partId $partId. This is not a expected behavior.") } def cleanup(): Unit = { diff --git a/core/src/main/scala/filodb.core/memstore/IndexBootstrapper.scala b/core/src/main/scala/filodb.core/memstore/IndexBootstrapper.scala index 3ecd3e32be..048b548821 100644 --- a/core/src/main/scala/filodb.core/memstore/IndexBootstrapper.scala +++ b/core/src/main/scala/filodb.core/memstore/IndexBootstrapper.scala @@ -5,6 +5,8 @@ import monix.eval.Task import monix.reactive.Observable import filodb.core.DatasetRef +import filodb.core.binaryrecord2.RecordBuilder +import filodb.core.metadata.Schemas import filodb.core.store.{ColumnStore, PartKeyRecord} class IndexBootstrapper(colStore: ColumnStore) { @@ -45,20 +47,23 @@ class IndexBootstrapper(colStore: ColumnStore) { } /** - * Refresh index - * @param fromHour - * @param toHour - * @param parallelism - * @param lookUpOrAssignPartId - * @return + * Refresh index with real-time data rom colStore's raw dataset + * @param fromHour fromHour inclusive + * @param toHour toHour inclusive + * @param parallelism number of threads to use to concurrently load the index + * @param lookUpOrAssignPartId function to invoke to assign (or lookup) partId to the partKey + * + * @return number of records refreshed */ - def refreshIndex(index: PartKeyLuceneIndex, + def refreshWithDownsamplePartKeys( + index: PartKeyLuceneIndex, shardNum: Int, ref: DatasetRef, fromHour: Long, toHour: Long, + schemas: Schemas, parallelism: Int = Runtime.getRuntime.availableProcessors()) - (lookUpOrAssignPartId: PartKeyRecord => Int): Task[Long] = { + (lookUpOrAssignPartId: Array[Byte] => Int): Task[Long] = { val tracer = Kamon.spanBuilder("downsample-store-refresh-index-latency") .asChildOf(Kamon.currentSpan()) .tag("dataset", ref.dataset) @@ -68,15 +73,17 @@ class IndexBootstrapper(colStore: ColumnStore) { colStore.getPartKeysByUpdateHour(ref, shardNum, hour) }.mapAsync(parallelism) { pk => Task { - val partId = lookUpOrAssignPartId(pk) - index.upsertPartKey(pk.partKey, partId, pk.startTime, pk.endTime)() + val downsamplPartKey = RecordBuilder.buildDownsamplePartKey(pk.partKey, schemas) + downsamplPartKey.foreach { dpk => + val partId = lookUpOrAssignPartId(dpk) + index.upsertPartKey(dpk, partId, pk.startTime, pk.endTime)() + } } } .countL .map { count => index.refreshReadersBlocking() tracer.finish() - count } } diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala index bacc167110..378053d3a2 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala @@ -737,7 +737,8 @@ class TimeSeriesShard(val ref: DatasetRef, val removedParts = debox.Buffer.empty[Int] InMemPartitionIterator2(partsToPurge).foreach { p => if (!p.ingesting) { - logger.debug(s"Purging partition with partId=${p.partID} from memory in dataset=$ref shard=$shardNum") + logger.debug(s"Purging partition with partId=${p.partID} ${p.stringPartition} from " + + s"memory in dataset=$ref shard=$shardNum") removePartition(p) removedParts += p.partID numDeleted += 1 @@ -946,12 +947,14 @@ class TimeSeriesShard(val ref: DatasetRef, val partKeyRecords = InMemPartitionIterator2(flushGroup.dirtyPartsToFlush).map { p => val pk = toPartKeyRecord(p) logger.debug(s"Adding entry into partKeys table partId=${p.partID} in dataset=$ref " + - s"shard=$shardNum partKey[${p.stringPartition}] with startTime=${pk.startTime} endTime=${pk.endTime}") + s"shard=$shardNum partKey[${p.stringPartition}] with startTime=${pk.startTime} endTime=${pk.endTime} " + + s"hash=${pk.hash}") pk } + val updateHour = System.currentTimeMillis() / 1000 / 60 / 60 colStore.writePartKeys(ref, shardNum, Observable.fromIterator(partKeyRecords), - storeConfig.diskTTLSeconds).map { case resp => + storeConfig.diskTTLSeconds, updateHour).map { case resp => if (flushGroup.dirtyPartsToFlush.length > 0) { logger.info(s"Finished flush of partKeys numPartKeys=${flushGroup.dirtyPartsToFlush.length}" + s" resp=$resp for dataset=$ref shard=$shardNum") @@ -1298,7 +1301,8 @@ class TimeSeriesShard(val ref: DatasetRef, if (endTime == PartKeyLuceneIndex.NOT_FOUND || endTime == Long.MaxValue) { logger.warn(s"endTime $endTime was not correct. how?", new IllegalStateException()) } else { - logger.debug(s"Evicting partId=${partitionObj.partID} from dataset=$ref shard=$shardNum") + logger.debug(s"Evicting partId=${partitionObj.partID} ${partitionObj.stringPartition} " + + s"from dataset=$ref shard=$shardNum") // add the evicted partKey to a bloom filter so that we are able to quickly // find out if a partId has been assigned to an ingesting partKey before a more expensive lookup. evictedPartKeys.synchronized { diff --git a/query/src/main/scala/filodb/query/QueryContext.scala b/core/src/main/scala/filodb.core/query/QueryContext.scala similarity index 87% rename from query/src/main/scala/filodb/query/QueryContext.scala rename to core/src/main/scala/filodb.core/query/QueryContext.scala index 2a0cabca69..e1b0df05a3 100644 --- a/query/src/main/scala/filodb/query/QueryContext.scala +++ b/core/src/main/scala/filodb.core/query/QueryContext.scala @@ -1,12 +1,18 @@ -package filodb.query +package filodb.core.query import java.util.UUID +import com.typesafe.config.Config + import filodb.core.{SpreadChange, SpreadProvider} -import filodb.core.query.{ColumnFilter, Filter} trait TsdbQueryParams -case class PromQlQueryParams(promQl: String, startSecs: Long, stepSecs: Long, endSecs: Long, + +/** + * This class provides PromQl query paramaters + * Config has routing parameters + */ +case class PromQlQueryParams(config: Config, promQl: String, startSecs: Long, stepSecs: Long, endSecs: Long, spread: Option[Int] = None, processFailure: Boolean = true) extends TsdbQueryParams case object UnavailablePromQlQueryParams extends TsdbQueryParams @@ -15,7 +21,7 @@ case object UnavailablePromQlQueryParams extends TsdbQueryParams */ final case class QueryContext(origQueryParams: TsdbQueryParams = UnavailablePromQlQueryParams, spreadOverride: Option[SpreadProvider] = None, - queryTimeoutSecs: Int = 30, + queryTimeoutMillis: Int = 30000, sampleLimit: Int = 1000000, shardOverrides: Option[Seq[Int]] = None, queryId: String = UUID.randomUUID().toString, diff --git a/core/src/main/scala/filodb.core/query/RangeVector.scala b/core/src/main/scala/filodb.core/query/RangeVector.scala index a2081a72ff..e1c029a061 100644 --- a/core/src/main/scala/filodb.core/query/RangeVector.scala +++ b/core/src/main/scala/filodb.core/query/RangeVector.scala @@ -124,16 +124,16 @@ final case class ScalarVaryingDouble(private val timeValueMap: Map[Long, Double] override def numRowsInt: Int = timeValueMap.size } -final case class RangeParams(start: Long, step: Long, end: Long) +final case class RangeParams(startSecs: Long, stepSecs: Long, endSecs: Long) trait ScalarSingleValue extends ScalarRangeVector { def rangeParams: RangeParams var numRowsInt : Int = 0 override def rows: Iterator[RowReader] = { - Iterator.from(0, rangeParams.step.toInt).takeWhile(_ <= rangeParams.end - rangeParams.start).map { i => + Iterator.from(0, rangeParams.stepSecs.toInt).takeWhile(_ <= rangeParams.endSecs - rangeParams.startSecs).map { i => numRowsInt += 1 - val t = i + rangeParams.start + val t = i + rangeParams.startSecs new TransientRow(t * 1000, getValue(t * 1000)) } } @@ -198,8 +198,8 @@ final case class DayOfMonthScalar(rangeParams: RangeParams) extends ScalarSingle */ final case class DayOfWeekScalar(rangeParams: RangeParams) extends ScalarSingleValue { override def getValue(time: Long): Double = { - val dayOfWeek = LocalDateTime.ofEpochSecond(time / 1000, 0, ZoneOffset.UTC).getDayOfWeek - if (dayOfWeek == 7) 0 else dayOfWeek.getValue + val dayOfWeek = LocalDateTime.ofEpochSecond(time / 1000, 0, ZoneOffset.UTC).getDayOfWeek.getValue + if (dayOfWeek == 7) 0 else dayOfWeek } } diff --git a/core/src/main/scala/filodb.core/query/ResultTypes.scala b/core/src/main/scala/filodb.core/query/ResultTypes.scala index ffae009e44..a5c706d737 100644 --- a/core/src/main/scala/filodb.core/query/ResultTypes.scala +++ b/core/src/main/scala/filodb.core/query/ResultTypes.scala @@ -52,9 +52,9 @@ final case class ResultSchema(columns: Seq[ColumnInfo], numRowKeyColumns: Int, columns(1).colType == HistogramColumn && columns(2).colType == DoubleColumn def hasSameColumnsAs(other: ResultSchema): Boolean = { - // exclude fixedVectorLen + // exclude fixedVectorLen & colIDs other.columns == columns && other.numRowKeyColumns == numRowKeyColumns && - other.brSchemas == brSchemas && other.colIDs == colIDs + other.brSchemas == brSchemas } } diff --git a/core/src/main/scala/filodb.core/store/ChunkSetInfo.scala b/core/src/main/scala/filodb.core/store/ChunkSetInfo.scala index ea149133ae..d808b97fa9 100644 --- a/core/src/main/scala/filodb.core/store/ChunkSetInfo.scala +++ b/core/src/main/scala/filodb.core/store/ChunkSetInfo.scala @@ -6,9 +6,10 @@ import com.googlecode.javaewah.EWAHCompressedBitmap import com.typesafe.scalalogging.StrictLogging import debox.Buffer +import filodb.core.QueryTimeoutException import filodb.core.Types._ import filodb.core.metadata.{Column, DataSchema} -import filodb.core.query.RawDataRangeVector +import filodb.core.query.{QueryContext, RawDataRangeVector} import filodb.memory.BinaryRegion.NativePointer import filodb.memory.MemFactory import filodb.memory.data.ElementIterator @@ -378,6 +379,7 @@ Exception(f"CorruptVector at 0x$ptr%016x startTime=$chunkStartTime shard=$shard * @param window the # of millis/time units that define the length of each window */ class WindowedChunkIterator(rv: RawDataRangeVector, start: Long, step: Long, end: Long, window: Long, + queryContext: QueryContext, // internal vars, put it here for better performance var curWindowEnd: Long = -1L, var curWindowStart: Long = -1L, @@ -417,6 +419,10 @@ extends Iterator[ChunkSetInfoReader] { // if new window end is beyond end of most recent chunkset, add more chunksets (if there are more) while (curWindowEnd > lastEndTime && infos.hasNext) { val next = infos.nextInfoReader + val queryTimeElapsed = System.currentTimeMillis() - queryContext.submitTime + if (queryTimeElapsed >= queryContext.queryTimeoutMillis) + throw QueryTimeoutException(queryTimeElapsed, this.getClass.getName) + // Add if next chunkset is within window and not empty. Otherwise keep going if (curWindowStart <= next.endTime && next.numRows > 0) { windowInfos += next diff --git a/core/src/main/scala/filodb.core/store/ChunkSink.scala b/core/src/main/scala/filodb.core/store/ChunkSink.scala index c78a66c80c..f2a1c126cb 100644 --- a/core/src/main/scala/filodb.core/store/ChunkSink.scala +++ b/core/src/main/scala/filodb.core/store/ChunkSink.scala @@ -48,7 +48,7 @@ trait ChunkSink { def writePartKeys(ref: DatasetRef, shard: Int, partKeys: Observable[PartKeyRecord], diskTTLSeconds: Int, - writeToPkUTTable: Boolean = true): Future[Response] + updateHour: Long, writeToPkUTTable: Boolean = true): Future[Response] /** * Initializes the ChunkSink for a given dataset. Must be called once before writing. */ @@ -157,7 +157,7 @@ class NullColumnStore(implicit sched: Scheduler) extends ColumnStore with Strict override def writePartKeys(ref: DatasetRef, shard: Int, partKeys: Observable[PartKeyRecord], diskTTLSeconds: Int, - writeToPkUTTable: Boolean = true): Future[Response] = { + updateHour: Long, writeToPkUTTable: Boolean = true): Future[Response] = { partKeys.countL.map(c => sinkStats.partKeysWrite(c.toInt)).runAsync.map(_ => Success) } diff --git a/core/src/test/scala/filodb.core/TestData.scala b/core/src/test/scala/filodb.core/TestData.scala index 32011208fe..5232f5604c 100644 --- a/core/src/test/scala/filodb.core/TestData.scala +++ b/core/src/test/scala/filodb.core/TestData.scala @@ -217,7 +217,7 @@ object GdeltTestData { object MachineMetricsData { import scala.util.Random.nextInt - val columns = Seq("timestamp:long", "min:double", "avg:double", "max:double", "count:long") + val columns = Seq("timestamp:ts", "min:double", "avg:double", "max:double", "count:long") val dummyContext = Map("test" -> "test") def singleSeriesData(initTs: Long = System.currentTimeMillis, @@ -346,7 +346,7 @@ object MachineMetricsData { Stream[Seq[Any]] = { val scheme = bv.GeometricBuckets(2.0, 2.0, numBuckets) histBucketScheme = scheme - val buckets = new Array[Double](numBuckets) + val buckets = new Array[Long](numBuckets) def updateBuckets(bucketNo: Int): Unit = { for { b <- bucketNo until numBuckets } { buckets(b) += 1 @@ -357,7 +357,7 @@ object MachineMetricsData { Seq(startTs + n * timeStep, (1 + n).toLong, buckets.sum.toLong, - bv.MutableHistogram(scheme, buckets.map(x => x)), + bv.LongHistogram(scheme, buckets.map(x => x)), "request-latency", extraTags ++ Map("_ws_".utf8 -> "demo".utf8, "_ns_".utf8 -> "testapp".utf8, "dc".utf8 -> s"${n % numSeries}".utf8)) } @@ -380,7 +380,7 @@ object MachineMetricsData { // Adds in the max column before h/hist def histMax(histStream: Stream[Seq[Any]]): Stream[Seq[Any]] = histStream.map { row => - val hist = row(3).asInstanceOf[bv.MutableHistogram] + val hist = row(3).asInstanceOf[bv.LongHistogram] // Set max to a fixed ratio of the "last bucket" top value, ie the last bucket with an actual increase val highestBucketVal = hist.bucketValue(hist.numBuckets - 1) val lastBucketNum = ((hist.numBuckets - 2) to 0 by -1).filter { b => hist.bucketValue(b) == highestBucketVal } diff --git a/core/src/test/scala/filodb.core/downsample/ShardDownsamplerSpec.scala b/core/src/test/scala/filodb.core/downsample/ShardDownsamplerSpec.scala index 17531fa815..6404239a33 100644 --- a/core/src/test/scala/filodb.core/downsample/ShardDownsamplerSpec.scala +++ b/core/src/test/scala/filodb.core/downsample/ShardDownsamplerSpec.scala @@ -226,7 +226,7 @@ class ShardDownsamplerSpec extends FunSpec with Matchers with BeforeAndAfterAll } val expectedSums = data.grouped(6).toSeq.map { dataRows => - dataRows.map(_(3).asInstanceOf[bv.MutableHistogram]) + dataRows.map(_(3).asInstanceOf[bv.LongHistogram]) .foldLeft(emptyAggHist) { case (agg, h) => agg.add(h); agg } } diff --git a/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala b/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala index 13f5a05f1c..46f70c2153 100644 --- a/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala +++ b/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala @@ -16,7 +16,7 @@ import filodb.core.metadata.Schemas import filodb.core.query.{ColumnFilter, Filter} import filodb.core.store._ import filodb.memory.format.{UnsafeUtils, ZeroCopyUTF8String} -import filodb.memory.format.vectors.MutableHistogram +import filodb.memory.format.vectors.LongHistogram class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter with ScalaFutures { implicit val s = monix.execution.Scheduler.Implicits.global @@ -121,7 +121,7 @@ class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter w val hists = memStore.scanRows(histDataset, Seq(3), FilteredPartitionScan(split, Seq(filter))) .map(_.getHistogram(0)) hists.zipWithIndex.foreach { case (h, i) => - h shouldEqual data(1 + 10*i)(3).asInstanceOf[MutableHistogram] + h shouldEqual data(1 + 10*i)(3).asInstanceOf[LongHistogram] } } diff --git a/core/src/test/scala/filodb.core/query/RangeVectorSpec.scala b/core/src/test/scala/filodb.core/query/RangeVectorSpec.scala index dbcafd1e60..c7e6419974 100644 --- a/core/src/test/scala/filodb.core/query/RangeVectorSpec.scala +++ b/core/src/test/scala/filodb.core/query/RangeVectorSpec.scala @@ -23,7 +23,7 @@ class RangeVectorSpec extends FunSpec with Matchers { } } - val cols = Array(new ColumnInfo("timestamp", ColumnType.LongColumn), + val cols = Array(new ColumnInfo("timestamp", ColumnType.TimestampColumn), new ColumnInfo("value", ColumnType.DoubleColumn)) it("should be able to create and read from SerializedRangeVector") { diff --git a/core/src/test/scala/filodb.core/store/ColumnStoreSpec.scala b/core/src/test/scala/filodb.core/store/ColumnStoreSpec.scala index a0dbe5eab4..e896aa6f5a 100644 --- a/core/src/test/scala/filodb.core/store/ColumnStoreSpec.scala +++ b/core/src/test/scala/filodb.core/store/ColumnStoreSpec.scala @@ -21,7 +21,7 @@ with BeforeAndAfter with BeforeAndAfterAll with ScalaFutures { import TestData._ implicit val defaultPatience = - PatienceConfig(timeout = Span(15, Seconds), interval = Span(250, Millis)) + PatienceConfig(timeout = Span(30, Seconds), interval = Span(250, Millis)) implicit val s = monix.execution.Scheduler.Implicits.global diff --git a/http/src/main/scala/filodb/http/PrometheusApiRoute.scala b/http/src/main/scala/filodb/http/PrometheusApiRoute.scala index 015b060718..d2ee28fbe5 100644 --- a/http/src/main/scala/filodb/http/PrometheusApiRoute.scala +++ b/http/src/main/scala/filodb/http/PrometheusApiRoute.scala @@ -7,6 +7,7 @@ import akka.http.scaladsl.model.{HttpEntity, HttpResponse, MediaTypes, StatusCod import akka.http.scaladsl.server.Directives._ import akka.stream.ActorMaterializer import akka.util.ByteString +import com.typesafe.config.ConfigFactory import com.typesafe.scalalogging.StrictLogging import de.heikoseeberger.akkahttpcirce.FailFastCirceSupport import org.xerial.snappy.Snappy @@ -15,6 +16,7 @@ import remote.RemoteStorage.ReadRequest import filodb.coordinator.client.IngestionCommands.UnknownDataset import filodb.coordinator.client.QueryCommands._ import filodb.core.{DatasetRef, SpreadChange, SpreadProvider} +import filodb.core.query.{PromQlQueryParams, QueryContext, TsdbQueryParams} import filodb.prometheus.ast.TimeStepParams import filodb.prometheus.parse.Parser import filodb.query._ @@ -45,8 +47,10 @@ class PrometheusApiRoute(nodeCoord: ActorRef, settings: HttpSettings)(implicit a { (query, start, end, histMap, step, explainOnly, verbose, spread) => val logicalPlan = Parser.queryRangeToLogicalPlan(query, TimeStepParams(start.toLong, step, end.toLong)) + // No cross-cluster failure routing in this API, hence we pass empty config askQueryAndRespond(dataset, logicalPlan, explainOnly.getOrElse(false), verbose.getOrElse(false), - spread, PromQlQueryParams(query, start.toLong, step.toLong, end.toLong, spread), histMap.getOrElse(false)) + spread, PromQlQueryParams(ConfigFactory.empty, query, start.toLong, step.toLong, end.toLong, spread), + histMap.getOrElse(false)) } } } ~ @@ -61,7 +65,8 @@ class PrometheusApiRoute(nodeCoord: ActorRef, settings: HttpSettings)(implicit a { (query, time, explainOnly, verbose, spread, histMap) => val logicalPlan = Parser.queryToLogicalPlan(query, time.toLong) askQueryAndRespond(dataset, logicalPlan, explainOnly.getOrElse(false), - verbose.getOrElse(false), spread, PromQlQueryParams(query, time.toLong, 1000, time.toLong, spread), + verbose.getOrElse(false), spread, PromQlQueryParams(ConfigFactory.empty, query, time.toLong, 1000, + time.toLong, spread), histMap.getOrElse(false)) } } diff --git a/jmh/src/main/scala/filodb.jmh/HistogramQueryBenchmark.scala b/jmh/src/main/scala/filodb.jmh/HistogramQueryBenchmark.scala index e62796c22f..239a2f83f2 100644 --- a/jmh/src/main/scala/filodb.jmh/HistogramQueryBenchmark.scala +++ b/jmh/src/main/scala/filodb.jmh/HistogramQueryBenchmark.scala @@ -19,11 +19,12 @@ 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.store._ import filodb.memory.MemFactory import filodb.memory.format.SeqRowReader import filodb.prometheus.parse.Parser -import filodb.query.{QueryConfig, QueryContext} +import filodb.query.QueryConfig //scalastyle:off regex /** @@ -89,7 +90,7 @@ class HistogramQueryBenchmark { // Single-threaded query test val numQueries = 500 val qContext = QueryContext(Some(new StaticSpreadProvider(SpreadChange(0, 1))), 100). - copy(shardOverrides = Some(Seq(0))) + copy(shardOverrides = Some(Seq(0)), queryTimeoutMillis = 60000) val hLogicalPlan = Parser.queryToLogicalPlan(histQuery, startTime/1000) val hExecPlan = hEngine.materialize(hLogicalPlan, qContext) val querySched = Scheduler.singleThread(s"benchmark-query") @@ -110,7 +111,7 @@ class HistogramQueryBenchmark { @OperationsPerInvocation(500) def histSchemaQuantileQuery(): Long = { val f = Observable.fromIterable(0 until numQueries).mapAsync(1) { n => - hExecPlan.execute(memStore, queryConfig)(querySched, 60.seconds) + hExecPlan.execute(memStore, queryConfig)(querySched) }.executeOn(querySched) .countL.runAsync Await.result(f, 60.seconds) @@ -122,7 +123,7 @@ class HistogramQueryBenchmark { @OperationsPerInvocation(500) def promSchemaQuantileQuery(): Long = { val f = Observable.fromIterable(0 until numQueries).mapAsync(1) { n => - pExecPlan.execute(memStore, queryConfig)(querySched, 60.seconds) + pExecPlan.execute(memStore, queryConfig)(querySched) }.executeOn(querySched) .countL.runAsync Await.result(f, 60.seconds) diff --git a/jmh/src/main/scala/filodb.jmh/QueryAndIngestBenchmark.scala b/jmh/src/main/scala/filodb.jmh/QueryAndIngestBenchmark.scala index ddb107682c..13ae91929d 100644 --- a/jmh/src/main/scala/filodb.jmh/QueryAndIngestBenchmark.scala +++ b/jmh/src/main/scala/filodb.jmh/QueryAndIngestBenchmark.scala @@ -17,12 +17,13 @@ import filodb.core.GlobalConfig import filodb.core.SpreadChange import filodb.core.binaryrecord2.RecordContainer import filodb.core.memstore.{SomeData, TimeSeriesMemStore} +import filodb.core.query.QueryContext import filodb.core.store.StoreConfig import filodb.gateway.GatewayServer import filodb.gateway.conversion.PrometheusInputRecord import filodb.prometheus.ast.TimeStepParams import filodb.prometheus.parse.Parser -import filodb.query.{QueryContext, QueryError => QError, QueryResult => QueryResult2} +import filodb.query.{QueryError => QError, QueryResult => QueryResult2} import filodb.timeseries.TestTimeseriesProducer //scalastyle:off regex diff --git a/jmh/src/main/scala/filodb.jmh/QueryHiCardInMemoryBenchmark.scala b/jmh/src/main/scala/filodb.jmh/QueryHiCardInMemoryBenchmark.scala index 1334732d14..8d1e2d1b94 100644 --- a/jmh/src/main/scala/filodb.jmh/QueryHiCardInMemoryBenchmark.scala +++ b/jmh/src/main/scala/filodb.jmh/QueryHiCardInMemoryBenchmark.scala @@ -18,10 +18,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.store.StoreConfig import filodb.prometheus.ast.TimeStepParams import filodb.prometheus.parse.Parser -import filodb.query.{QueryConfig, QueryContext} +import filodb.query.QueryConfig import filodb.query.exec.ExecPlan import filodb.timeseries.TestTimeseriesProducer diff --git a/jmh/src/main/scala/filodb.jmh/QueryInMemoryBenchmark.scala b/jmh/src/main/scala/filodb.jmh/QueryInMemoryBenchmark.scala index 3b1480ceea..699f8b2f73 100644 --- a/jmh/src/main/scala/filodb.jmh/QueryInMemoryBenchmark.scala +++ b/jmh/src/main/scala/filodb.jmh/QueryInMemoryBenchmark.scala @@ -19,10 +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.store.StoreConfig import filodb.prometheus.ast.TimeStepParams import filodb.prometheus.parse.Parser -import filodb.query.{QueryConfig, QueryContext, QueryError => QError, QueryResult => QueryResult2} +import filodb.query.{QueryConfig, QueryError => QError, QueryResult => QueryResult2} import filodb.timeseries.TestTimeseriesProducer //scalastyle:off regex @@ -184,7 +185,7 @@ class QueryInMemoryBenchmark extends StrictLogging { @OperationsPerInvocation(500) def singleThreadedRawQuery(): Long = { val f = Observable.fromIterable(0 until numQueries).mapAsync(1) { n => - execPlan.execute(cluster.memStore, queryConfig)(querySched, 60.seconds) + execPlan.execute(cluster.memStore, queryConfig)(querySched) }.executeOn(querySched) .countL.runAsync Await.result(f, 60.seconds) @@ -200,7 +201,7 @@ 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, 60.seconds) + minEP.execute(cluster.memStore, queryConfig)(querySched) }.executeOn(querySched) .countL.runAsync Await.result(f, 60.seconds) @@ -216,7 +217,7 @@ 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, 60.seconds) + sumRateEP.execute(cluster.memStore, queryConfig)(querySched) }.executeOn(querySched) .countL.runAsync Await.result(f, 60.seconds) diff --git a/jmh/src/main/scala/filodb.jmh/QueryOnDemandBenchmark.scala b/jmh/src/main/scala/filodb.jmh/QueryOnDemandBenchmark.scala index 24118c1a09..b7db80b42c 100644 --- a/jmh/src/main/scala/filodb.jmh/QueryOnDemandBenchmark.scala +++ b/jmh/src/main/scala/filodb.jmh/QueryOnDemandBenchmark.scala @@ -16,10 +16,11 @@ import org.openjdk.jmh.annotations.{Level => JmhLevel, _} import filodb.core.SpreadChange import filodb.core.binaryrecord2.RecordContainer import filodb.core.memstore.{SomeData, TimeSeriesMemStore} +import filodb.core.query.QueryContext import filodb.core.store.StoreConfig import filodb.prometheus.ast.TimeStepParams import filodb.prometheus.parse.Parser -import filodb.query.{QueryContext, QueryError => QError, QueryResult => QueryResult2} +import filodb.query.{QueryError => QError, QueryResult => QueryResult2} import filodb.timeseries.TestTimeseriesProducer //scalastyle:off regex diff --git a/memory/src/main/scala/filodb.memory/format/vectors/DoubleVector.scala b/memory/src/main/scala/filodb.memory/format/vectors/DoubleVector.scala index 945d48f90c..48fb52013a 100644 --- a/memory/src/main/scala/filodb.memory/format/vectors/DoubleVector.scala +++ b/memory/src/main/scala/filodb.memory/format/vectors/DoubleVector.scala @@ -236,15 +236,19 @@ object DoubleVectorDataReader64 extends DoubleVectorDataReader { s"length=${length(acc, vector)}") var addr = vector + OffsetData + start * 8 val untilAddr = vector + OffsetData + end * 8 + 8 // one past the end - var sum: Double = 0d + var sum: Double = Double.NaN if (ignoreNaN) { while (addr < untilAddr) { val nextDbl = acc.getDouble(addr) // There are many possible values of NaN. Use a function to ignore them reliably. - if (!java.lang.Double.isNaN(nextDbl)) sum += nextDbl + if (!java.lang.Double.isNaN(nextDbl)) { + if (sum.isNaN) sum = 0d + sum += nextDbl + } addr += 8 } } else { + sum = 0d while (addr < untilAddr) { sum += acc.getDouble(addr) addr += 8 diff --git a/memory/src/main/scala/filodb.memory/format/vectors/Histogram.scala b/memory/src/main/scala/filodb.memory/format/vectors/Histogram.scala index 321f6f749d..5be25be45a 100644 --- a/memory/src/main/scala/filodb.memory/format/vectors/Histogram.scala +++ b/memory/src/main/scala/filodb.memory/format/vectors/Histogram.scala @@ -47,7 +47,8 @@ trait Histogram extends Ordered[Histogram] { bucketNo } - final def topBucketValue: Double = bucketValue(numBuckets - 1) + final def topBucketValue: Double = + if (numBuckets <= 0) Double.NaN else bucketValue(numBuckets - 1) /** * Calculates histogram quantile based on bucket values using Prometheus scheme (increasing/LE) @@ -184,7 +185,7 @@ final case class MutableHistogram(buckets: HistogramBuckets, values: Array[Doubl final def bucketValue(no: Int): Double = values(no) final def serialize(intoBuf: Option[MutableDirectBuffer] = None): MutableDirectBuffer = { val buf = intoBuf.getOrElse(BinaryHistogram.histBuf) - BinaryHistogram.writeDelta(buckets, values.map(_.toLong), buf) + BinaryHistogram.writeDoubles(buckets, values, buf) buf } @@ -265,6 +266,12 @@ object MutableHistogram { def empty(buckets: HistogramBuckets): MutableHistogram = MutableHistogram(buckets, Array.fill(buckets.numBuckets)(Double.NaN)) + def fromPacked(bucketDef: HistogramBuckets, packedValues: DirectBuffer): Option[MutableHistogram] = { + val values = new Array[Double](bucketDef.numBuckets) + val res = NibblePack.unpackDoubleXOR(packedValues, values) + if (res == NibblePack.Ok) Some(MutableHistogram(bucketDef, values)) else None + } + def apply(h: Histogram): MutableHistogram = h match { case hb: HistogramWithBuckets => MutableHistogram(hb.buckets, hb.valueArray) case other: Histogram => ??? diff --git a/memory/src/main/scala/filodb.memory/format/vectors/HistogramVector.scala b/memory/src/main/scala/filodb.memory/format/vectors/HistogramVector.scala index 94e900fb8d..5df2d5cf4b 100644 --- a/memory/src/main/scala/filodb.memory/format/vectors/HistogramVector.scala +++ b/memory/src/main/scala/filodb.memory/format/vectors/HistogramVector.scala @@ -66,6 +66,12 @@ object BinaryHistogram extends StrictLogging { case HistFormat_Custom_Delta => val bucketDef = HistogramBuckets.custom(buf.byteArray, bucketDefOffset - 2) LongHistogram.fromPacked(bucketDef, valuesByteSlice).getOrElse(Histogram.empty) + case HistFormat_Geometric_XOR => + val bucketDef = HistogramBuckets.geometric(buf.byteArray, bucketDefOffset, false) + MutableHistogram.fromPacked(bucketDef, valuesByteSlice).getOrElse(Histogram.empty) + case HistFormat_Custom_XOR => + val bucketDef = HistogramBuckets.custom(buf.byteArray, bucketDefOffset - 2) + MutableHistogram.fromPacked(bucketDef, valuesByteSlice).getOrElse(Histogram.empty) case x => logger.debug(s"Unrecognizable histogram format code $x, returning empty histogram") Histogram.empty @@ -97,6 +103,8 @@ object BinaryHistogram extends StrictLogging { val HistFormat_Geometric_Delta = 0x03.toByte val HistFormat_Geometric1_Delta = 0x04.toByte val HistFormat_Custom_Delta = 0x05.toByte + val HistFormat_Geometric_XOR = 0x08.toByte // Double values XOR compressed + val HistFormat_Custom_XOR = 0x0a.toByte def isValidFormatCode(code: Byte): Boolean = (code == HistFormat_Null) || (code == HistFormat_Geometric1_Delta) || (code == HistFormat_Geometric_Delta) || @@ -127,7 +135,7 @@ object BinaryHistogram extends StrictLogging { writeDelta(buckets, values, histBuf) /** - * Encodes binary histogram with geometric bucket definition and data which is strictly increasing and positive. + * Encodes binary histogram with integral data which is strictly nondecreasing and positive. * All histograms after ingestion are expected to be increasing. * Delta encoding is applied for compression. * @param buf the buffer to write the histogram to. Highly recommended this be an ExpandableArrayBuffer or equiv. @@ -136,10 +144,9 @@ object BinaryHistogram extends StrictLogging { */ def writeDelta(buckets: HistogramBuckets, values: Array[Long], buf: MutableDirectBuffer): Int = { require(buckets.numBuckets == values.size, s"Values array size of ${values.size} != ${buckets.numBuckets}") - val formatCode = buckets match { + val formatCode = if (buckets.numBuckets == 0) HistFormat_Null else buckets match { case g: GeometricBuckets if g.minusOne => HistFormat_Geometric1_Delta case g: GeometricBuckets => HistFormat_Geometric_Delta - case c: CustomBuckets if c.numBuckets == 0 => HistFormat_Null case c: CustomBuckets => HistFormat_Custom_Delta } @@ -153,6 +160,30 @@ object BinaryHistogram extends StrictLogging { buf.putShort(0, (finalPos - 2).toShort) finalPos } + + /** + * Encodes binary histogram with double data, XOR compressed with NibblePack. + * @param buf the buffer to write the histogram to. Highly recommended this be an ExpandableArrayBuffer or equiv. + * so it can grow. + * @return the number of bytes written, including the length prefix + */ + def writeDoubles(buckets: HistogramBuckets, values: Array[Double], buf: MutableDirectBuffer): Int = { + require(buckets.numBuckets == values.size, s"Values array size of ${values.size} != ${buckets.numBuckets}") + val formatCode = if (buckets.numBuckets == 0) HistFormat_Null else buckets match { + case g: GeometricBuckets => HistFormat_Geometric_XOR + case c: CustomBuckets => HistFormat_Custom_XOR + } + + buf.putByte(2, formatCode) + val finalPos = if (formatCode == HistFormat_Null) { 3 } + else { + val valuesIndex = buckets.serialize(buf, 3) + NibblePack.packDoubles(values, buf, valuesIndex) + } + require(finalPos <= 65535, s"Histogram data is too large: $finalPos bytes needed") + buf.putShort(0, (finalPos - 2).toShort) + finalPos + } } object HistogramVector extends StrictLogging { diff --git a/memory/src/test/scala/filodb.memory/format/vectors/HistogramTest.scala b/memory/src/test/scala/filodb.memory/format/vectors/HistogramTest.scala index 81bbda7c01..5b8d22b508 100644 --- a/memory/src/test/scala/filodb.memory/format/vectors/HistogramTest.scala +++ b/memory/src/test/scala/filodb.memory/format/vectors/HistogramTest.scala @@ -102,6 +102,16 @@ class HistogramTest extends NativeVectorTest { } } + it("should serialize to and from BinHistograms for MutableHistograms with doubles") { + val dblHist = MutableHistogram(mutableHistograms.head.buckets, + mutableHistograms.head.values.map(_ + .5)) + val buf = new ExpandableArrayBuffer() + dblHist.serialize(Some(buf)) + + val deserHist = BinaryHistogram.BinHistogram(buf).toHistogram + deserHist shouldEqual dblHist + } + it("should serialize to and from BinaryHistograms with custom buckets") { val longHist = LongHistogram(customScheme, Array[Long](10, 15, 17, 20, 25, 34, 76)) val buf = new ExpandableArrayBuffer() diff --git a/prometheus/src/main/scala/filodb/prometheus/ast/Functions.scala b/prometheus/src/main/scala/filodb/prometheus/ast/Functions.scala index 46dfff4495..a8ad23c897 100644 --- a/prometheus/src/main/scala/filodb/prometheus/ast/Functions.scala +++ b/prometheus/src/main/scala/filodb/prometheus/ast/Functions.scala @@ -21,6 +21,67 @@ trait Functions extends Base with Operators with Vectors { throw new IllegalArgumentException(s"Invalid function name [$name]") } + // Below code is for validating the syntax of promql functions belonging to RangeFunctionID. + // It takes care of validating syntax of the tokenized input query before creating the logical plan. + // In case of invalid params/invalid syntax, then throw exceptions with similar error-messages like promql. + + // error messages + val errWrongArgumentCount = "argument(s) in call to function " + + val functionId = RangeFunctionId.withNameLowercaseOnlyOption(name.toLowerCase) + if (functionId.nonEmpty) { + val funcName = functionId.get.entryName + // get the parameter spec of the function from RangeFunctionID + val paramSpec = functionId.get.paramSpec + + // if the length of the args in param spec != to the args in + // the i/p query, then the i/p query is INCORRECT, + // throw invalid no. of args exception. + if (paramSpec.length != allParams.length) + throw new IllegalArgumentException(s"Expected ${paramSpec.length} " + + s"$errWrongArgumentCount $funcName, got ${allParams.size}") + + // if length of param spec and all params is same, + // then check the type of each argument and check the order of the arguments. + else { + paramSpec.zipWithIndex.foreach { + case (specType, index) => specType match { + case RangeVectorParam(errorMsg) => + if (!allParams(index).isInstanceOf[RangeExpression]) + throw new IllegalArgumentException(s"$errorMsg $funcName, " + + s"got ${allParams(index).getClass.getSimpleName}") + + case InstantVectorParam(errorMsg) => + if (!allParams(index).isInstanceOf[InstantExpression]) + throw new IllegalArgumentException(s"$errorMsg $funcName, " + + s"got ${allParams(index).getClass.getSimpleName}") + + case ScalarParam(errorMsg) => + if (!allParams(index).isInstanceOf[ScalarExpression]) + throw new IllegalArgumentException(s"$errorMsg $funcName, " + + s"got ${allParams(index).getClass.getSimpleName}") + + case ScalarRangeParam(min, max, errorMsg) => + val paramObj = allParams(index) + // Function like "Holt-winters" needs trend & smoothing factor between 0 and 1. + // If the obj is Scalar Expression, validate the value of the obj to be between 0 and 1. + // If the obj is not Scalar Expression, then throw exception. + if (!paramObj.isInstanceOf[ScalarExpression]) + throw new IllegalArgumentException(s"$errorMsg $funcName, " + + s"got ${allParams(index).getClass.getSimpleName}") + + else { + val paramValue = paramObj.asInstanceOf[ScalarExpression].toScalar + if (!(paramValue > min && paramValue < max)) + throw new IllegalArgumentException(s"$errorMsg $paramValue") + } + + case _ => throw new IllegalArgumentException("Invalid Query") + } + } + } + } + /** * * @return true when function is scalar or time @@ -83,6 +144,7 @@ trait Functions extends Base with Operators with Vectors { def toSeriesPlanMisc(seriesParam: Series, otherParams: Seq[FunctionArgsPlan], timeParams: TimeRangeParams): PeriodicSeriesPlan = { + val miscellaneousFunctionIdOpt = MiscellaneousFunctionId.withNameInsensitiveOption(name) val scalarFunctionIdOpt = ScalarFunctionId.withNameInsensitiveOption(name) val sortFunctionIdOpt = SortFunctionId.withNameInsensitiveOption(name) diff --git a/prometheus/src/main/scala/filodb/prometheus/parse/Parser.scala b/prometheus/src/main/scala/filodb/prometheus/parse/Parser.scala index 69cadaee85..354187909e 100644 --- a/prometheus/src/main/scala/filodb/prometheus/parse/Parser.scala +++ b/prometheus/src/main/scala/filodb/prometheus/parse/Parser.scala @@ -293,15 +293,26 @@ trait Expression extends Aggregates with Selector with Numeric with Join { case name ~ params => Function(name.str, params) } - lazy val aggregateExpression: PackratParser[AggregateExpression] = - aggregateOperator ~ functionParams.? ~ aggregateGrouping.? ~ functionParams.? ^^ { - case fn ~ params ~ ag ~ ls => AggregateExpression( - fn, params.getOrElse(Seq.empty), ag, ls.getOrElse(Seq.empty) + // For queries with aggregateGrouping before metric name + // Example: sum without (sum_label) (some_metric) + lazy val aggregateExpression1: PackratParser[AggregateExpression] = + aggregateOperator ~ aggregateGrouping.? ~ functionParams ~ functionParams.? ^^ { + case fn ~ ag ~ params ~ ls => AggregateExpression( + fn, params, ag, ls.getOrElse(Seq.empty) + ) + } + + // For queries with aggregateGrouping after metric name + // Example: sum (some_metric) without (some_label) + lazy val aggregateExpression2: PackratParser[AggregateExpression] = + aggregateOperator ~ functionParams ~ aggregateGrouping.? ~ functionParams.? ^^ { + case fn ~ params ~ ag ~ ls => AggregateExpression( + fn, params, ag, ls.getOrElse(Seq.empty) ) } lazy val expression: PackratParser[Expression] = - binaryExpression | aggregateExpression | + binaryExpression | aggregateExpression2 | aggregateExpression1 | function | unaryExpression | vector | numericalExpression | simpleSeries | "(" ~> expression <~ ")" } diff --git a/prometheus/src/main/scala/filodb/prometheus/query/PrometheusModel.scala b/prometheus/src/main/scala/filodb/prometheus/query/PrometheusModel.scala index 4ebcd9837f..5a621dde0b 100644 --- a/prometheus/src/main/scala/filodb/prometheus/query/PrometheusModel.scala +++ b/prometheus/src/main/scala/filodb/prometheus/query/PrometheusModel.scala @@ -75,7 +75,8 @@ object PrometheusModel { } def toPromSuccessResponse(qr: FiloQueryResult, verbose: Boolean): SuccessResponse = { - val results = if (qr.resultSchema.columns(1).colType == ColumnType.HistogramColumn) + val results = if (qr.resultSchema.columns.nonEmpty && + qr.resultSchema.columns(1).colType == ColumnType.HistogramColumn) qr.result.map(toHistResult(_, verbose, qr.resultType)) else qr.result.map(toPromResult(_, verbose, qr.resultType)) diff --git a/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala b/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala index 5df0645566..81ef2a13c2 100644 --- a/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala +++ b/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala @@ -174,7 +174,6 @@ class ParserSpec extends FunSpec with Matchers { parseError("some_metric OFFSET 1m[5m]") parseError("(foo + bar)[5m]") - parseSuccessfully("sum by (foo)(some_metric)") parseSuccessfully("avg by (foo)(some_metric)") parseSuccessfully("max by (foo)(some_metric)") @@ -186,9 +185,15 @@ class ParserSpec extends FunSpec with Matchers { parseSuccessfully("topk(5, some_metric)") parseSuccessfully("count_values(\"value\",some_metric)") parseSuccessfully("sum without(and, by, avg, count, alert, annotations)(some_metric)") - parseSuccessfully("sum_over_time(foo)") - - + parseSuccessfully("sum:some_metric:dataset:1m{_ws_=\"some_workspace\", _ns_=\"some_namespace\"}") + parseSuccessfully("count:some_metric:dataset:1m{_ws_=\"some_workspace\", _ns_=\"some_namespace\"}") + parseSuccessfully("avg:some_metric:dataset:1m{_ws_=\"some_workspace\", _ns_=\"some_namespace\"}") + parseSuccessfully("min:some_metric:dataset:1m{_ws_=\"some_workspace\", _ns_=\"some_namespace\"}") + parseSuccessfully("max:some_metric:dataset:1m{_ws_=\"some_workspace\", _ns_=\"some_namespace\"}") + parseSuccessfully("stddev:some_metric:dataset:1m{_ws_=\"some_workspace\", _ns_=\"some_namespace\"}") + parseSuccessfully("stdvar:some_metric:dataset:1m{_ws_=\"some_workspace\", _ns_=\"some_namespace\"}") + + parseError("sum_over_time(foo)") parseError("sum(other_metric) by (foo)(some_metric)") parseError("sum without(==)(some_metric)") parseError("MIN keep_common (some_metric)") @@ -239,8 +244,135 @@ class ParserSpec extends FunSpec with Matchers { parseSuccessfully("or1{job=\"SNRT-App-0\"}[1m] ") parseSuccessfully("and1{job=\"SNRT-App-0\"}[1m] ") parseSuccessfully("and{job=\"SNRT-App-0\"}[1m] ") + + // negative/positive test-cases for functions in RangeFunctionID + // avg_over_time + parseSuccessfully("avg_over_time(some_metric[5m])") + parseError("avg_over_time(some_metric)") // reason : Expected range-vector + parseError("avg_over_time(some_metric[5m], hello)") // reason : Expected only 1 arg, got 2 + parseError("avg_over_time(abcd, some_metric[5m])") // reason : Expected range, got instant + + // changes + parseSuccessfully("changes(some_metric[5m])") + parseError("changes(some_metric)") // reason : Expected range-vector + parseError("changes(some_metric[5m], hello)") // reason : Expected only 1 arg, got 2 + parseError("changes(abcd, some_metric[5m])") // reason : Expected range, got instant + + // count_over_time + parseSuccessfully("count_over_time(some_metric[5m])") + parseError("count_over_time(some_metric)") // reason : Expected range-vector + parseError("count_over_time(some_metric[5m], hello)") // reason : Expected only 1 arg, got 2 + parseError("count_over_time(hello, some_metric[5m])") // reason : Expected range, got instant + + // delta + parseSuccessfully("delta(some_metric[5m])") + parseError("delta(some_metric)") // reason : Expected range-vector + parseError("delta(some_metric[5m], hello)") // reason : Expected only 1 arg, got 2 + parseError("delta(hello, some_metric[5m])") // reason : Expected range, got instant + + // deriv + parseSuccessfully("deriv(some_metric[5m])") + parseError("delta(some_metric)") // reason : Expected range-vector + parseError("deriv(some_metric[5m], hello)") // reason : Expected only 1 arg, got 2 + parseError("deriv(hello, some_metric[5m])") // reason : Expected range, got instant + + // holt_winters + parseSuccessfully("holt_winters(some_metric[5m], 0.5, 0.5)") + parseError("holt_winters(some_metric, 0.5, 0.5)") // reason : Expected range-vector, got instant + parseError("holt_winters(some_metric[5m])") // reason : Expected 3 args, got 1 + parseError("holt_winters(some_metric[5m], 1, 0.1 )") // reason : Invalid smoothing value, 0 "ScalarVectorBinaryOperation(ADD,ScalarTimeBasedPlan(Time,RangeParams(1524855988,1000,1524855988)),PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000,None),false)", @@ -315,11 +447,9 @@ class ParserSpec extends FunSpec with Matchers { "changes(http_requests_total{job=\"api-server\"}[5m])" -> "PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(job,Equals(api-server)), ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000,300000,Changes,List(),None)", - "quantile_over_time(http_requests_total{job=\"api-server\"}[5m], 0.4)" -> + "quantile_over_time(0.4,http_requests_total{job=\"api-server\"}[5m])" -> "PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(job,Equals(api-server)), ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000,300000,QuantileOverTime,List(ScalarFixedDoublePlan(0.4,RangeParams(1524855988,1000,1524855988))),None)", - "quantile_over_time(http_requests_total{job=\"api-server\"}[5m], Scalar(http_requests_total))" -> - "PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(job,Equals(api-server)), ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000,300000,QuantileOverTime,List(ScalarVaryingDoublePlan(PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000,None),Scalar,List())),None)", - "label_replace(http_requests_total,instance,new-label,instance,\"(.*)-(.*)\")" -> "ApplyMiscellaneousFunction(PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000,None),LabelReplace,List(instance, new-label, instance, (.*)-(.*)))", + "label_replace(http_requests_total,instance,new-label,instance,\"(.*)-(.*)\")" -> "ApplyMiscellaneousFunction(PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000,None),LabelReplace,List(instance, new-label, instance, (.*)-(.*)))", "hist_to_prom_vectors(http_request_latency)" -> "ApplyMiscellaneousFunction(PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(http_request_latency))),List()),1524855988000,1000000,1524855988000,None),HistToPromVectors,List())", "holt_winters(http_requests_total{job=\"api-server\"}[5m], 0.01, 0.1)" -> @@ -367,7 +497,8 @@ class ParserSpec extends FunSpec with Matchers { "absent(http_requests_total{host=\"api-server\"})" -> "ApplyAbsentFunction(PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(host,Equals(api-server)), ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000,None),List(ColumnFilter(host,Equals(api-server)), ColumnFilter(__name__,Equals(http_requests_total))),RangeParams(1524855988,1000,1524855988),List())", "count_values(\"freq\", http_requests_total)" -> "Aggregate(CountValues,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000,None),List(\"freq\"),List(),List())", - "timestamp(http_requests_total)" -> "PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000,0,Timestamp,List(),None)" + "timestamp(http_requests_total)" -> "PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000,0,Timestamp,List(),None)", + "sum:some_metric:dataset:1m{_ws_=\"demo\", _ns_=\"test\"}" -> "PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(_ws_,Equals(demo)), ColumnFilter(_ns_,Equals(test)), ColumnFilter(__name__,Equals(sum:some_metric:dataset:1m))),List()),1524855988000,1000000,1524855988000,None)" ) val qts: Long = 1524855988L diff --git a/query/src/main/scala/filodb/query/LogicalPlan.scala b/query/src/main/scala/filodb/query/LogicalPlan.scala index 149aadcfa5..7cd7bd60e1 100644 --- a/query/src/main/scala/filodb/query/LogicalPlan.scala +++ b/query/src/main/scala/filodb/query/LogicalPlan.scala @@ -266,6 +266,7 @@ final case class ScalarVaryingDoublePlan(vectors: PeriodicSeriesPlan, override def startMs: Long = vectors.startMs override def stepMs: Long = vectors.stepMs override def endMs: Long = vectors.endMs + override def isRoutable: Boolean = vectors.isRoutable } /** @@ -274,9 +275,9 @@ final case class ScalarVaryingDoublePlan(vectors: PeriodicSeriesPlan, */ final case class ScalarTimeBasedPlan(function: ScalarFunctionId, rangeParams: RangeParams) extends ScalarPlan { override def isRoutable: Boolean = false - override def startMs: Long = rangeParams.start - override def stepMs: Long = rangeParams.step - override def endMs: Long = rangeParams.end + override def startMs: Long = rangeParams.startSecs * 1000 + override def stepMs: Long = rangeParams.stepSecs * 1000 + override def endMs: Long = rangeParams.endSecs * 1000 } /** @@ -287,9 +288,9 @@ final case class ScalarFixedDoublePlan(scalar: Double, timeStepParams: RangeParams) extends ScalarPlan with FunctionArgsPlan { override def isRoutable: Boolean = false - override def startMs: Long = timeStepParams.start - override def stepMs: Long = timeStepParams.step - override def endMs: Long = timeStepParams.end + override def startMs: Long = timeStepParams.startSecs * 1000 + override def stepMs: Long = timeStepParams.stepSecs * 1000 + override def endMs: Long = timeStepParams.endSecs * 1000 } //scalastyle:off number.of.types diff --git a/query/src/main/scala/filodb/query/PlanEnums.scala b/query/src/main/scala/filodb/query/PlanEnums.scala index fd8dcaf4d1..60f69e50d1 100644 --- a/query/src/main/scala/filodb/query/PlanEnums.scala +++ b/query/src/main/scala/filodb/query/PlanEnums.scala @@ -9,94 +9,67 @@ object InstantFunctionId extends Enum[InstantFunctionId] { val values = findValues case object Abs extends InstantFunctionId("abs") - case object Ceil extends InstantFunctionId("ceil") - case object ClampMax extends InstantFunctionId("clamp_max") - case object ClampMin extends InstantFunctionId("clamp_min") - case object Exp extends InstantFunctionId("exp") - case object Floor extends InstantFunctionId("floor") - case object HistogramQuantile extends InstantFunctionId("histogram_quantile") - case object HistogramMaxQuantile extends InstantFunctionId("histogram_max_quantile") - case object HistogramBucket extends InstantFunctionId("histogram_bucket") - case object Ln extends InstantFunctionId("ln") - case object Log10 extends InstantFunctionId("log10") - case object Log2 extends InstantFunctionId("log2") - case object Round extends InstantFunctionId("round") - case object Sqrt extends InstantFunctionId("sqrt") - case object DaysInMonth extends InstantFunctionId("days_in_month") - case object DayOfMonth extends InstantFunctionId("day_of_month") - case object DayOfWeek extends InstantFunctionId("day_of_week") - case object Hour extends InstantFunctionId("hour") - case object Minute extends InstantFunctionId("minute") - case object Month extends InstantFunctionId("month") - case object Year extends InstantFunctionId("year") - // TODO time, vector, scalar } -sealed abstract class RangeFunctionId(override val entryName: String) extends EnumEntry - -object RangeFunctionId extends Enum[RangeFunctionId] { - val values = findValues - - case object AvgOverTime extends RangeFunctionId("avg_over_time") - - case object Changes extends RangeFunctionId("changes") - - case object CountOverTime extends RangeFunctionId("count_over_time") - - case object Delta extends RangeFunctionId("delta") - - case object Deriv extends RangeFunctionId("deriv") - - case object HoltWinters extends RangeFunctionId("holt_winters") - - case object ZScore extends RangeFunctionId("z_score") - - case object Idelta extends RangeFunctionId("idelta") - - case object Increase extends RangeFunctionId("increase") - - case object Irate extends RangeFunctionId("irate") - - case object MaxOverTime extends RangeFunctionId("max_over_time") - - case object MinOverTime extends RangeFunctionId("min_over_time") - case object PredictLinear extends RangeFunctionId("predict_linear") +/** + * Below trait is for defining the type of arguments expected by all promQL functions defined below. + * All the classes extending the ParamSpec trait will check if the input arg is of certain type or not. If not, + * the throw default error message if not overriden in the object. + */ - case object QuantileOverTime extends RangeFunctionId("quantile_over_time") +sealed trait ParamSpec +final case class RangeVectorParam(errorMsg: String = "Expected type range vector in call to function") extends ParamSpec +final case class InstantVectorParam(errorMsg: String = "Expected type instant vector in call to function") extends ParamSpec +final case class ScalarParam(errorMsg: String = "Expected type scalar in call to function") extends ParamSpec +final case class ScalarRangeParam(min: Double, max: Double, errorMsg: String) extends ParamSpec +sealed abstract class RangeFunctionId(override val entryName: String, val paramSpec: Seq[ParamSpec]) extends EnumEntry - case object Rate extends RangeFunctionId("rate") - - case object Resets extends RangeFunctionId("resets") - - case object StdDevOverTime extends RangeFunctionId("stddev_over_time") - - case object StdVarOverTime extends RangeFunctionId("stdvar_over_time") - - case object SumOverTime extends RangeFunctionId("sum_over_time") +object RangeFunctionId extends Enum[RangeFunctionId] { + val values = findValues - case object Timestamp extends RangeFunctionId("timestamp") + case object AvgOverTime extends RangeFunctionId("avg_over_time", Seq(RangeVectorParam())) + case object Changes extends RangeFunctionId("changes", Seq(RangeVectorParam())) + case object CountOverTime extends RangeFunctionId("count_over_time", Seq(RangeVectorParam())) + case object Delta extends RangeFunctionId("delta", Seq(RangeVectorParam())) + case object Deriv extends RangeFunctionId("deriv", Seq(RangeVectorParam())) + case object HoltWinters extends RangeFunctionId("holt_winters", Seq(RangeVectorParam(), + ScalarRangeParam(0, 1, "Invalid Smoothing factor. Expected: 0 < sf < 1, got:"), ScalarRangeParam(0, 1, "Invalid Trend factor. Expected: 0 < tf < 1, got:"))) + case object ZScore extends RangeFunctionId("z_score", Seq(RangeVectorParam())) + case object Idelta extends RangeFunctionId("idelta", Seq(RangeVectorParam())) + case object Increase extends RangeFunctionId("increase", Seq(RangeVectorParam())) + case object Irate extends RangeFunctionId("irate", Seq(RangeVectorParam())) + case object MaxOverTime extends RangeFunctionId("max_over_time", Seq(RangeVectorParam())) + case object MinOverTime extends RangeFunctionId("min_over_time", Seq(RangeVectorParam())) + case object PredictLinear extends RangeFunctionId("predict_linear", Seq(RangeVectorParam(), ScalarParam())) + case object QuantileOverTime extends RangeFunctionId("quantile_over_time", Seq(ScalarParam(), RangeVectorParam())) + case object Rate extends RangeFunctionId("rate", Seq(RangeVectorParam())) + case object Resets extends RangeFunctionId("resets", Seq(RangeVectorParam())) + case object StdDevOverTime extends RangeFunctionId("stddev_over_time", Seq(RangeVectorParam())) + case object StdVarOverTime extends RangeFunctionId("stdvar_over_time", Seq(RangeVectorParam())) + case object SumOverTime extends RangeFunctionId("sum_over_time", Seq(RangeVectorParam())) + case object Timestamp extends RangeFunctionId("timestamp", Seq(InstantVectorParam())) } sealed abstract class FiloFunctionId(override val entryName: String) extends EnumEntry @@ -113,27 +86,16 @@ object AggregationOperator extends Enum[AggregationOperator] { val values = findValues case object Avg extends AggregationOperator("avg") - case object Count extends AggregationOperator("count") - case object Sum extends AggregationOperator("sum") - case object Min extends AggregationOperator("min") - case object Max extends AggregationOperator("max") - case object Stddev extends AggregationOperator("stddev") - case object Stdvar extends AggregationOperator("stdvar") - case object TopK extends AggregationOperator("topk") - case object BottomK extends AggregationOperator("bottomk") - case object CountValues extends AggregationOperator("count_values") - case object Quantile extends AggregationOperator("quantile") - } sealed abstract class BinaryOperator extends EnumEntry { @@ -142,53 +104,32 @@ sealed abstract class BinaryOperator extends EnumEntry { } sealed class MathOperator (val precedence: Int = 0, val isRightAssociative: Boolean = false) extends BinaryOperator - sealed class SetOperator(val precedence: Int = 0, val isRightAssociative: Boolean = false) extends BinaryOperator - sealed class ComparisonOperator(val precedence: Int = 0, val isRightAssociative: Boolean = false) extends BinaryOperator object BinaryOperator extends Enum[BinaryOperator] { val values = findValues - case object SUB extends MathOperator(4) + case object SUB extends MathOperator(4) case object ADD extends MathOperator(4) - case object MUL extends MathOperator(5) - case object MOD extends MathOperator(5) - case object DIV extends MathOperator(5) - case object POW extends MathOperator(6, true) - case object LAND extends SetOperator(2) - case object LOR extends SetOperator(1) - case object LUnless extends SetOperator(2) - case object EQL extends ComparisonOperator(3) - case object NEQ extends ComparisonOperator(3) - case object LTE extends ComparisonOperator(3) - case object LSS extends ComparisonOperator(3) - case object GTE extends ComparisonOperator(3) - case object GTR extends ComparisonOperator(3) - case object EQL_BOOL extends ComparisonOperator(3) - case object NEQ_BOOL extends ComparisonOperator(3) - case object LTE_BOOL extends ComparisonOperator(3) - case object LSS_BOOL extends ComparisonOperator(3) - case object GTE_BOOL extends ComparisonOperator(3) - case object GTR_BOOL extends ComparisonOperator(3) case object EQLRegex extends BinaryOperator { // FIXME when implemented @@ -208,13 +149,9 @@ object Cardinality extends Enum[Cardinality] { val values = findValues case object OneToOne extends Cardinality - case object OneToMany extends Cardinality - case object ManyToOne extends Cardinality - case object ManyToMany extends Cardinality - } sealed abstract class MiscellaneousFunctionId(override val entryName: String) extends EnumEntry @@ -223,9 +160,7 @@ object MiscellaneousFunctionId extends Enum[MiscellaneousFunctionId] { val values = findValues case object LabelReplace extends MiscellaneousFunctionId("label_replace") - case object LabelJoin extends MiscellaneousFunctionId("label_join") - case object HistToPromVectors extends MiscellaneousFunctionId("hist_to_prom_vectors") } @@ -233,8 +168,8 @@ sealed abstract class SortFunctionId(override val entryName: String) extends Enu object SortFunctionId extends Enum[SortFunctionId] { val values = findValues - case object Sort extends SortFunctionId("sort") + case object Sort extends SortFunctionId("sort") case object SortDesc extends SortFunctionId("sort_desc") } @@ -244,29 +179,21 @@ object ScalarFunctionId extends Enum[ScalarFunctionId] { val values = findValues case object Scalar extends ScalarFunctionId("scalar") - case object Time extends ScalarFunctionId("time") - case object DaysInMonth extends ScalarFunctionId("days_in_month") - case object DayOfMonth extends ScalarFunctionId("day_of_month") - case object DayOfWeek extends ScalarFunctionId("day_of_week") - case object Hour extends ScalarFunctionId("hour") - case object Minute extends ScalarFunctionId("minute") - case object Month extends ScalarFunctionId("month") - case object Year extends ScalarFunctionId("year") - } sealed abstract class VectorFunctionId(override val entryName: String) extends EnumEntry object VectorFunctionId extends Enum[VectorFunctionId] { val values = findValues + case object Vector extends VectorFunctionId("vector") } @@ -274,6 +201,7 @@ sealed abstract class AbsentFunctionId(override val entryName: String) extends E object AbsentFunctionId extends Enum[AbsentFunctionId] { val values = findValues + case object Absent extends AbsentFunctionId("absent") } diff --git a/query/src/main/scala/filodb/query/PromQlInvocationParams.scala b/query/src/main/scala/filodb/query/PromQlInvocationParams.scala deleted file mode 100644 index 156edfa682..0000000000 --- a/query/src/main/scala/filodb/query/PromQlInvocationParams.scala +++ /dev/null @@ -1,7 +0,0 @@ -package filodb.query - -import com.typesafe.config.Config - -case class PromQlInvocationParams(config: Config, promQl: String, startSecs: Long, stepSecs: Long, endSecs: Long, - spread: Option[Int] = None, processFailure: Boolean = true) - diff --git a/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala b/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala index 859713f54a..496a0e2d4e 100644 --- a/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala +++ b/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala @@ -14,7 +14,7 @@ import filodb.query.exec.aggregator.RowAggregator * Reduce combined aggregates from children. Can be applied in a * hierarchical manner multiple times to arrive at result. */ -final case class ReduceAggregateExec(id: String, +final case class ReduceAggregateExec(queryContext: QueryContext, dispatcher: PlanDispatcher, childAggregates: Seq[ExecPlan], aggrOp: AggregationOperator, @@ -27,7 +27,7 @@ final case class ReduceAggregateExec(id: String, firstSchema: Task[ResultSchema], queryConfig: QueryConfig): Observable[RangeVector] = { val results = childResponses.flatMap { - case (QueryResult(_, schema, result), _) => Observable.fromIterable(result) + case (QueryResult(_, _, result), _) => Observable.fromIterable(result) case (QueryError(_, ex), _) => throw ex } val task = for { schema <- firstSchema } @@ -133,7 +133,7 @@ object RangeVectorAggregator extends StrictLogging { val groupedResult = mapReduceInternal(rvs, rowAgg, skipMapPhase, grouping) groupedResult.map { case (rvk, aggHolder) => val rowIterator = aggHolder.map(_.toRowReader) - new IteratorBackedRangeVector(rvk, rowIterator) + IteratorBackedRangeVector(rvk, rowIterator) } } Observable.fromTask(task).flatMap(rvs => Observable.fromIterable(rvs)) @@ -212,7 +212,7 @@ object RangeVectorAggregator extends StrictLogging { aggObs.flatMap { _ => if (count > 0) { - Observable.now(new IteratorBackedRangeVector(CustomRangeVectorKey.empty, accs.toIterator.map(_.toRowReader))) + Observable.now(IteratorBackedRangeVector(CustomRangeVectorKey.empty, accs.toIterator.map(_.toRowReader))) } else { Observable.empty } diff --git a/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala b/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala index 9c8ad44166..b63e76240b 100644 --- a/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala +++ b/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala @@ -5,6 +5,7 @@ import scala.collection.mutable import monix.eval.Task import monix.reactive.Observable +import filodb.core.memstore.SchemaMismatch import filodb.core.query._ import filodb.memory.format.{RowReader, ZeroCopyUTF8String => Utf8Str} import filodb.memory.format.ZeroCopyUTF8String._ @@ -33,7 +34,7 @@ import filodb.query.exec.binaryOp.BinaryOperatorFunction * @param ignoring fields from range vector keys to exclude while performing the join * @param include labels specified in group_left/group_right to be included from one side */ -final case class BinaryJoinExec(id: String, +final case class BinaryJoinExec(queryContext: QueryContext, dispatcher: PlanDispatcher, lhs: Seq[ExecPlan], rhs: Seq[ExecPlan], @@ -154,5 +155,19 @@ final case class BinaryJoinExec(id: String, } } } + + /** + * overridden to allow schemas with different vector lengths, colids as long as the columns are same - to handle + * binary joins between scalar/rangevectors + */ + override def reduceSchemas(rs: ResultSchema, resp: QueryResult): ResultSchema = { + resp match { + case QueryResult(_, schema, _) if rs == ResultSchema.empty => + schema /// First schema, take as is + case QueryResult(_, schema, _) => + if (!rs.hasSameColumnsAs(schema)) throw SchemaMismatch(rs.toString, schema.toString) + else rs + } + } } diff --git a/query/src/main/scala/filodb/query/exec/DistConcatExec.scala b/query/src/main/scala/filodb/query/exec/DistConcatExec.scala index fbb534213b..cab9ebd719 100644 --- a/query/src/main/scala/filodb/query/exec/DistConcatExec.scala +++ b/query/src/main/scala/filodb/query/exec/DistConcatExec.scala @@ -5,12 +5,11 @@ import monix.reactive.Observable import filodb.core.query._ import filodb.query._ -import filodb.query.Query.qLogger /** * Simply concatenate results from child ExecPlan objects */ -final case class DistConcatExec(id: String, +final case class DistConcatExec(queryContext: QueryContext, dispatcher: PlanDispatcher, children: Seq[ExecPlan]) extends NonLeafExecPlan { require(children.nonEmpty) @@ -20,7 +19,6 @@ final case class DistConcatExec(id: String, protected def compose(childResponses: Observable[(QueryResponse, Int)], firstSchema: Task[ResultSchema], queryConfig: QueryConfig): Observable[RangeVector] = { - qLogger.debug(s"DistConcatExec: Concatenating results") 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 new file mode 100644 index 0000000000..44f91e8863 --- /dev/null +++ b/query/src/main/scala/filodb/query/exec/EmptyResultExec.scala @@ -0,0 +1,29 @@ +package filodb.query.exec + +import monix.eval.Task +import monix.execution.Scheduler + +import filodb.core.DatasetRef +import filodb.core.metadata.Column.ColumnType +import filodb.core.query.{ColumnInfo, QueryContext, ResultSchema} +import filodb.core.store.ChunkSource +import filodb.query.{QueryConfig, QueryResponse, QueryResult} + +case class EmptyResultExec(queryContext: QueryContext, + dataset: DatasetRef) extends LeafExecPlan { + override def dispatcher: PlanDispatcher = InProcessPlanDispatcher + + override def execute(source: ChunkSource, + queryConfig: QueryConfig) + (implicit sched: Scheduler): Task[QueryResponse] = { + Task(QueryResult(queryContext.queryId, + new ResultSchema(Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), + ColumnInfo("value", ColumnType.DoubleColumn)), 1), + Seq.empty)) + } + + override def doExecute(source: ChunkSource, queryConfig: QueryConfig) + (implicit sched: Scheduler): ExecResult = ??? + + override protected def args: String = "" +} \ No newline at end of file diff --git a/query/src/main/scala/filodb/query/exec/ExecPlan.scala b/query/src/main/scala/filodb/query/exec/ExecPlan.scala index 297e1b20d0..2cbf1d7f85 100644 --- a/query/src/main/scala/filodb/query/exec/ExecPlan.scala +++ b/query/src/main/scala/filodb/query/exec/ExecPlan.scala @@ -1,14 +1,12 @@ package filodb.query.exec import scala.collection.mutable.ArrayBuffer -import scala.concurrent.duration.FiniteDuration import scala.util.control.NonFatal import kamon.Kamon import monix.eval.Task import monix.execution.Scheduler import monix.reactive.Observable -import monix.reactive.observables.ConnectableObservable import filodb.core.DatasetRef import filodb.core.memstore.{FiloSchedulers, SchemaMismatch} @@ -41,28 +39,22 @@ final case class ExecResult(rvs: Observable[RangeVector], schema: Task[ResultSch */ trait ExecPlan extends QueryCommand { /** - * The query id + * Query Processing parameters */ - def id: String - - /** - * Child execution plans representing sub-queries - */ - def children: Seq[ExecPlan] - - def submitTime: Long - - /** - * Limit on number of samples returned by this ExecPlan - */ - def limit: Int + def queryContext: QueryContext /** * Throw error if the size of the resultset is greater than Limit * Take first n (limit) elements if the flag is false. Applicable for Metadata Queries + * It is not in QueryContext since for some queries it should be false */ def enforceLimit: Boolean = true + /** + * Child execution plans representing sub-queries + */ + def children: Seq[ExecPlan] + def dataset: DatasetRef /** @@ -101,7 +93,7 @@ trait ExecPlan extends QueryCommand { */ // scalastyle:off method.length def execute(source: ChunkSource, queryConfig: QueryConfig) - (implicit sched: Scheduler, timeout: FiniteDuration): Task[QueryResponse] = { + (implicit sched: Scheduler): Task[QueryResponse] = { val parentSpan = Kamon.currentSpan() // NOTE: we launch the preparatory steps as a Task too. This is important because scanPartitions, @@ -111,10 +103,9 @@ trait ExecPlan extends QueryCommand { lazy val step1 = Task { val span = Kamon.spanBuilder(s"execute-step1-${getClass.getSimpleName}") .asChildOf(parentSpan) - .tag("query-id", id) + .tag("query-id", queryContext.queryId) .start() FiloSchedulers.assertThreadName(QuerySchedName) - qLogger.debug(s"queryId: ${id} Setting up ExecPlan ${getClass.getSimpleName} with $args") // Please note that the following needs to be wrapped inside `runWithSpan` so that the context will be propagated // across threads. Note that task/observable will not run on the thread where span is present since // kamon uses thread-locals. @@ -127,22 +118,21 @@ trait ExecPlan extends QueryCommand { def step2(res: ExecResult) = res.schema.map { resSchema => val span = Kamon.spanBuilder(s"execute-step2-${getClass.getSimpleName}") .asChildOf(parentSpan) - .tag("query-id", id) + .tag("query-id", queryContext.queryId) .start() FiloSchedulers.assertThreadName(QuerySchedName) val dontRunTransformers = if (allTransformers.isEmpty) true else !allTransformers.forall(_.canHandleEmptySchemas) span.tag("dontRunTransformers", dontRunTransformers) // It is possible a null schema is returned (due to no time series). In that case just return empty results val resultTask = if (resSchema == ResultSchema.empty && dontRunTransformers) { - qLogger.debug(s"Empty plan $this, returning empty results") + qLogger.debug(s"queryId: ${queryContext.queryId} Empty plan $this, returning empty results") span.mark("empty-plan") - Task.eval(QueryResult(id, resSchema, Nil)) + Task.eval(QueryResult(queryContext.queryId, resSchema, Nil)) } else { val finalRes = allTransformers.foldLeft((res.rvs, resSchema)) { (acc, transf) => - qLogger.debug(s"queryId: ${id} Setting up Transformer ${transf.getClass.getSimpleName} with ${transf.args}") span.mark(transf.getClass.getSimpleName) val paramRangeVector: Seq[Observable[ScalarRangeVector]] = transf.funcParams.map(_.getResult) - (transf.apply(acc._1, queryConfig, limit, acc._2, paramRangeVector), transf.schema(acc._2)) + (transf.apply(acc._1, queryConfig, queryContext.sampleLimit, acc._2, paramRangeVector), transf.schema(acc._2)) } val recSchema = SerializedRangeVector.toSchema(finalRes._2.columns, finalRes._2.brSchemas) val builder = SerializedRangeVector.newBuilder() @@ -152,8 +142,8 @@ trait ExecPlan extends QueryCommand { case srv: SerializableRangeVector => numResultSamples += srv.numRowsInt // fail the query instead of limiting range vectors and returning incomplete/inaccurate results - if (enforceLimit && numResultSamples > limit) - throw new BadQueryException(s"This query results in more than $limit samples. " + + if (enforceLimit && numResultSamples > queryContext.sampleLimit) + throw new BadQueryException(s"This query results in more than ${queryContext.sampleLimit} samples. " + s"Try applying more filters or reduce time range.") srv case rv: RangeVector => @@ -161,12 +151,12 @@ trait ExecPlan extends QueryCommand { val srv = SerializedRangeVector(rv, builder, recSchema, printTree(false)) numResultSamples += srv.numRowsInt // fail the query instead of limiting range vectors and returning incomplete/inaccurate results - if (enforceLimit && numResultSamples > limit) - throw new BadQueryException(s"This query results in more than $limit samples. " + + if (enforceLimit && numResultSamples > queryContext.sampleLimit) + throw new BadQueryException(s"This query results in more than $queryContext.sampleLimit samples. " + s"Try applying more filters or reduce time range.") srv } - .take(limit) + .take(queryContext.sampleLimit) .toListL .map { r => val numBytes = builder.allContainers.map(_.numBytes).sum @@ -176,29 +166,28 @@ trait ExecPlan extends QueryCommand { // 5MB limit. Configure if necessary later. // 250 RVs * (250 bytes for RV-Key + 200 samples * 32 bytes per sample) // is < 2MB - qLogger.warn(s"queryId: ${id} result was " + - s"large size ${numBytes}. May need to tweak limits. " + - s"ExecPlan was: ${printTree()} " + - s"Limit was: ${limit}") + qLogger.warn(s"queryId: ${queryContext.queryId} result was large size $numBytes. May need to " + + s"tweak limits. ExecPlan was: ${printTree()} ; Limit was: ${queryContext.sampleLimit}") } - qLogger.debug(s"queryId: ${id} Successful execution of ${getClass.getSimpleName} with transformers") span.mark(s"num-result-samples: $numResultSamples") span.mark(s"num-range-vectors: ${r.size}") span.finish() - QueryResult(id, finalRes._2, r) + QueryResult(queryContext.queryId, finalRes._2, r) } } resultTask.onErrorHandle { case ex: Throwable => if (!ex.isInstanceOf[BadQueryException]) // dont log user errors - qLogger.error(s"queryId: ${id} Exception during execution of query: ${printTree(false)}", ex) + qLogger.error(s"queryId: ${queryContext.queryId} Exception during execution of query: " + + s"${printTree(false)}", ex) span.fail(ex) - QueryError(id, ex) + QueryError(queryContext.queryId, ex) } }.flatten .onErrorRecover { case NonFatal(ex) => if (!ex.isInstanceOf[BadQueryException]) // dont log user errors - qLogger.error(s"queryId: ${id} Exception during orchestration of query: ${printTree(false)}", ex) - QueryError(id, ex) + qLogger.error(s"queryId: ${queryContext.queryId} Exception during orchestration of query:" + + s" ${printTree(false)}", ex) + QueryError(queryContext.queryId, ex) } for { res <- step1 @@ -216,8 +205,7 @@ trait ExecPlan extends QueryCommand { */ def doExecute(source: ChunkSource, queryConfig: QueryConfig) - (implicit sched: Scheduler, - timeout: FiniteDuration): ExecResult + (implicit sched: Scheduler): ExecResult /** * Args to use for the ExecPlan for printTree purposes only. @@ -247,7 +235,7 @@ trait ExecPlan extends QueryCommand { val nextLevel = rangeVectorTransformers.size + level val curNode = s"${"-"*nextLevel}E~${getClass.getSimpleName}($args) on ${dispatcher}" val childr : Seq[String]= children.flatMap(_.getPlan(nextLevel + 1)) - ((transf :+ curNode) ++ childr) + (transf :+ curNode) ++ childr } protected def printRangeVectorTransformersForLevel(level: Int = 0) = { @@ -288,6 +276,7 @@ trait ExecPlan extends QueryCommand { abstract class LeafExecPlan extends ExecPlan { final def children: Seq[ExecPlan] = Nil + final def submitTime: Long = queryContext.submitTime } /** @@ -295,7 +284,7 @@ abstract class LeafExecPlan extends ExecPlan { * getResult will get the ScalarRangeVector for the FuncArg */ sealed trait FuncArgs { - def getResult (implicit sched: Scheduler, timeout: FiniteDuration) : Observable[ScalarRangeVector] + def getResult (implicit sched: Scheduler) : Observable[ScalarRangeVector] } /** @@ -303,11 +292,12 @@ sealed trait FuncArgs { */ final case class ExecPlanFuncArgs(execPlan: ExecPlan, timeStepParams: RangeParams) extends FuncArgs { - override def getResult(implicit sched: Scheduler, timeout: FiniteDuration): Observable[ScalarRangeVector] = { + override def getResult(implicit sched: Scheduler): Observable[ScalarRangeVector] = { Observable.fromTask( execPlan.dispatcher.dispatch(execPlan).onErrorHandle { case ex: Throwable => - qLogger.error(s"queryId: ${execPlan.id} Execution failed for sub-query ${execPlan.printTree()}", ex) - QueryError(execPlan.id, ex) + qLogger.error(s"queryId: ${execPlan.queryContext.queryId} Execution failed for sub-query" + + s" ${execPlan.printTree()}", ex) + QueryError(execPlan.queryContext.queryId, ex) }.map { case QueryResult(_, _, result) => // Result is empty because of NaN so create ScalarFixedDouble with NaN if (result.isEmpty) { @@ -329,9 +319,8 @@ final case class ExecPlanFuncArgs(execPlan: ExecPlan, timeStepParams: RangeParam * FuncArgs for scalar parameter */ final case class StaticFuncArgs(scalar: Double, timeStepParams: RangeParams) extends FuncArgs { - override def getResult(implicit sched: Scheduler, timeout: FiniteDuration): Observable[ScalarRangeVector] = { - Observable.now( - new ScalarFixedDouble(timeStepParams, scalar)) + override def getResult(implicit sched: Scheduler): Observable[ScalarRangeVector] = { + Observable.now(ScalarFixedDouble(timeStepParams, scalar)) } } @@ -339,9 +328,8 @@ final case class StaticFuncArgs(scalar: Double, timeStepParams: RangeParams) ext * FuncArgs for date and time functions */ final case class TimeFuncArgs(timeStepParams: RangeParams) extends FuncArgs { - override def getResult(implicit sched: Scheduler, timeout: FiniteDuration): Observable[ScalarRangeVector] = { - Observable.now( - new TimeScalar(timeStepParams)) + override def getResult(implicit sched: Scheduler): Observable[ScalarRangeVector] = { + Observable.now(TimeScalar(timeStepParams)) } } @@ -352,21 +340,17 @@ abstract class NonLeafExecPlan extends ExecPlan { */ final def dataset: DatasetRef = children.head.dataset - final def submitTime: Long = children.head.submitTime - - final def limit: Int = children.head.limit - - private var multicast: ConnectableObservable[(QueryResponse, Int)] = _ + final def submitTime: Long = children.head.queryContext.submitTime private def dispatchRemotePlan(plan: ExecPlan, span: kamon.trace.Span) - (implicit sched: Scheduler, timeout: FiniteDuration) = { + (implicit sched: Scheduler) = { // Please note that the following needs to be wrapped inside `runWithSpan` so that the context will be propagated // across threads. Note that task/observable will not run on the thread where span is present since // kamon uses thread-locals. Kamon.runWithSpan(span, false) { plan.dispatcher.dispatch(plan).onErrorHandle { case ex: Throwable => - qLogger.error(s"queryId: ${id} Execution failed for sub-query ${plan.printTree()}", ex) - QueryError(id, ex) + qLogger.error(s"queryId: ${queryContext.queryId} Execution failed for sub-query ${plan.printTree()}", ex) + QueryError(queryContext.queryId, ex) } } } @@ -380,8 +364,7 @@ abstract class NonLeafExecPlan extends ExecPlan { */ final def doExecute(source: ChunkSource, queryConfig: QueryConfig) - (implicit sched: Scheduler, - timeout: FiniteDuration): ExecResult = { + (implicit sched: Scheduler): ExecResult = { val parentSpan = Kamon.currentSpan() parentSpan.mark("create-child-tasks") // Create tasks for all results. @@ -398,8 +381,8 @@ abstract class NonLeafExecPlan extends ExecPlan { case (res @ QueryResult(_, schema, _), i) if schema != ResultSchema.empty => sch = reduceSchemas(sch, res) (res, i.toInt) - case (e: QueryError, i) => - (e, i.toInt) + case (e: QueryError, _) => + throw e.t // cache caches results so that multiple subscribers can process }.cache diff --git a/query/src/main/scala/filodb/query/exec/InProcessPlanDispatcher.scala b/query/src/main/scala/filodb/query/exec/InProcessPlanDispatcher.scala index 1a028fe4a6..5443360877 100644 --- a/query/src/main/scala/filodb/query/exec/InProcessPlanDispatcher.scala +++ b/query/src/main/scala/filodb/query/exec/InProcessPlanDispatcher.scala @@ -1,7 +1,5 @@ package filodb.query.exec -import scala.concurrent.duration.FiniteDuration - import kamon.Kamon import monix.eval.Task import monix.execution.Scheduler @@ -23,8 +21,7 @@ case object InProcessPlanDispatcher extends PlanDispatcher { // Empty query config, since its does not apply in case of non-leaf plans val queryConfig: QueryConfig = EmptyQueryConfig - override def dispatch(plan: ExecPlan)(implicit sched: Scheduler, - timeout: FiniteDuration): Task[QueryResponse] = { + override def dispatch(plan: ExecPlan)(implicit sched: Scheduler): Task[QueryResponse] = { // unsupported source since its does not apply in case of non-leaf plans val source = UnsupportedChunkSource() diff --git a/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala b/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala index 593fc3ee85..0b61d8e2d1 100644 --- a/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala +++ b/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala @@ -1,7 +1,5 @@ package filodb.query.exec -import scala.concurrent.duration.FiniteDuration - import kamon.Kamon import monix.eval.Task import monix.execution.Scheduler @@ -19,9 +17,9 @@ import filodb.memory.format.ZeroCopyUTF8String._ import filodb.query._ import filodb.query.Query.qLogger -final case class PartKeysDistConcatExec(id: String, - dispatcher: PlanDispatcher, - children: Seq[ExecPlan]) extends NonLeafExecPlan { +final case class PartKeysDistConcatExec(queryContext: QueryContext, + dispatcher: PlanDispatcher, + children: Seq[ExecPlan]) extends NonLeafExecPlan { require(!children.isEmpty) @@ -51,9 +49,9 @@ final case class PartKeysDistConcatExec(id: String, } -final case class LabelValuesDistConcatExec(id: String, - dispatcher: PlanDispatcher, - children: Seq[ExecPlan]) extends NonLeafExecPlan { +final case class LabelValuesDistConcatExec(queryContext: QueryContext, + dispatcher: PlanDispatcher, + children: Seq[ExecPlan]) extends NonLeafExecPlan { require(!children.isEmpty) @@ -97,9 +95,7 @@ final case class LabelValuesDistConcatExec(id: String, } -final case class PartKeysExec(id: String, - submitTime: Long, - limit: Int, +final case class PartKeysExec(queryContext: QueryContext, dispatcher: PlanDispatcher, dataset: DatasetRef, shard: Int, @@ -112,11 +108,10 @@ final case class PartKeysExec(id: String, def doExecute(source: ChunkSource, queryConfig: QueryConfig) - (implicit sched: Scheduler, - timeout: FiniteDuration): ExecResult = { + (implicit sched: Scheduler): ExecResult = { val rvs = source match { case memStore: MemStore => - val response = memStore.partKeysWithFilters(dataset, shard, filters, end, start, limit) + val response = memStore.partKeysWithFilters(dataset, shard, filters, end, start, queryContext.sampleLimit) Observable.now(IteratorBackedRangeVector(new CustomRangeVectorKey(Map.empty), new PartKeyRowReader(response))) case other => Observable.empty @@ -127,12 +122,10 @@ final case class PartKeysExec(id: String, ExecResult(rvs, Task.eval(sch)) } - def args: String = s"shard=$shard, filters=$filters, limit=$limit" + def args: String = s"shard=$shard, filters=$filters, limit=${queryContext.sampleLimit}" } -final case class LabelValuesExec(id: String, - submitTime: Long, - limit: Int, +final case class LabelValuesExec(queryContext: QueryContext, dispatcher: PlanDispatcher, dataset: DatasetRef, shard: Int, @@ -144,8 +137,7 @@ final case class LabelValuesExec(id: String, def doExecute(source: ChunkSource, queryConfig: QueryConfig) - (implicit sched: Scheduler, - timeout: FiniteDuration): ExecResult = { + (implicit sched: Scheduler): ExecResult = { val parentSpan = Kamon.currentSpan() val rvs = if (source.isInstanceOf[MemStore]) { val memStore = source.asInstanceOf[MemStore] @@ -154,12 +146,13 @@ final case class LabelValuesExec(id: String, val start = end - lookBackInMillis val response = filters.isEmpty match { // retrieves label values for a single label - no column filter - case true if (columns.size == 1) => memStore.labelValues(dataset, shard, columns.head, limit) + case true if (columns.size == 1) => memStore.labelValues(dataset, shard, columns.head, queryContext.sampleLimit) .map(termInfo => Map(columns.head.utf8 -> termInfo.term)) .toIterator case true => throw new BadQueryException("either label name is missing " + "or there are multiple label names without filter") - case false => memStore.labelValuesWithFilters(dataset, shard, filters, columns, end, start, limit) + case false => memStore.labelValuesWithFilters(dataset, shard, filters, columns, end, start, + queryContext.sampleLimit) } Observable.now(IteratorBackedRangeVector(new CustomRangeVectorKey(Map.empty), new UTF8MapIteratorRowReader(response))) @@ -171,5 +164,6 @@ final case class LabelValuesExec(id: String, ExecResult(rvs, Task.eval(sch)) } - def args: String = s"shard=$shard, filters=$filters, col=$columns, limit=$limit, lookBackInMillis=$lookBackInMillis" + def args: String = s"shard=$shard, filters=$filters, col=$columns, limit=${queryContext.sampleLimit}, " + + s"lookBackInMillis=$lookBackInMillis" } diff --git a/query/src/main/scala/filodb/query/exec/MultiSchemaPartitionsExec.scala b/query/src/main/scala/filodb/query/exec/MultiSchemaPartitionsExec.scala index 9d43b16be3..0af44b02bf 100644 --- a/query/src/main/scala/filodb/query/exec/MultiSchemaPartitionsExec.scala +++ b/query/src/main/scala/filodb/query/exec/MultiSchemaPartitionsExec.scala @@ -1,13 +1,11 @@ package filodb.query.exec -import scala.concurrent.duration.FiniteDuration - import kamon.Kamon import monix.execution.Scheduler -import filodb.core.DatasetRef +import filodb.core.{DatasetRef, QueryTimeoutException} import filodb.core.metadata.Schemas -import filodb.core.query.ColumnFilter +import filodb.core.query.{ColumnFilter, QueryContext} import filodb.core.store._ import filodb.query.Query.qLogger import filodb.query.QueryConfig @@ -27,9 +25,7 @@ final case class UnknownSchemaQueryErr(id: Int) extends * @param colName optional column name to select for querying. If not supplied, the default valueColumn from * data schema definition is used. For downsampled gauges, column is automatically chosen. */ -final case class MultiSchemaPartitionsExec(id: String, - submitTime: Long, - limit: Int, +final case class MultiSchemaPartitionsExec(queryContext: QueryContext, dispatcher: PlanDispatcher, dataset: DatasetRef, shard: Int, @@ -49,6 +45,10 @@ final case class MultiSchemaPartitionsExec(id: String, val lookupRes = source.lookupPartitions(dataset, partMethod, chunkMethod) Kamon.currentSpan().mark("lookup-partitions-done") + val queryTimeElapsed = System.currentTimeMillis() - queryContext.submitTime + if (queryTimeElapsed >= queryContext.queryTimeoutMillis) + throw QueryTimeoutException(queryTimeElapsed, this.getClass.getName) + // Find the schema if one wasn't supplied val schemas = source.schemas(dataset).get // If we cannot find a schema, or none is provided, we cannot move ahead with specific SRPE planning @@ -68,7 +68,7 @@ final case class MultiSchemaPartitionsExec(id: String, val colIDs1 = getColumnIDs(sch, colName.toSeq, rangeVectorTransformers) val colIDs = addIDsForHistMax(sch, colIDs1) - val newPlan = SelectRawPartitionsExec(id, submitTime, limit, dispatcher, dataset, + val newPlan = SelectRawPartitionsExec(queryContext, dispatcher, dataset, Some(sch), Some(lookupRes), schema.isDefined, colIDs) qLogger.debug(s"Discovered schema ${sch.name} and created inner plan $newPlan") @@ -76,7 +76,7 @@ final case class MultiSchemaPartitionsExec(id: String, newPlan }.getOrElse { qLogger.debug(s"No time series found for filters $filters... employing empty plan") - SelectRawPartitionsExec(id, submitTime, limit, dispatcher, dataset, + SelectRawPartitionsExec(queryContext, dispatcher, dataset, None, Some(lookupRes), schema.isDefined, Nil) } @@ -84,10 +84,9 @@ final case class MultiSchemaPartitionsExec(id: String, def doExecute(source: ChunkSource, queryConfig: QueryConfig) - (implicit sched: Scheduler, - timeout: FiniteDuration): ExecResult = { + (implicit sched: Scheduler): ExecResult = { finalPlan = finalizePlan(source) - finalPlan.doExecute(source, queryConfig)(sched, timeout) + finalPlan.doExecute(source, queryConfig)(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 f0c8822c3a..fb541eec99 100644 --- a/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala +++ b/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala @@ -5,13 +5,13 @@ import monix.reactive.Observable import org.jctools.queues.SpscUnboundedArrayQueue import filodb.core.metadata.Column.ColumnType -import filodb.core.metadata.Schemas import filodb.core.query._ import filodb.core.store.WindowedChunkIterator import filodb.memory.format._ import filodb.memory.format.vectors.LongBinaryVector import filodb.query._ import filodb.query.Query.qLogger +import filodb.query.exec.InternalRangeFunction.AvgWithSumAndCountOverTime import filodb.query.exec.rangefn._ import filodb.query.util.IndexedArrayQueue @@ -26,6 +26,7 @@ final case class PeriodicSamplesMapper(start: Long, end: Long, window: Option[Long], functionId: Option[InternalRangeFunction], + queryContext: QueryContext, funcParams: Seq[FuncArgs] = Nil, offset: Option[Long] = None, rawSource: Boolean = true) extends RangeVectorTransformer { @@ -71,14 +72,14 @@ final case class PeriodicSamplesMapper(start: Long, val histRow = if (hasMaxCol) new TransientHistMaxRow() else new TransientHistRow() IteratorBackedRangeVector(rv.key, new ChunkedWindowIteratorH(rv.asInstanceOf[RawDataRangeVector], start, step, end, - windowLength, rangeFuncGen().asChunkedH, queryConfig, histRow)) + windowLength, rangeFuncGen().asChunkedH, queryConfig, queryContext, 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], start, step, end, - windowLength, rangeFuncGen().asChunkedD, queryConfig)) + windowLength, rangeFuncGen().asChunkedD, queryConfig, queryContext)) } // Iterator-based: Wrap long columns to yield a double value case f: RangeFunction if valColType == ColumnType.LongColumn => @@ -116,10 +117,14 @@ final case class PeriodicSamplesMapper(start: Long, // Transform source double or long to double schema override def schema(source: ResultSchema): ResultSchema = { - // Special treatment for downsampled gauge schema - return regular timestamp/value - if (source.columns == Schemas.dsGauge.dataInfos) { - source.copy(columns = Seq(ColumnInfo("timestamp", ColumnType.LongColumn), - ColumnInfo("value", ColumnType.DoubleColumn))) + // Special treatment for downsampled gauge schema. + // Source schema will be the selected columns. Result of this mapper should be regular timestamp/value + // since the avg will be calculated using sum and count + // FIXME dont like that this is hardcoded; but the check is needed. + if (functionId.contains(AvgWithSumAndCountOverTime) && + source.columns.map(_.name) == Seq("timestamp", "sum", "count")) { + source.copy(columns = Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), + ColumnInfo("value", ColumnType.DoubleColumn))) } else { source.copy(columns = source.columns.zipWithIndex.map { // Transform if its not a row key column @@ -149,12 +154,13 @@ abstract class ChunkedWindowIterator[R <: MutableRowReader]( end: Long, window: Long, rangeFunction: ChunkedRangeFunction[R], - queryConfig: QueryConfig) + queryConfig: QueryConfig, + queryContext: QueryContext) 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) + val it = new WindowedChunkIterator(rv, start, step, end, window, queryContext) // 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. @@ -208,17 +214,19 @@ class ChunkedWindowIteratorD(rv: RawDataRangeVector, start: Long, step: Long, end: Long, window: Long, rangeFunction: ChunkedRangeFunction[TransientRow], queryConfig: QueryConfig, + queryContext: QueryContext = QueryContext(), // put emitter here in constructor for faster access var sampleToEmit: TransientRow = new TransientRow()) extends -ChunkedWindowIterator[TransientRow](rv, start, step, end, window, rangeFunction, queryConfig) +ChunkedWindowIterator[TransientRow](rv, start, step, end, window, rangeFunction, queryConfig, queryContext) class ChunkedWindowIteratorH(rv: RawDataRangeVector, start: Long, step: Long, end: Long, window: Long, rangeFunction: ChunkedRangeFunction[TransientHistRow], queryConfig: QueryConfig, + queryContext: QueryContext = QueryContext(), // put emitter here in constructor for faster access var sampleToEmit: TransientHistRow = new TransientHistRow()) extends -ChunkedWindowIterator[TransientHistRow](rv, start, step, end, window, rangeFunction, queryConfig) +ChunkedWindowIterator[TransientHistRow](rv, start, step, end, window, rangeFunction, queryConfig, queryContext) class QueueBasedWindow(q: IndexedArrayQueue[TransientRow]) extends Window { def size: Int = q.size diff --git a/query/src/main/scala/filodb/query/exec/PlanDispatcher.scala b/query/src/main/scala/filodb/query/exec/PlanDispatcher.scala index 8ca1f87e08..263c17eaf7 100644 --- a/query/src/main/scala/filodb/query/exec/PlanDispatcher.scala +++ b/query/src/main/scala/filodb/query/exec/PlanDispatcher.scala @@ -1,5 +1,7 @@ package filodb.query.exec +import java.util.concurrent.TimeUnit + import scala.concurrent.duration.FiniteDuration import akka.actor.ActorRef @@ -8,6 +10,7 @@ import akka.util.Timeout import monix.eval.Task import monix.execution.Scheduler +import filodb.core.QueryTimeoutException import filodb.query.QueryResponse /** @@ -16,8 +19,7 @@ import filodb.query.QueryResponse */ trait PlanDispatcher extends java.io.Serializable { def dispatch(plan: ExecPlan) - (implicit sched: Scheduler, - timeout: FiniteDuration): Task[QueryResponse] + (implicit sched: Scheduler): Task[QueryResponse] } /** @@ -26,11 +28,14 @@ trait PlanDispatcher extends java.io.Serializable { */ case class ActorPlanDispatcher(target: ActorRef) extends PlanDispatcher { - def dispatch(plan: ExecPlan) - (implicit sched: Scheduler, - timeout: FiniteDuration): Task[QueryResponse] = { - implicit val _ = Timeout(timeout) - val fut = (target ? plan).map { + def dispatch(plan: ExecPlan)(implicit sched: Scheduler): Task[QueryResponse] = { + + 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") } diff --git a/query/src/main/scala/filodb/query/exec/PromQlExec.scala b/query/src/main/scala/filodb/query/exec/PromQlExec.scala index 9bbc4d4b0e..c42834abe0 100644 --- a/query/src/main/scala/filodb/query/exec/PromQlExec.scala +++ b/query/src/main/scala/filodb/query/exec/PromQlExec.scala @@ -1,5 +1,7 @@ package filodb.query.exec +import java.util.concurrent.TimeUnit + import com.softwaremill.sttp.asynchttpclient.future.AsyncHttpClientFutureBackend import com.softwaremill.sttp.circe._ import com.typesafe.scalalogging.StrictLogging @@ -18,12 +20,10 @@ import filodb.memory.format.RowReader import filodb.memory.format.ZeroCopyUTF8String._ import filodb.query._ -case class PromQlExec(id: String, +case class PromQlExec(queryContext: QueryContext, dispatcher: PlanDispatcher, dataset: DatasetRef, - params: PromQlInvocationParams, - submitTime: Long = System.currentTimeMillis()) - extends LeafExecPlan { + params: PromQlQueryParams) extends LeafExecPlan { protected def args: String = params.toString import PromQlExec._ @@ -38,24 +38,22 @@ case class PromQlExec(id: String, * node */ def doExecute(source: ChunkSource, queryConfig: QueryConfig) - (implicit sched: Scheduler, timeout: FiniteDuration): ExecResult = ??? + (implicit sched: Scheduler): ExecResult = ??? override def execute(source: ChunkSource, queryConfig: QueryConfig) - (implicit sched: Scheduler, - timeout: FiniteDuration): Task[QueryResponse] = { + (implicit sched: Scheduler): Task[QueryResponse] = { val execPlan2Span = Kamon.spanBuilder(s"execute-${getClass.getSimpleName}") .asChildOf(Kamon.currentSpan()) - .tag("query-id", id) + .tag("query-id", queryContext.queryId) .start() - val queryResponse = PromQlExec.httpGet(params).map { response => - + val queryResponse = PromQlExec.httpGet(params, queryContext.submitTime). + map { response => response.unsafeBody match { - case Left(error) => QueryError(id, error.error) - case Right(successResponse) => toQueryResponse(successResponse.data, id, execPlan2Span) + case Left(error) => QueryError(queryContext.queryId, error.error) + case Right(successResponse) => toQueryResponse(successResponse.data, queryContext.queryId, execPlan2Span) } - } // Please note that the following needs to be wrapped inside `runWithSpan` so that the context will be propagated // across threads. Note that task/observable will not run on the thread where span is present since @@ -100,13 +98,13 @@ case class PromQlExec(id: String, } -object PromQlExec extends StrictLogging{ +object PromQlExec extends StrictLogging { import com.softwaremill.sttp._ import io.circe.generic.auto._ import net.ceedubs.ficus.Ficus._ - val columns: Seq[ColumnInfo] = Seq(ColumnInfo("timestamp", ColumnType.LongColumn), + val columns: Seq[ColumnInfo] = Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), ColumnInfo("value", ColumnType.DoubleColumn)) val recSchema = SerializedRangeVector.toSchema(columns) val resultSchema = ResultSchema(columns, 1) @@ -118,17 +116,19 @@ object PromQlExec extends StrictLogging{ ShutdownHookThread(shutdown()) - def httpGet(params: PromQlInvocationParams)(implicit scheduler: Scheduler): + def httpGet(promQlQueryParams: PromQlQueryParams, submitTime: Long)(implicit scheduler: Scheduler): Future[Response[scala.Either[DeserializationError[io.circe.Error], SuccessResponse]]] = { - val endpoint = params.config.as[Option[String]]("buddy.http.endpoint").get - val readTimeout = params.config.as[Option[FiniteDuration]]("buddy.http.timeout").getOrElse(60.seconds) - var urlParams = Map("query" -> params.promQl, - "start" -> params.startSecs, - "end" -> params.endSecs, - "step" -> params.stepSecs, - "processFailure" -> params.processFailure) - if (params.spread.isDefined) - urlParams = urlParams + ("spread" -> params.spread.get) + val endpoint = promQlQueryParams.config.as[Option[String]]("buddy.http.endpoint").get + val queryTimeElapsed = System.currentTimeMillis() - submitTime + val buddyHttpTimeout = promQlQueryParams.config.as[Option[FiniteDuration]]("buddy.http.timeout"). + getOrElse(60000.millis) + val readTimeout = FiniteDuration(buddyHttpTimeout.toMillis - queryTimeElapsed, TimeUnit.MILLISECONDS) + var urlParams = Map("query" -> promQlQueryParams.promQl, + "start" -> promQlQueryParams.startSecs, + "end" -> promQlQueryParams.endSecs, + "step" -> promQlQueryParams.stepSecs, + "processFailure" -> promQlQueryParams.processFailure) + if (promQlQueryParams.spread.isDefined) urlParams = urlParams + ("spread" -> promQlQueryParams.spread.get) val url = uri"$endpoint?$urlParams" logger.debug("promqlexec url is {}", url) diff --git a/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala b/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala index 409a74c767..82e7c61cd9 100644 --- a/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala +++ b/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala @@ -67,8 +67,7 @@ object RangeVectorTransformer { */ final case class InstantVectorFunctionMapper(function: InstantFunctionId, funcParams: Seq[FuncArgs] = Nil) extends RangeVectorTransformer { - protected[exec] def args: String = - s"function=$function" + protected[exec] def args: String = s"function=$function" def evaluate(source: Observable[RangeVector], scalarRangeVector: Seq[ScalarRangeVector], queryConfig: QueryConfig, limit: Int, sourceSchema: ResultSchema) : Observable[RangeVector] = { @@ -194,8 +193,7 @@ private class HD2DoubleInstantFuncIterator(rows: Iterator[RowReader], final case class ScalarOperationMapper(operator: BinaryOperator, scalarOnLhs: Boolean, funcParams: Seq[FuncArgs]) extends RangeVectorTransformer { - protected[exec] def args: String = - s"operator=$operator, scalarOnLhs=$scalarOnLhs" + protected[exec] def args: String = s"operator=$operator, scalarOnLhs=$scalarOnLhs" val operatorFunction = BinaryOperatorFunction.factoryMethod(operator) @@ -259,8 +257,7 @@ final case class MiscellaneousFunctionMapper(function: MiscellaneousFunctionId, } final case class SortFunctionMapper(function: SortFunctionId) extends RangeVectorTransformer { - protected[exec] def args: String = - s"function=$function" + protected[exec] def args: String = s"function=$function" def apply(source: Observable[RangeVector], queryConfig: QueryConfig, @@ -297,8 +294,7 @@ final case class SortFunctionMapper(function: SortFunctionId) extends RangeVecto final case class ScalarFunctionMapper(function: ScalarFunctionId, timeStepParams: RangeParams) extends RangeVectorTransformer { - protected[exec] def args: String = - s"function=$function, funcParams=$funcParams" + protected[exec] def args: String = s"function=$function, funcParams=$funcParams" def scalarImpl(source: Observable[RangeVector]): Observable[RangeVector] = { @@ -327,8 +323,7 @@ final case class ScalarFunctionMapper(function: ScalarFunctionId, } final case class VectorFunctionMapper() extends RangeVectorTransformer { - protected[exec] def args: String = - s"funcParams=$funcParams" + protected[exec] def args: String = s"funcParams=$funcParams" def apply(source: Observable[RangeVector], queryConfig: QueryConfig, @@ -373,7 +368,7 @@ final case class AbsentFunctionMapper(columnFilter: Seq[ColumnFilter], rangePara val resultRv = nonNanTimestamps.map { t => val rowList = new ListBuffer[TransientRow]() - for (i <- rangeParams.start to rangeParams.end by rangeParams.step) { + for (i <- rangeParams.startSecs to rangeParams.endSecs by rangeParams.stepSecs) { if (!t.contains(i * 1000)) rowList += new TransientRow(i * 1000, 1) } @@ -387,7 +382,7 @@ final case class AbsentFunctionMapper(columnFilter: Seq[ColumnFilter], rangePara } override def funcParams: Seq[FuncArgs] = Nil override def schema(source: ResultSchema): ResultSchema = ResultSchema(Seq(ColumnInfo("timestamp", - ColumnType.LongColumn), ColumnInfo("value", ColumnType.DoubleColumn)), 1) + ColumnType.TimestampColumn), ColumnInfo("value", ColumnType.DoubleColumn)), 1) override def canHandleEmptySchemas: Boolean = true } @@ -459,7 +454,8 @@ final case class HistToPromSeriesMapper(sch: PartitionSchema) extends RangeVecto override def schema(source: ResultSchema): ResultSchema = if (valueColumnType(source) != ColumnType.HistogramColumn) source else - ResultSchema(Seq(ColumnInfo("timestamp", ColumnType.LongColumn), ColumnInfo("value", ColumnType.DoubleColumn)), 1) + ResultSchema(Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), + ColumnInfo("value", ColumnType.DoubleColumn)), 1) private def addNewBuckets(newScheme: HistogramBuckets, buckets: debox.Map[Double, debox.Buffer[Double]], diff --git a/query/src/main/scala/filodb/query/exec/ScalarFixedDoubleExec.scala b/query/src/main/scala/filodb/query/exec/ScalarFixedDoubleExec.scala index a854c940cc..944460b2da 100644 --- a/query/src/main/scala/filodb/query/exec/ScalarFixedDoubleExec.scala +++ b/query/src/main/scala/filodb/query/exec/ScalarFixedDoubleExec.scala @@ -1,7 +1,5 @@ package filodb.query.exec -import scala.concurrent.duration.FiniteDuration - import kamon.Kamon import monix.eval.Task import monix.execution.Scheduler @@ -12,21 +10,18 @@ import filodb.core.metadata.Column.ColumnType import filodb.core.query._ import filodb.core.store.ChunkSource import filodb.query.{QueryConfig, QueryResponse, QueryResult} -import filodb.query.Query.qLogger /** * Exec Plans for fixed scalars which can execute locally without being dispatched * Query example: 3, 4.2 */ -case class ScalarFixedDoubleExec(id: String, +case class ScalarFixedDoubleExec(queryContext: QueryContext, dataset: DatasetRef, params: RangeParams, - value: Double, - limit: Int, - submitTime: Long = System.currentTimeMillis()) extends LeafExecPlan { + value: Double) extends LeafExecPlan { - val columns: Seq[ColumnInfo] = Seq(ColumnInfo("timestamp", ColumnType.LongColumn), + val columns: Seq[ColumnInfo] = Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), ColumnInfo("value", ColumnType.DoubleColumn)) /** @@ -35,7 +30,7 @@ case class ScalarFixedDoubleExec(id: String, * node */ override def doExecute(source: ChunkSource, queryConfig: QueryConfig) - (implicit sched: Scheduler, timeout: FiniteDuration): ExecResult = { + (implicit sched: Scheduler): ExecResult = { throw new IllegalStateException("doExecute should not be called for ScalarFixedDoubleExec since it represents a " + "readily available static value") } @@ -49,11 +44,10 @@ case class ScalarFixedDoubleExec(id: String, override def execute(source: ChunkSource, queryConfig: QueryConfig) - (implicit sched: Scheduler, - timeout: FiniteDuration): Task[QueryResponse] = { + (implicit sched: Scheduler): Task[QueryResponse] = { val execPlan2Span = Kamon.spanBuilder(s"execute-${getClass.getSimpleName}") .asChildOf(Kamon.currentSpan()) - .tag("query-id", id) + .tag("query-id", queryContext.queryId) .start() val resultSchema = ResultSchema(columns, 1) val rangeVectors : Seq[RangeVector] = Seq(ScalarFixedDouble(params, value)) @@ -64,15 +58,14 @@ case class ScalarFixedDoubleExec(id: String, Task { val span = Kamon.spanBuilder(s"trasnform-${getClass.getSimpleName}") .asChildOf(execPlan2Span) - .tag("query-id", id) + .tag("query-id", queryContext.queryId) .start() rangeVectorTransformers.foldLeft((Observable.fromIterable(rangeVectors), resultSchema)) { (acc, transf) => - qLogger.debug(s"queryId: ${id} Setting up Transformer ${transf.getClass.getSimpleName} with ${transf.args}") val paramRangeVector: Seq[Observable[ScalarRangeVector]] = transf.funcParams.map(_.getResult) - (transf.apply(acc._1, queryConfig, limit, acc._2, paramRangeVector), transf.schema(acc._2)) + (transf.apply(acc._1, queryConfig, queryContext.sampleLimit, acc._2, paramRangeVector), transf.schema(acc._2)) }._1.toListL.map({ span.finish() - QueryResult(id, resultSchema, _) + QueryResult(queryContext.queryId, resultSchema, _) }) }.flatten } diff --git a/query/src/main/scala/filodb/query/exec/SelectChunkInfosExec.scala b/query/src/main/scala/filodb/query/exec/SelectChunkInfosExec.scala index 85a7eba842..e13e6ad34c 100644 --- a/query/src/main/scala/filodb/query/exec/SelectChunkInfosExec.scala +++ b/query/src/main/scala/filodb/query/exec/SelectChunkInfosExec.scala @@ -1,7 +1,5 @@ package filodb.query.exec -import scala.concurrent.duration.FiniteDuration - import kamon.Kamon import monix.eval.Task import monix.execution.Scheduler @@ -33,9 +31,7 @@ object SelectChunkInfosExec { * in the given shard, for the given row key range, for one particular column * ID (Long), NumRows (Int), startTime (Long), endTime (Long), numBytes(I) of chunk, readerclass of chunk */ -final case class SelectChunkInfosExec(id: String, - submitTime: Long, - limit: Int, +final case class SelectChunkInfosExec(queryContext: QueryContext, dispatcher: PlanDispatcher, dataset: DatasetRef, shard: Int, @@ -47,8 +43,7 @@ final case class SelectChunkInfosExec(id: String, def doExecute(source: ChunkSource, queryConfig: QueryConfig) - (implicit sched: Scheduler, - timeout: FiniteDuration): ExecResult = { + (implicit sched: Scheduler): ExecResult = { val partMethod = FilteredPartitionScan(ShardSplit(shard), filters) val lookupRes = source.lookupPartitions(dataset, partMethod, chunkMethod) diff --git a/query/src/main/scala/filodb/query/exec/SelectRawPartitionsExec.scala b/query/src/main/scala/filodb/query/exec/SelectRawPartitionsExec.scala index 5d135d0670..438bd0b133 100644 --- a/query/src/main/scala/filodb/query/exec/SelectRawPartitionsExec.scala +++ b/query/src/main/scala/filodb/query/exec/SelectRawPartitionsExec.scala @@ -1,7 +1,5 @@ package filodb.query.exec -import scala.concurrent.duration.FiniteDuration - import kamon.Kamon import monix.eval.Task import monix.execution.Scheduler @@ -10,7 +8,7 @@ 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.ResultSchema +import filodb.core.query.{QueryContext, ResultSchema} import filodb.core.store._ import filodb.query.{Query, QueryConfig} import filodb.query.Query.qLogger @@ -28,7 +26,7 @@ object SelectRawPartitionsExec extends { } def findFirstRangeFunction(transformers: Seq[RangeVectorTransformer]): Option[InternalRangeFunction] = - transformers.collect { case p: PeriodicSamplesMapper => p.functionId }.headOption.flatten + transformers.collectFirst { case p: PeriodicSamplesMapper => p.functionId }.flatten def replaceRangeFunction(transformers: Seq[RangeVectorTransformer], oldFunc: Option[InternalRangeFunction], @@ -104,9 +102,7 @@ object SelectRawPartitionsExec extends { * if false, the given schema is expected for all partitions, error will be thrown otherwise. * @param colIds the exact column IDs that are needed for querying the raw data */ -final case class SelectRawPartitionsExec(id: String, - submitTime: Long, - limit: Int, +final case class SelectRawPartitionsExec(queryContext: QueryContext, dispatcher: PlanDispatcher, datasetRef: DatasetRef, dataSchema: Option[Schema], @@ -125,13 +121,13 @@ final case class SelectRawPartitionsExec(id: String, def doExecute(source: ChunkSource, queryConfig: QueryConfig) - (implicit sched: Scheduler, - timeout: FiniteDuration): ExecResult = { + (implicit sched: Scheduler): ExecResult = { val span = Kamon.spanBuilder(s"execute-${getClass.getSimpleName}") .asChildOf(Kamon.currentSpan()) .start() - Query.qLogger.debug(s"queryId=$id on dataset=$datasetRef shard=${lookupRes.map(_.shard).getOrElse("")} " + - s"schema=${dataSchema.map(_.name)} is configured to use columnIDs=$colIds") + Query.qLogger.debug(s"queryId=${queryContext.queryId} on dataset=$datasetRef shard=" + + s"${lookupRes.map(_.shard).getOrElse("")} " + s"schema=" + + s"${dataSchema.map(_.name)} is configured to use columnIDs=$colIds") val rvs = dataSchema.map { sch => source.rangeVectors(datasetRef, lookupRes.get, colIds, sch, filterSchemas) }.getOrElse(Observable.empty) diff --git a/query/src/main/scala/filodb/query/exec/SetOperatorExec.scala b/query/src/main/scala/filodb/query/exec/SetOperatorExec.scala index 2514ad2053..3f72cf7067 100644 --- a/query/src/main/scala/filodb/query/exec/SetOperatorExec.scala +++ b/query/src/main/scala/filodb/query/exec/SetOperatorExec.scala @@ -6,6 +6,7 @@ import scala.collection.mutable.ListBuffer import monix.eval.Task import monix.reactive.Observable +import filodb.core.memstore.SchemaMismatch import filodb.core.query._ import filodb.memory.format.{ZeroCopyUTF8String => Utf8Str} import filodb.memory.format.ZeroCopyUTF8String._ @@ -24,17 +25,16 @@ import filodb.query.BinaryOperator.{LAND, LOR, LUnless} * @param lhs ExecPlan that will return results of LHS expression * @param rhs ExecPlan that will return results of RHS expression * @param binaryOp the binary operator - * @param cardinality the cardinality of the join relationship as a hint * @param on fields from range vector keys to include while performing the join * @param ignoring fields from range vector keys to exclude while performing the join */ -final case class SetOperatorExec(id: String, - dispatcher: PlanDispatcher, - lhs: Seq[ExecPlan], - rhs: Seq[ExecPlan], - binaryOp: BinaryOperator, - on: Seq[String], - ignoring: Seq[String], +final case class SetOperatorExec(queryContext: QueryContext, + dispatcher: PlanDispatcher, + lhs: Seq[ExecPlan], + rhs: Seq[ExecPlan], + binaryOp: BinaryOperator, + on: Seq[String], + ignoring: Seq[String], metricColumn: String) extends NonLeafExecPlan { require(on == Nil || ignoring == Nil, "Cannot specify both 'on' and 'ignoring' clause") require(!on.contains(metricColumn), "On cannot contain metric name") @@ -85,7 +85,7 @@ final case class SetOperatorExec(id: String, var result = new ListBuffer[RangeVector]() rhsRvs.foreach { rv => val jk = joinKeys(rv.key) - if (!jk.isEmpty) + if (jk.nonEmpty) rhsKeysSet += jk } @@ -137,4 +137,18 @@ final case class SetOperatorExec(id: String, } result.toList } + + /** + * overridden to allow schemas with different vector lengths, colids as long as the columns are same - to handle + * binary joins between scalar/rangevectors + */ + override def reduceSchemas(rs: ResultSchema, resp: QueryResult): ResultSchema = { + resp match { + case QueryResult(_, schema, _) if rs == ResultSchema.empty => + schema /// First schema, take as is + case QueryResult(_, schema, _) => + if (!rs.hasSameColumnsAs(schema)) throw SchemaMismatch(rs.toString, schema.toString) + else rs + } + } } diff --git a/query/src/main/scala/filodb/query/exec/StitchRvsExec.scala b/query/src/main/scala/filodb/query/exec/StitchRvsExec.scala index aeff6fb51e..7cad5555b5 100644 --- a/query/src/main/scala/filodb/query/exec/StitchRvsExec.scala +++ b/query/src/main/scala/filodb/query/exec/StitchRvsExec.scala @@ -28,7 +28,7 @@ object StitchRvsExec { bVectors.foreach { r => if (r.hasNext) { val t = r.head.getLong(0) - if (mins.size == 0) { + if (mins.isEmpty) { minTime = t mins += r } @@ -57,7 +57,7 @@ object StitchRvsExec { /** * Use when data for same time series spans multiple shards, or clusters. */ -final case class StitchRvsExec(id: String, +final case class StitchRvsExec(queryContext: QueryContext, dispatcher: PlanDispatcher, children: Seq[ExecPlan]) extends NonLeafExecPlan { require(children.nonEmpty) diff --git a/query/src/main/scala/filodb/query/exec/TimeScalarGeneratorExec.scala b/query/src/main/scala/filodb/query/exec/TimeScalarGeneratorExec.scala index 6ff0cebe6d..f9c344de55 100644 --- a/query/src/main/scala/filodb/query/exec/TimeScalarGeneratorExec.scala +++ b/query/src/main/scala/filodb/query/exec/TimeScalarGeneratorExec.scala @@ -1,7 +1,5 @@ package filodb.query.exec -import scala.concurrent.duration.FiniteDuration - import kamon.Kamon import monix.eval.Task import monix.execution.Scheduler @@ -12,20 +10,17 @@ 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.Query.qLogger import filodb.query.ScalarFunctionId.{DayOfMonth, DayOfWeek, DaysInMonth, Hour, Minute, Month, Time, Year} /** * Exec Plans for time functions which can execute locally without being dispatched as they don't have any input * Query example: time(), hour() */ -case class TimeScalarGeneratorExec(id: String, +case class TimeScalarGeneratorExec(queryContext: QueryContext, dataset: DatasetRef, params: RangeParams, - function: ScalarFunctionId, - limit: Int, - submitTime: Long = System.currentTimeMillis()) extends LeafExecPlan { + function: ScalarFunctionId) extends LeafExecPlan { - val columns: Seq[ColumnInfo] = Seq(ColumnInfo("timestamp", ColumnType.LongColumn), + val columns: Seq[ColumnInfo] = Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), ColumnInfo("value", ColumnType.DoubleColumn)) /** @@ -34,7 +29,7 @@ case class TimeScalarGeneratorExec(id: String, * node */ override def doExecute(source: ChunkSource, queryConfig: QueryConfig) - (implicit sched: Scheduler, timeout: FiniteDuration): ExecResult = { + (implicit sched: Scheduler): ExecResult = { throw new IllegalStateException("doExecute should not be called for TimeScalarGeneratorExec since it represents" + "a readily available static value") } @@ -47,11 +42,10 @@ case class TimeScalarGeneratorExec(id: String, override def execute(source: ChunkSource, queryConfig: QueryConfig) - (implicit sched: Scheduler, - timeout: FiniteDuration): Task[QueryResponse] = { + (implicit sched: Scheduler): Task[QueryResponse] = { val execPlan2Span = Kamon.spanBuilder(s"execute-${getClass.getSimpleName}") .asChildOf(Kamon.currentSpan()) - .tag("query-id", id) + .tag("query-id", queryContext.queryId) .start() val resultSchema = ResultSchema(columns, 1) val rangeVectors : Seq[RangeVector] = function match { @@ -72,16 +66,15 @@ case class TimeScalarGeneratorExec(id: String, Task { val span = Kamon.spanBuilder(s"transform-${getClass.getSimpleName}") .asChildOf(execPlan2Span) - .tag("query-id", id) + .tag("query-id", queryContext.queryId) .start() rangeVectorTransformers.foldLeft((Observable.fromIterable(rangeVectors), resultSchema)) { (acc, transf) => - qLogger.debug(s"queryId: ${id} Setting up Transformer ${transf.getClass.getSimpleName} with ${transf.args}") span.mark(transf.getClass.getSimpleName) val paramRangeVector: Seq[Observable[ScalarRangeVector]] = transf.funcParams.map(_.getResult) - (transf.apply(acc._1, queryConfig, limit, acc._2, paramRangeVector), transf.schema(acc._2)) + (transf.apply(acc._1, queryConfig, queryContext.sampleLimit, acc._2, paramRangeVector), transf.schema(acc._2)) }._1.toListL.map({ span.finish() - QueryResult(id, resultSchema, _) + QueryResult(queryContext.queryId, resultSchema, _) }) }.flatten } diff --git a/query/src/main/scala/filodb/query/exec/aggregator/CountValuesRowAggregator.scala b/query/src/main/scala/filodb/query/exec/aggregator/CountValuesRowAggregator.scala index ead750c1a3..a51799f6d1 100644 --- a/query/src/main/scala/filodb/query/exec/aggregator/CountValuesRowAggregator.scala +++ b/query/src/main/scala/filodb/query/exec/aggregator/CountValuesRowAggregator.scala @@ -82,7 +82,8 @@ class CountValuesRowAggregator(label: String, limit: Int = 1000) extends RowAggr } def present(aggRangeVector: RangeVector, limit: Int): Seq[RangeVector] = { - val colSchema = Seq(ColumnInfo("timestamp", ColumnType.LongColumn), ColumnInfo("value", ColumnType.DoubleColumn)) + val colSchema = Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), + ColumnInfo("value", ColumnType.DoubleColumn)) val recSchema = SerializedRangeVector.toSchema(colSchema) val resRvs = mutable.Map[RangeVectorKey, RecordBuilder]() // Important TODO / TechDebt: We need to replace Iterators with cursors to better control @@ -109,7 +110,7 @@ class CountValuesRowAggregator(label: String, limit: Int = 1000) extends RowAggr ChunkMap.releaseAllSharedLocks() } resRvs.map { case (key, builder) => - val numRows = builder.allContainers.map(_.countRecords).sum + val numRows = builder.allContainers.map(_.countRecords()).sum new SerializedRangeVector(key, numRows, builder.allContainers, recSchema, 0) }.toSeq } @@ -122,7 +123,7 @@ class CountValuesRowAggregator(label: String, limit: Int = 1000) extends RowAggr } def presentationSchema(reductionSchema: ResultSchema): ResultSchema = { - ResultSchema(Array(reductionSchema.columns(0), ColumnInfo("value", ColumnType.DoubleColumn)), 1) + ResultSchema(Array(reductionSchema.columns.head, ColumnInfo("value", ColumnType.DoubleColumn)), 1) } } diff --git a/query/src/main/scala/filodb/query/exec/aggregator/TopBottomKRowAggregator.scala b/query/src/main/scala/filodb/query/exec/aggregator/TopBottomKRowAggregator.scala index 212ee90823..2eaa39c3c5 100644 --- a/query/src/main/scala/filodb/query/exec/aggregator/TopBottomKRowAggregator.scala +++ b/query/src/main/scala/filodb/query/exec/aggregator/TopBottomKRowAggregator.scala @@ -82,7 +82,8 @@ class TopBottomKRowAggregator(k: Int, bottomK: Boolean) extends RowAggregator { } def present(aggRangeVector: RangeVector, limit: Int): Seq[RangeVector] = { - val colSchema = Seq(ColumnInfo("timestamp", ColumnType.LongColumn), ColumnInfo("value", ColumnType.DoubleColumn)) + val colSchema = Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), + ColumnInfo("value", ColumnType.DoubleColumn)) val recSchema = SerializedRangeVector.toSchema(colSchema) val resRvs = mutable.Map[RangeVectorKey, RecordBuilder]() // Important TODO / TechDebt: We need to replace Iterators with cursors to better control 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 fbb0c1c691..994d3fbcc8 100644 --- a/query/src/main/scala/filodb/query/exec/rangefn/AggrOverTimeFunctions.scala +++ b/query/src/main/scala/filodb/query/exec/rangefn/AggrOverTimeFunctions.scala @@ -155,11 +155,11 @@ class SumOverTimeChunkedFunctionD extends SumOverTimeChunkedFunction() with Chun doubleReader: bv.DoubleVectorDataReader, startRowNum: Int, endRowNum: Int): Unit = { - // NaN values are ignored by default in the sum method - if (sum.isNaN) { - sum = 0d - } - sum += doubleReader.sum(doubleVectAcc, doubleVect, startRowNum, endRowNum) + + //Takes care of computing sum in all seq types : combination of NaN & not NaN & all numbers. + val chunkSum = doubleReader.sum(doubleVectAcc, doubleVect, startRowNum, endRowNum) + if(!JLDouble.isNaN(chunkSum) && JLDouble.isNaN(sum)) sum = 0d + sum += chunkSum } } @@ -408,15 +408,15 @@ class AvgOverTimeFunction(var sum: Double = Double.NaN, var count: Int = 0) exte } } -abstract class AvgOverTimeChunkedFunction(var sum: Double = Double.NaN, var count: Double = 0) +abstract class AvgOverTimeChunkedFunction(var sum: Double = Double.NaN, var count: Int = 0) extends ChunkedRangeFunction[TransientRow] { override final def reset(): Unit = { - sum = Double.NaN; - count = 0d + sum = Double.NaN + count = 0 } final def apply(endTimestamp: Long, sampleToEmit: TransientRow): Unit = { - sampleToEmit.setValues(endTimestamp, if (count > 0) sum/count else if (sum.isNaN()) sum else 0d) + sampleToEmit.setValues(endTimestamp, if (count > 0) sum / count else if (sum.isNaN()) sum else 0d) } } @@ -426,10 +426,11 @@ class AvgOverTimeChunkedFunctionD extends AvgOverTimeChunkedFunction() with Chun doubleReader: bv.DoubleVectorDataReader, startRowNum: Int, endRowNum: Int): Unit = { - if (sum.isNaN) { - sum = 0d - } - sum += doubleReader.sum(doubleVectAcc, doubleVect, startRowNum, endRowNum) + + // Takes care of computing sum in all seq types : combination of NaN & not NaN & all numbers. + val chunkSum = doubleReader.sum(doubleVectAcc, doubleVect, startRowNum, endRowNum) + if(!JLDouble.isNaN(chunkSum) && JLDouble.isNaN(sum)) sum = 0d + sum += chunkSum count += doubleReader.count(doubleVectAcc, doubleVect, startRowNum, endRowNum) } } @@ -493,45 +494,64 @@ class StdVarOverTimeFunction(var sum: Double = 0d, } } -abstract class VarOverTimeChunkedFunctionD(var sum: Double = 0d, +abstract class VarOverTimeChunkedFunctionD(var sum: Double = Double.NaN, var count: Int = 0, - var squaredSum: Double = 0d) extends ChunkedDoubleRangeFunction { - override final def reset(): Unit = { sum = 0d; count = 0; squaredSum = 0d } + var squaredSum: Double = Double.NaN, + var lastSample: Double = Double.NaN) extends ChunkedDoubleRangeFunction { + override final def reset(): Unit = { sum = Double.NaN; count = 0; squaredSum = Double.NaN } final def addTimeDoubleChunks(doubleVectAcc: MemoryReader, doubleVect: BinaryVector.BinaryVectorPtr, doubleReader: bv.DoubleVectorDataReader, startRowNum: Int, endRowNum: Int): Unit = { + // Takes care of computing sum in all seq types : combination of NaN & not NaN & all numbers. val it = doubleReader.iterate(doubleVectAcc, doubleVect, startRowNum) - var _sum = 0d - var _sqSum = 0d var elemNo = startRowNum + var chunkSum = Double.NaN + var chunkSquaredSum = Double.NaN + var chunkCount = 0 while (elemNo <= endRowNum) { val nextValue = it.next if (!JLDouble.isNaN(nextValue)) { - _sum += nextValue - _sqSum += nextValue * nextValue - elemNo += 1 + if (chunkSum.isNaN()) chunkSum = 0d + if (chunkSquaredSum.isNaN()) chunkSquaredSum = 0d + if (elemNo == endRowNum) lastSample = nextValue + chunkSum += nextValue + chunkSquaredSum += nextValue * nextValue + chunkCount +=1 } + elemNo += 1 } - count += (endRowNum - startRowNum + 1) - sum += _sum - squaredSum += _sqSum + if(!JLDouble.isNaN(chunkSum) && JLDouble.isNaN(sum)) sum = 0d + sum += chunkSum + if(!JLDouble.isNaN(chunkSquaredSum) && JLDouble.isNaN(squaredSum)) squaredSum = 0d + squaredSum += chunkSquaredSum + count += chunkCount } } class StdDevOverTimeChunkedFunctionD extends VarOverTimeChunkedFunctionD() { final def apply(endTimestamp: Long, sampleToEmit: TransientRow): Unit = { - val avg = if (count > 0) sum/count else 0d - val stdDev = Math.sqrt(squaredSum/count - avg*avg) + var stdDev = Double.NaN + if (count > 0) { + val avg = sum / count + stdDev = Math.sqrt((squaredSum / count) - (avg * avg)) + } + else if (sum.isNaN()) stdDev = sum + else stdDev = 0d sampleToEmit.setValues(endTimestamp, stdDev) } } class StdVarOverTimeChunkedFunctionD extends VarOverTimeChunkedFunctionD() { final def apply(endTimestamp: Long, sampleToEmit: TransientRow): Unit = { - val avg = if (count > 0) sum/count else 0d - val stdVar = squaredSum/count - avg*avg + var stdVar = Double.NaN + if (count > 0) { + val avg = sum / count + stdVar = (squaredSum / count) - (avg * avg) + } + else if (sum.isNaN()) stdVar = sum + else stdVar = 0d sampleToEmit.setValues(endTimestamp, stdVar) } } @@ -931,46 +951,16 @@ class PredictLinearChunkedFunctionL(funcParams: Seq[Any]) extends PredictLinearC * It represents the distance between the raw score and the population mean in units of the standard deviation. * Refer https://en.wikipedia.org/wiki/Standard_score#Calculation to understand how to calculate **/ - -abstract class ZScoreChunkedFunction(var stdOverTimeChunkedFunction: StdDevOverTimeChunkedFunctionD = null, - var avgOverTimeChunkedFunction: AvgOverTimeChunkedFunctionD = null, - var lastSampleChunkedFunction: LastSampleChunkedFunctionWithNanD = null) - extends ChunkedRangeFunction[TransientRow] { - - override final def reset(): Unit = { stdOverTimeChunkedFunction = new StdDevOverTimeChunkedFunctionD() - avgOverTimeChunkedFunction = new AvgOverTimeChunkedFunctionD() - lastSampleChunkedFunction = new LastSampleChunkedFunctionWithNanD() } - +class ZScoreChunkedFunctionD extends VarOverTimeChunkedFunctionD() { final def apply(endTimestamp: Long, sampleToEmit: TransientRow): Unit = { - val stdDevTransientRow, avgTransientRow, lastSampleTransientRow = new TransientRow() - stdOverTimeChunkedFunction.apply(endTimestamp, stdDevTransientRow) - avgOverTimeChunkedFunction.apply(endTimestamp, avgTransientRow) - lastSampleChunkedFunction.apply(endTimestamp, lastSampleTransientRow) - val stdDev = stdDevTransientRow.value - val avg = avgTransientRow.value - val lastSample = lastSampleTransientRow.value - val zscore = (lastSample - avg)/stdDev + var zscore = Double.NaN + if (count > 0) { + val avg = sum / count + val stdDev = Math.sqrt(squaredSum / count - avg*avg) + zscore = (lastSample - avg) / stdDev + } + else if (sum.isNaN()) zscore = sum + else zscore = 0d sampleToEmit.setValues(endTimestamp, zscore) } -} - -class ZScoreChunkedFunctionD() extends ZScoreChunkedFunction() - with ChunkedRangeFunction[TransientRow] { - - override final def addChunks(tsVectorAcc: MemoryReader, tsVector: BinaryVector.BinaryVectorPtr, - tsReader: bv.LongVectorDataReader, valueVectorAcc: MemoryReader, - valueVector: BinaryVector.BinaryVectorPtr, valueReader: VectorDataReader, - startTime: Long, endTime: Long, info: ChunkSetInfoReader, - queryConfig: QueryConfig): Unit = { - - val startRowNum = tsReader.binarySearch(tsVectorAcc, tsVector, startTime) & 0x7fffffff - val endRowNum = Math.min(tsReader.ceilingIndex(tsVectorAcc, tsVector, endTime), info.numRows - 1) - stdOverTimeChunkedFunction.addTimeDoubleChunks(valueVectorAcc, valueVector, valueReader.asDoubleReader, - startRowNum, endRowNum) - avgOverTimeChunkedFunction.addTimeDoubleChunks(valueVectorAcc, valueVector, valueReader.asDoubleReader, - startRowNum, endRowNum) - lastSampleChunkedFunction.addChunks(tsVectorAcc, tsVector, tsReader, valueVectorAcc, valueVector, valueReader, - startTime, endTime, info, queryConfig) - - } } \ No newline at end of file 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 4f9f5f5493..0dcf546f7a 100644 --- a/query/src/main/scala/filodb/query/exec/rangefn/RangeFunction.scala +++ b/query/src/main/scala/filodb/query/exec/rangefn/RangeFunction.scala @@ -557,16 +557,3 @@ class TimestampChunkedFunction (var value: Double = Double.NaN) extends ChunkedR } } -class LastSampleChunkedFunctionWithNanD extends LastSampleChunkedFuncDblVal() { - def updateValue(ts: Long, valAcc: MemoryReader, valVector: BinaryVectorPtr, - valReader: VectorDataReader, endRowNum: Int): Unit = { - val dblReader = valReader.asDoubleReader - val doubleVal = dblReader(valAcc, valVector, endRowNum) - // If the last value is NaN, that may be Prometheus end of time series marker. - // In that case try to get the sample before last. - // If endRowNum==0, we are at beginning of chunk, and if the window included the last chunk, then - // the call to addChunks to the last chunk would have gotten the last sample value anyways. - timestamp = ts - value = doubleVal - } -} diff --git a/query/src/test/scala/filodb/query/ResultTypesSpec.scala b/query/src/test/scala/filodb/query/ResultTypesSpec.scala index 7186036a8d..7c4bd4edd1 100644 --- a/query/src/test/scala/filodb/query/ResultTypesSpec.scala +++ b/query/src/test/scala/filodb/query/ResultTypesSpec.scala @@ -10,7 +10,7 @@ import filodb.memory.format.{RowReader, ZeroCopyUTF8String} class ResultTypesSpec extends FunSpec with Matchers with ScalaFutures { - val columns: Seq[ColumnInfo] = Seq(ColumnInfo("timestamp", ColumnType.LongColumn), + val columns: Seq[ColumnInfo] = Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), ColumnInfo("value", ColumnType.DoubleColumn)) val resultSchema = ResultSchema(columns, 1) val ignoreKey = CustomRangeVectorKey( diff --git a/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala b/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala index 9027170f21..44b4a3fb97 100644 --- a/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala +++ b/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala @@ -19,7 +19,7 @@ class AggrOverRangeVectorsSpec extends RawDataWindowingSpec with ScalaFutures { val rand = new Random() val error = 0.0000001d - val tvSchema = ResultSchema(Seq(ColumnInfo("timestamp", ColumnType.LongColumn), + val tvSchema = ResultSchema(Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), ColumnInfo("value", ColumnType.DoubleColumn)), 1) val histSchema = ResultSchema(MMD.histDataset.schema.infosFromIDs(Seq(0, 3)), 1) val histMaxSchema = ResultSchema(MMD.histMaxDS.schema.infosFromIDs(Seq(0, 4, 3)), 1, colIDs = Seq(0, 4, 3)) @@ -282,7 +282,7 @@ class AggrOverRangeVectorsSpec extends RawDataWindowingSpec with ScalaFutures { val result7 = resultObs7.toListL.runAsync.futureValue result7.size shouldEqual 1 - val recSchema = SerializedRangeVector.toSchema(Seq(ColumnInfo("timestamp", ColumnType.LongColumn), + val recSchema = SerializedRangeVector.toSchema(Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), ColumnInfo("tdig", ColumnType.StringColumn))) val builder = SerializedRangeVector.newBuilder() val srv = SerializedRangeVector(result7(0), builder, recSchema, "Unit-Test") @@ -482,8 +482,8 @@ class AggrOverRangeVectorsSpec extends RawDataWindowingSpec with ScalaFutures { result(0).key shouldEqual noKey val sums = data1.zip(data2).map { case (row1, row2) => - val h1 = bv.MutableHistogram(row1(3).asInstanceOf[bv.MutableHistogram]) - h1.add(row2(3).asInstanceOf[bv.MutableHistogram]) + val h1 = bv.MutableHistogram(row1(3).asInstanceOf[bv.LongHistogram]) + h1.add(row2(3).asInstanceOf[bv.LongHistogram]) h1 }.toList @@ -512,8 +512,8 @@ class AggrOverRangeVectorsSpec extends RawDataWindowingSpec with ScalaFutures { result(0).key shouldEqual noKey val sums = data1.zip(data2).map { case (row1, row2) => - val h1 = bv.MutableHistogram(row1(4).asInstanceOf[bv.MutableHistogram]) - h1.add(row2(4).asInstanceOf[bv.MutableHistogram]) + val h1 = bv.MutableHistogram(row1(4).asInstanceOf[bv.LongHistogram]) + h1.add(row2(4).asInstanceOf[bv.LongHistogram]) h1 }.toList @@ -547,7 +547,7 @@ class AggrOverRangeVectorsSpec extends RawDataWindowingSpec with ScalaFutures { result.flatMap(_.rows.map(x => (x.getLong(0), x.getDouble(1))).toList).sameElements(expectedRows) shouldEqual true } - + @tailrec final private def compareIter(it1: Iterator[Double], it2: Iterator[Double]) : Unit = { (it1.hasNext, it2.hasNext) match{ diff --git a/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala b/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala index 5288e887f2..aa4a067b78 100644 --- a/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala @@ -1,6 +1,5 @@ package filodb.query.exec -import scala.concurrent.duration.FiniteDuration import scala.util.Random import com.typesafe.config.ConfigFactory @@ -25,16 +24,15 @@ class BinaryJoinExecSpec extends FunSpec with Matchers with ScalaFutures { val rand = new Random() val error = 0.00000001d - val tvSchema = ResultSchema(Seq(ColumnInfo("timestamp", ColumnType.LongColumn), + val tvSchema = ResultSchema(Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), ColumnInfo("value", ColumnType.DoubleColumn)), 1) - val schema = Seq(ColumnInfo("timestamp", ColumnType.LongColumn), + val schema = Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), ColumnInfo("value", ColumnType.DoubleColumn)) val tvSchemaTask = Task.now(tvSchema) val dummyDispatcher = new PlanDispatcher { override def dispatch(plan: ExecPlan) - (implicit sched: Scheduler, - timeout: FiniteDuration): Task[QueryResponse] = ??? + (implicit sched: Scheduler): Task[QueryResponse] = ??? } private def data(i: Int) = Stream.from(0).map(n => new TransientRow(n.toLong, i.toDouble)).take(20) @@ -84,7 +82,7 @@ class BinaryJoinExecSpec extends FunSpec with Matchers with ScalaFutures { val samplesRhs2 = scala.util.Random.shuffle(samplesRhs.toList) // they may come out of order - val execPlan = BinaryJoinExec("someID", dummyDispatcher, + val execPlan = BinaryJoinExec(QueryContext(), dummyDispatcher, Array(dummyPlan), // cannot be empty as some compose's rely on the schema new Array[ExecPlan](1), // empty since we test compose, not execute or doExecute BinaryOperator.ADD, @@ -114,7 +112,7 @@ class BinaryJoinExecSpec extends FunSpec with Matchers with ScalaFutures { val samplesRhs2 = scala.util.Random.shuffle(samplesRhs.take(100).toList) // they may come out of order - val execPlan = BinaryJoinExec("someID", dummyDispatcher, + val execPlan = BinaryJoinExec(QueryContext(), dummyDispatcher, new Array[ExecPlan](1), // empty since we test compose, not execute or doExecute new Array[ExecPlan](1), // empty since we test compose, not execute or doExecute BinaryOperator.ADD, @@ -150,7 +148,7 @@ class BinaryJoinExecSpec extends FunSpec with Matchers with ScalaFutures { } val samplesRhs2 = scala.util.Random.shuffle(duplicate +: samplesRhs.toList) // they may come out of order - val execPlan = BinaryJoinExec("someID", dummyDispatcher, + val execPlan = BinaryJoinExec(QueryContext(), dummyDispatcher, new Array[ExecPlan](1), // empty since we test compose, not execute or doExecute new Array[ExecPlan](1), // empty since we test compose, not execute or doExecute BinaryOperator.ADD, @@ -181,7 +179,7 @@ class BinaryJoinExecSpec extends FunSpec with Matchers with ScalaFutures { val samplesLhs2 = scala.util.Random.shuffle(duplicate +: samplesLhs.toList) // they may come out of order - val execPlan = BinaryJoinExec("some ID", dummyDispatcher, + val execPlan = BinaryJoinExec(QueryContext(), dummyDispatcher, new Array[ExecPlan](1), // empty since we test compose, not execute or doExecute new Array[ExecPlan](1), // empty since we test compose, not execute or doExecute BinaryOperator.ADD, @@ -202,7 +200,7 @@ class BinaryJoinExecSpec extends FunSpec with Matchers with ScalaFutures { } it("should join one-to-one with ignoring") { - val execPlan = BinaryJoinExec("someID", dummyDispatcher, + val execPlan = BinaryJoinExec(QueryContext(), dummyDispatcher, Array(dummyPlan), // cannot be empty as some compose's rely on the schema new Array[ExecPlan](1), // empty since we test compose, not execute or doExecute BinaryOperator.ADD, @@ -231,7 +229,7 @@ class BinaryJoinExecSpec extends FunSpec with Matchers with ScalaFutures { it("should join one-to-one with on") { - val execPlan = BinaryJoinExec("someID", dummyDispatcher, + val execPlan = BinaryJoinExec(QueryContext(), dummyDispatcher, Array(dummyPlan), // cannot be empty as some compose's rely on the schema new Array[ExecPlan](1), // empty since we test compose, not execute or doExecute BinaryOperator.ADD, @@ -258,7 +256,7 @@ class BinaryJoinExecSpec extends FunSpec with Matchers with ScalaFutures { } it("should join one-to-one when metric name is not _name_") { - val execPlan = BinaryJoinExec("someID", dummyDispatcher, + val execPlan = BinaryJoinExec(QueryContext(), dummyDispatcher, Array(dummyPlan), // cannot be empty as some compose's rely on the schema new Array[ExecPlan](1), // empty since we test compose, not execute or doExecute BinaryOperator.ADD, @@ -327,7 +325,7 @@ class BinaryJoinExecSpec extends FunSpec with Matchers with ScalaFutures { } } - val execPlan = BinaryJoinExec("someID", dummyDispatcher, + val execPlan = BinaryJoinExec(QueryContext(), dummyDispatcher, Array(dummyPlan), // cannot be empty as some compose's rely on the schema new Array[ExecPlan](1), // empty since we test compose, not execute or doExecute BinaryOperator.GTR, diff --git a/query/src/test/scala/filodb/query/exec/BinaryJoinGroupingSpec.scala b/query/src/test/scala/filodb/query/exec/BinaryJoinGroupingSpec.scala index 4b7c511b1d..d88f1a1109 100644 --- a/query/src/test/scala/filodb/query/exec/BinaryJoinGroupingSpec.scala +++ b/query/src/test/scala/filodb/query/exec/BinaryJoinGroupingSpec.scala @@ -1,6 +1,5 @@ package filodb.query.exec -import scala.concurrent.duration.FiniteDuration import scala.util.Random import com.typesafe.config.ConfigFactory import monix.eval.Task @@ -23,9 +22,9 @@ 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 tvSchema = ResultSchema(Seq(ColumnInfo("timestamp", ColumnType.LongColumn), + val tvSchema = ResultSchema(Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), ColumnInfo("value", ColumnType.DoubleColumn)), 1) - val schema = Seq(ColumnInfo("timestamp", ColumnType.LongColumn), + val schema = Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), ColumnInfo("value", ColumnType.DoubleColumn)) val tvSchemaTask = Task.now(tvSchema) @@ -33,8 +32,7 @@ class BinaryJoinGroupingSpec extends FunSpec with Matchers with ScalaFutures { val dummyDispatcher = new PlanDispatcher { override def dispatch(plan: ExecPlan) - (implicit sched: Scheduler, - timeout: FiniteDuration): Task[QueryResponse] = ??? + (implicit sched: Scheduler): Task[QueryResponse] = ??? } val sampleNodeCpu: Array[RangeVector] = Array( @@ -114,7 +112,7 @@ class BinaryJoinGroupingSpec extends FunSpec with Matchers with ScalaFutures { val samplesRhs2 = scala.util.Random.shuffle(sampleNodeRole.toList) // they may come out of order - val execPlan = BinaryJoinExec("someID", dummyDispatcher, + val execPlan = BinaryJoinExec(QueryContext(), dummyDispatcher, Array(dummyPlan), // cannot be empty as some compose's rely on the schema new Array[ExecPlan](1), // empty since we test compose, not execute or doExecute BinaryOperator.MUL, @@ -149,7 +147,7 @@ class BinaryJoinGroupingSpec extends FunSpec with Matchers with ScalaFutures { val samplesRhs2 = scala.util.Random.shuffle(sampleNodeRole.toList) // they may come out of order - val execPlan = BinaryJoinExec("someID", dummyDispatcher, + val execPlan = BinaryJoinExec(QueryContext(), dummyDispatcher, Array(dummyPlan), new Array[ExecPlan](1), BinaryOperator.MUL, @@ -189,7 +187,7 @@ class BinaryJoinGroupingSpec extends FunSpec with Matchers with ScalaFutures { val resultObs4 = RangeVectorAggregator.mapReduce(agg, true, mapped, rv=>rv.key) val samplesRhs = resultObs4.toListL.runAsync.futureValue - val execPlan = BinaryJoinExec("someID", dummyDispatcher, + val execPlan = BinaryJoinExec(QueryContext(), dummyDispatcher, Array(dummyPlan), new Array[ExecPlan](1), BinaryOperator.DIV, @@ -233,7 +231,7 @@ class BinaryJoinGroupingSpec extends FunSpec with Matchers with ScalaFutures { val samplesRhs2 = scala.util.Random.shuffle(sampleNodeVar.toList) // they may come out of order - val execPlan = BinaryJoinExec("someID", dummyDispatcher, + val execPlan = BinaryJoinExec(QueryContext(), dummyDispatcher, Array(dummyPlan), new Array[ExecPlan](1), BinaryOperator.MUL, @@ -268,7 +266,7 @@ class BinaryJoinGroupingSpec extends FunSpec with Matchers with ScalaFutures { val resultObs4 = RangeVectorAggregator.mapReduce(agg, true, mapped, rv=>rv.key) val samplesRhs = resultObs4.toListL.runAsync.futureValue - val execPlan = BinaryJoinExec("someID", dummyDispatcher, + val execPlan = BinaryJoinExec(QueryContext(), dummyDispatcher, Array(dummyPlan), new Array[ExecPlan](1), BinaryOperator.DIV, @@ -348,7 +346,7 @@ class BinaryJoinGroupingSpec extends FunSpec with Matchers with ScalaFutures { } ) - val execPlan = BinaryJoinExec("someID", dummyDispatcher, + val execPlan = BinaryJoinExec(QueryContext(), dummyDispatcher, Array(dummyPlan), // cannot be empty as some compose's rely on the schema new Array[ExecPlan](1), // empty since we test compose, not execute or doExecute BinaryOperator.GTR, diff --git a/query/src/test/scala/filodb/query/exec/BinaryJoinSetOperatorSpec.scala b/query/src/test/scala/filodb/query/exec/BinaryJoinSetOperatorSpec.scala index 25f4c35144..07d47254a5 100644 --- a/query/src/test/scala/filodb/query/exec/BinaryJoinSetOperatorSpec.scala +++ b/query/src/test/scala/filodb/query/exec/BinaryJoinSetOperatorSpec.scala @@ -1,6 +1,5 @@ package filodb.query.exec -import scala.concurrent.duration.FiniteDuration import scala.util.Random import com.typesafe.config.ConfigFactory @@ -25,9 +24,9 @@ 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 tvSchema = ResultSchema(Seq(ColumnInfo("timestamp", ColumnType.LongColumn), + val tvSchema = ResultSchema(Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), ColumnInfo("value", ColumnType.DoubleColumn)), 1) - val schema = Seq(ColumnInfo("timestamp", ColumnType.LongColumn), + val schema = Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), ColumnInfo("value", ColumnType.DoubleColumn)) val rand = new Random() @@ -36,8 +35,7 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures val dummyDispatcher = new PlanDispatcher { override def dispatch(plan: ExecPlan) - (implicit sched: Scheduler, - timeout: FiniteDuration): Task[QueryResponse] = ??? + (implicit sched: Scheduler): Task[QueryResponse] = ??? } val resultSchema = ResultSchema(MetricsTestData.timeseriesSchema.infosFromIDs(0 to 1), 1) val resSchemaTask = Task.now(resultSchema) @@ -178,7 +176,7 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures it("should join many-to-many with and") { val sampleRhsShuffled = scala.util.Random.shuffle(sampleInstance0.toList) - val execPlan = SetOperatorExec("someID", dummyDispatcher, + val execPlan = SetOperatorExec(QueryContext(), dummyDispatcher, Array(dummyPlan), new Array[ExecPlan](1), BinaryOperator.LAND, @@ -213,7 +211,7 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures val sampleRhsShuffled = scala.util.Random.shuffle(sampleInstance0.toList) - val execPlan = SetOperatorExec("someID", dummyDispatcher, + val execPlan = SetOperatorExec(QueryContext(), dummyDispatcher, Array(dummyPlan), new Array[ExecPlan](1), BinaryOperator.LAND, @@ -250,7 +248,7 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures val sampleRhsShuffled = scala.util.Random.shuffle(sampleProductionInstance0.toList) - val execPlan = SetOperatorExec("someID", dummyDispatcher, + val execPlan = SetOperatorExec(QueryContext(), dummyDispatcher, Array(dummyPlan), new Array[ExecPlan](1), BinaryOperator.LAND, @@ -287,7 +285,7 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures val sampleRhsShuffled = scala.util.Random.shuffle(sampleProductionInstance0.toList) - val execPlan = SetOperatorExec("someID", dummyDispatcher, + val execPlan = SetOperatorExec(QueryContext(), dummyDispatcher, Array(dummyPlan), new Array[ExecPlan](1), BinaryOperator.LAND, @@ -323,7 +321,7 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures val sampleRhsShuffled = scala.util.Random.shuffle(sampleProductionInstance0.toList) - val execPlan = SetOperatorExec("someID", dummyDispatcher, + val execPlan = SetOperatorExec(QueryContext(), dummyDispatcher, Array(dummyPlan), new Array[ExecPlan](1), BinaryOperator.LAND, @@ -358,7 +356,7 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures it("should do LAND with ignoring having multiple labels") { val sampleRhsShuffled = scala.util.Random.shuffle(sampleProductionInstance0.toList) - val execPlan = SetOperatorExec("someID", dummyDispatcher, + val execPlan = SetOperatorExec(QueryContext(), dummyDispatcher, Array(dummyPlan), new Array[ExecPlan](1), BinaryOperator.LAND, @@ -393,7 +391,7 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures it("should return Lhs when LAND is done with vector having no labels with on dummy") { val sampleRhsShuffled = scala.util.Random.shuffle(sampleNoKey.toList) - val execPlan = SetOperatorExec("someID", dummyDispatcher, + val execPlan = SetOperatorExec(QueryContext(), dummyDispatcher, Array(dummyPlan), new Array[ExecPlan](1), BinaryOperator.LAND, @@ -415,7 +413,7 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures it("should return Lhs when LAND is done with vector having no labels and ignoring is used om Lhs labels") { val sampleRhsShuffled = scala.util.Random.shuffle(sampleNoKey.toList) - val execPlan = SetOperatorExec("someID", dummyDispatcher, + val execPlan = SetOperatorExec(QueryContext(), dummyDispatcher, Array(dummyPlan), new Array[ExecPlan](1), BinaryOperator.LAND, @@ -437,7 +435,7 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures it("should join many-to-many with or") { val sampleRhsShuffled = scala.util.Random.shuffle(sampleProduction.toList) - val execPlan = SetOperatorExec("someID", dummyDispatcher, + val execPlan = SetOperatorExec(QueryContext(), dummyDispatcher, Array(dummyPlan), new Array[ExecPlan](1), BinaryOperator.LOR, @@ -465,7 +463,7 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures val sampleRhsShuffled = scala.util.Random.shuffle(sampleInstance1.toList) - val execPlan = SetOperatorExec("someID", dummyDispatcher, + val execPlan = SetOperatorExec(QueryContext(), dummyDispatcher, Array(dummyPlan), new Array[ExecPlan](1), BinaryOperator.LOR, @@ -525,7 +523,7 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures "the instance label when performing LOR on instance") { // Query (http_requests{group="canary"} + 1) or on(instance) (http_requests or vector_matching_a) - val execPlan1 = SetOperatorExec("someID", dummyDispatcher, + val execPlan1 = SetOperatorExec(QueryContext(), dummyDispatcher, Array(dummyPlan), new Array[ExecPlan](1), BinaryOperator.LOR, @@ -540,7 +538,7 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures val result1 = execPlan1.compose(Observable.fromIterable(Seq((rhs1, 1), (lhs1, 0))), resSchemaTask, queryConfig) .toListL.runAsync.futureValue - val execPlan2 = SetOperatorExec("someID", dummyDispatcher, + val execPlan2 = SetOperatorExec(QueryContext(), dummyDispatcher, Array(dummyPlan), new Array[ExecPlan](1), BinaryOperator.LOR, @@ -596,7 +594,7 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures "the instance label when performing LOR with ignoring on l, group and job") { // Query (http_requests{group="canary"} + 1) or ignoring(l, group, job) (http_requests or vector_matching_a) - val execPlan1 = SetOperatorExec("someID", dummyDispatcher, + val execPlan1 = SetOperatorExec(QueryContext(), dummyDispatcher, Array(dummyPlan), new Array[ExecPlan](1), BinaryOperator.LOR, @@ -611,7 +609,7 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures val result1 = execPlan1.compose(Observable.fromIterable(Seq((rhs1, 1), (lhs1, 0))), resSchemaTask, queryConfig) .toListL.runAsync.futureValue - val execPlan2 = SetOperatorExec("someID", dummyDispatcher, + val execPlan2 = SetOperatorExec(QueryContext(), dummyDispatcher, Array(dummyPlan), new Array[ExecPlan](1), BinaryOperator.LOR, @@ -666,7 +664,7 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures it("should join many-to-many with unless") { val sampleRhsShuffled = scala.util.Random.shuffle(sampleInstance0.toList) - val execPlan = SetOperatorExec("someID", dummyDispatcher, + val execPlan = SetOperatorExec(QueryContext(), dummyDispatcher, Array(dummyPlan), new Array[ExecPlan](1), BinaryOperator.LUnless, @@ -700,7 +698,7 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures it("should not return any results when rhs has same vector on joining with on labels with LUnless") { val sampleRhsShuffled = scala.util.Random.shuffle(sampleInstance0.toList) - val execPlan = SetOperatorExec("someID", dummyDispatcher, + val execPlan = SetOperatorExec(QueryContext(), dummyDispatcher, Array(dummyPlan), new Array[ExecPlan](1), BinaryOperator.LUnless, @@ -731,7 +729,7 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures it("LUnless should return lhs samples which are not present in rhs and where on labels are not equal") { val sampleRhsShuffled = scala.util.Random.shuffle(sampleInstance0.toList) - val execPlan = SetOperatorExec("someID", dummyDispatcher, + val execPlan = SetOperatorExec(QueryContext(), dummyDispatcher, Array(dummyPlan), new Array[ExecPlan](1), BinaryOperator.LUnless, @@ -766,7 +764,7 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures it("should not return any results when rhs has same vector on joining without ignoring labels with LUnless") { val sampleRhsShuffled = scala.util.Random.shuffle(sampleInstance0.toList) - val execPlan = SetOperatorExec("someID", dummyDispatcher, + val execPlan = SetOperatorExec(QueryContext(), dummyDispatcher, Array(dummyPlan), new Array[ExecPlan](1), BinaryOperator.LUnless, @@ -798,7 +796,7 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures "ignoring labels are not equal") { val sampleRhsShuffled = scala.util.Random.shuffle(sampleInstance0.toList) - val execPlan = SetOperatorExec("someID", dummyDispatcher, + val execPlan = SetOperatorExec(QueryContext(), dummyDispatcher, Array(dummyPlan), new Array[ExecPlan](1), BinaryOperator.LUnless, diff --git a/query/src/test/scala/filodb/query/exec/HistToPromSeriesMapperSpec.scala b/query/src/test/scala/filodb/query/exec/HistToPromSeriesMapperSpec.scala index b961bd1bce..6ea08e747a 100644 --- a/query/src/test/scala/filodb/query/exec/HistToPromSeriesMapperSpec.scala +++ b/query/src/test/scala/filodb/query/exec/HistToPromSeriesMapperSpec.scala @@ -24,7 +24,7 @@ class HistToPromSeriesMapperSpec extends FunSpec with Matchers with ScalaFutures val eightBTimes = eightBucketData.map(_(0).asInstanceOf[Long]) val eightBHists = eightBucketData.map(_(3).asInstanceOf[HistogramWithBuckets]) val rows = eightBTimes.zip(eightBHists).map { case (t, h) => new TransientHistRow(t, h) } - val sourceSchema = new ResultSchema(Seq(ColumnInfo("timestamp", ColumnType.LongColumn), + val sourceSchema = new ResultSchema(Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), ColumnInfo("value", ColumnType.HistogramColumn)), 1) it("should convert single schema histogram to appropriate Prom bucket time series") { @@ -33,7 +33,7 @@ class HistToPromSeriesMapperSpec extends FunSpec with Matchers with ScalaFutures val mapper = HistToPromSeriesMapper(MMD.histDataset.schema.partition) val sourceObs = Observable.now(rv) - mapper.schema(sourceSchema).columns shouldEqual Seq(ColumnInfo("timestamp", ColumnType.LongColumn), + mapper.schema(sourceSchema).columns shouldEqual Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), ColumnInfo("value", ColumnType.DoubleColumn)) val destObs = mapper.apply(sourceObs, queryConfig, 1000, sourceSchema, Nil) @@ -68,7 +68,7 @@ class HistToPromSeriesMapperSpec extends FunSpec with Matchers with ScalaFutures val mapper = HistToPromSeriesMapper(MMD.histDataset.schema.partition) val sourceObs = Observable.now(rv) - mapper.schema(sourceSchema).columns shouldEqual Seq(ColumnInfo("timestamp", ColumnType.LongColumn), + mapper.schema(sourceSchema).columns shouldEqual Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), ColumnInfo("value", ColumnType.DoubleColumn)) val destObs = mapper.apply(sourceObs, queryConfig, 1000, sourceSchema, Nil) diff --git a/query/src/test/scala/filodb/query/exec/HistogramQuantileMapperSpec.scala b/query/src/test/scala/filodb/query/exec/HistogramQuantileMapperSpec.scala index 9a541cae71..74a52b4ed1 100644 --- a/query/src/test/scala/filodb/query/exec/HistogramQuantileMapperSpec.scala +++ b/query/src/test/scala/filodb/query/exec/HistogramQuantileMapperSpec.scala @@ -58,7 +58,7 @@ class HistogramQuantileMapperSpec extends FunSpec with Matchers with ScalaFuture val hqMapper = HistogramQuantileMapper(Seq(StaticFuncArgs(q, rangeParams))) val result = hqMapper.apply(Observable.fromIterable(histRvs), queryConfig, 10, - new ResultSchema(Seq(ColumnInfo("timestamp", ColumnType.LongColumn), + new ResultSchema(Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), ColumnInfo("value", ColumnType.DoubleColumn)), 1), Nil) .toListL.runAsync.futureValue for { i <- expectedResult.indices } { diff --git a/query/src/test/scala/filodb/query/exec/InProcessPlanDispatcherSpec.scala b/query/src/test/scala/filodb/query/exec/InProcessPlanDispatcherSpec.scala index 0df68d0573..cab2054b0d 100644 --- a/query/src/test/scala/filodb/query/exec/InProcessPlanDispatcherSpec.scala +++ b/query/src/test/scala/filodb/query/exec/InProcessPlanDispatcherSpec.scala @@ -16,7 +16,7 @@ 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} +import filodb.core.query.{ColumnFilter, Filter, QueryContext} import filodb.core.store.{AllChunkScan, InMemoryMetaStore, NullColumnStore} import filodb.memory.MemFactory import filodb.memory.format.{SeqRowReader, ZeroCopyUTF8String} @@ -93,12 +93,12 @@ class InProcessPlanDispatcherSpec extends FunSpec with Matchers with ScalaFuture val dummyDispatcher = DummyDispatcher(memStore, queryConfig) - val execPlan1 = MultiSchemaPartitionsExec("someQueryId", now, numRawSamples, dummyDispatcher, - timeseriesDataset.ref, 0, filters, AllChunkScan) - val execPlan2 = MultiSchemaPartitionsExec("someQueryId", now, numRawSamples, dummyDispatcher, - timeseriesDataset.ref, 0, filters, AllChunkScan) + val execPlan1 = MultiSchemaPartitionsExec(QueryContext(), dummyDispatcher, timeseriesDataset.ref, + 0, filters, AllChunkScan) + val execPlan2 = MultiSchemaPartitionsExec(QueryContext(), dummyDispatcher, timeseriesDataset.ref, + 0, filters, AllChunkScan) - val sep = StitchRvsExec(queryId, dispatcher, Seq(execPlan1, execPlan2)) + val sep = StitchRvsExec(QueryContext(), dispatcher, Seq(execPlan1, execPlan2)) val result = dispatcher.dispatch(sep).runAsync.futureValue result match { @@ -121,12 +121,12 @@ class InProcessPlanDispatcherSpec extends FunSpec with Matchers with ScalaFuture val dummyDispatcher = DummyDispatcher(memStore, queryConfig) - val execPlan1 = MultiSchemaPartitionsExec("someQueryId", now, numRawSamples, dummyDispatcher, - timeseriesDataset.ref, 0, filters, AllChunkScan) - val execPlan2 = MultiSchemaPartitionsExec("someQueryId", now, numRawSamples, dummyDispatcher, - timeseriesDataset.ref, 0, emptyFilters, AllChunkScan) + val execPlan1 = MultiSchemaPartitionsExec(QueryContext(), dummyDispatcher, timeseriesDataset.ref, + 0, filters, AllChunkScan) + val execPlan2 = MultiSchemaPartitionsExec(QueryContext(), dummyDispatcher, timeseriesDataset.ref, + 0, emptyFilters, AllChunkScan) - val sep = StitchRvsExec(queryId, dispatcher, Seq(execPlan1, execPlan2)) + val sep = StitchRvsExec(QueryContext(), dispatcher, Seq(execPlan1, execPlan2)) val result = dispatcher.dispatch(sep).runAsync.futureValue result match { @@ -138,7 +138,7 @@ class InProcessPlanDispatcherSpec extends FunSpec with Matchers with ScalaFuture } // Switch the order and make sure it's OK if the first result doesn't have any data - val sep2 = StitchRvsExec(queryId, dispatcher, Seq(execPlan2, execPlan1)) + val sep2 = StitchRvsExec(QueryContext(), dispatcher, Seq(execPlan2, execPlan1)) val result2 = dispatcher.dispatch(sep2).runAsync.futureValue result2 match { @@ -150,7 +150,7 @@ class InProcessPlanDispatcherSpec extends FunSpec with Matchers with ScalaFuture } // Two children none of which returns data - val sep3 = StitchRvsExec(queryId, dispatcher, Seq(execPlan2, execPlan2)) + val sep3 = StitchRvsExec(QueryContext(), dispatcher, Seq(execPlan2, execPlan2)) val result3 = dispatcher.dispatch(sep3).runAsync.futureValue result3 match { @@ -165,8 +165,7 @@ class InProcessPlanDispatcherSpec extends FunSpec with Matchers with ScalaFuture case class DummyDispatcher(memStore: TimeSeriesMemStore, queryConfig: QueryConfig) extends PlanDispatcher { // run locally withing any check. override def dispatch(plan: ExecPlan) - (implicit sched: Scheduler, - timeout: FiniteDuration): Task[QueryResponse] = { + (implicit sched: Scheduler): Task[QueryResponse] = { plan.execute(memStore, queryConfig) } } \ No newline at end of file diff --git a/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala b/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala index 07bdd0d1b9..eaf85acafa 100644 --- a/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala @@ -11,12 +11,12 @@ import org.scalatest.{BeforeAndAfterAll, FunSpec, Matchers} import org.scalatest.concurrent.ScalaFutures import org.scalatest.time.{Millis, Seconds, Span} -import filodb.core.{query, TestData} +import filodb.core.{TestData, query} import filodb.core.MetricsTestData._ import filodb.core.binaryrecord2.BinaryRecordRowReader import filodb.core.memstore.{FixedMaxPartitionsEvictionPolicy, SomeData, TimeSeriesMemStore} import filodb.core.metadata.Schemas -import filodb.core.query.{ColumnFilter, Filter, SerializedRangeVector} +import filodb.core.query.{ColumnFilter, Filter, QueryContext, SerializedRangeVector} import filodb.core.store.{InMemoryMetaStore, NullColumnStore} import filodb.memory.format.{SeqRowReader, ZeroCopyUTF8String} import filodb.query._ @@ -75,8 +75,7 @@ class MetadataExecSpec extends FunSpec with Matchers with ScalaFutures with Befo val dummyDispatcher = new PlanDispatcher { override def dispatch(plan: ExecPlan) - (implicit sched: Scheduler, - timeout: FiniteDuration): Task[QueryResponse] = ??? + (implicit sched: Scheduler): Task[QueryResponse] = ??? } it ("should read the job names from timeseriesindex matching the columnfilters") { @@ -84,7 +83,7 @@ class MetadataExecSpec extends FunSpec with Matchers with ScalaFutures with Befo val filters = Seq (ColumnFilter("_metric_", Filter.Equals("http_req_total".utf8)), ColumnFilter("job", Filter.Equals("myCoolService".utf8))) - val execPlan = LabelValuesExec("someQueryId", now, limit, dummyDispatcher, + val execPlan = LabelValuesExec(QueryContext(), dummyDispatcher, timeseriesDataset.ref, 0, filters, Seq("job"), 10) val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue @@ -105,7 +104,7 @@ class MetadataExecSpec extends FunSpec with Matchers with ScalaFutures with Befo val filters = Seq (ColumnFilter("__name__", Filter.Equals("http_req_total1".utf8)), ColumnFilter("job", Filter.Equals("myCoolService".utf8))) - val execPlan = PartKeysExec("someQueryId", now, limit, dummyDispatcher, + val execPlan = PartKeysExec(QueryContext(), dummyDispatcher, timeseriesDataset.ref, 0, Schemas.promCounter.partition, filters, now-5000, now) val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue @@ -119,7 +118,7 @@ class MetadataExecSpec extends FunSpec with Matchers with ScalaFutures with Befo import ZeroCopyUTF8String._ val filters = Seq (ColumnFilter("job", Filter.Equals("myCoolService".utf8))) - val execPlan = PartKeysExec("someQueryId", now, limit, dummyDispatcher, + val execPlan = PartKeysExec(QueryContext(), dummyDispatcher, timeseriesDataset.ref, 0, Schemas.promCounter.partition, filters, now-5000, now) val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue @@ -139,7 +138,7 @@ class MetadataExecSpec extends FunSpec with Matchers with ScalaFutures with Befo val filters = Seq (ColumnFilter("job", Filter.Equals("myCoolService".utf8))) //Reducing limit results in truncated metadata response - val execPlan = PartKeysExec("someQueryId", now, limit - 1, dummyDispatcher, + val execPlan = PartKeysExec(QueryContext(sampleLimit = limit-1), dummyDispatcher, timeseriesDataset.ref, 0, Schemas.promCounter.partition, filters, now-5000, now) val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue diff --git a/query/src/test/scala/filodb/query/exec/MultiSchemaPartitionsExecSpec.scala b/query/src/test/scala/filodb/query/exec/MultiSchemaPartitionsExecSpec.scala index 37b410055a..bc31a90f90 100644 --- a/query/src/test/scala/filodb/query/exec/MultiSchemaPartitionsExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/MultiSchemaPartitionsExecSpec.scala @@ -7,7 +7,7 @@ import monix.execution.Scheduler.Implicits.global import org.scalatest.{BeforeAndAfterAll, FunSpec, Matchers} import org.scalatest.concurrent.ScalaFutures import org.scalatest.time.{Millis, Seconds, Span} -import filodb.core.{DatasetRef, TestData, Types} +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 @@ -18,17 +18,16 @@ 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 { override def dispatch(plan: ExecPlan) - (implicit sched: Scheduler, - timeout: FiniteDuration): Task[QueryResponse] = ??? + (implicit sched: Scheduler): Task[QueryResponse] = ??? } val dsRef = DatasetRef("raw-metrics") - val dummyPlan = MultiSchemaPartitionsExec("someQueryId", System.currentTimeMillis, 100, dummyDispatcher, - dsRef, 0, Nil, AllChunkScan) + val dummyPlan = MultiSchemaPartitionsExec(QueryContext(), dummyDispatcher, dsRef, 0, Nil, AllChunkScan) val builder = new RecordBuilder(MemFactory.onHeapFactory) } @@ -99,7 +98,7 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu import ZeroCopyUTF8String._ val filters = Seq (ColumnFilter("_metric_", Filter.Equals("http_req_total".utf8)), ColumnFilter("job", Filter.Equals("myCoolService".utf8))) - val execPlan = MultiSchemaPartitionsExec("someQueryId", now, numRawSamples, dummyDispatcher, + val execPlan = MultiSchemaPartitionsExec(QueryContext(), dummyDispatcher, dsRef, 0, filters, AllChunkScan) val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue @@ -120,7 +119,7 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu val startTime = now - numRawSamples * reportingInterval val endTime = now - (numRawSamples-10) * reportingInterval - val execPlan = MultiSchemaPartitionsExec("someQueryId", now, numRawSamples, dummyDispatcher, + val execPlan = MultiSchemaPartitionsExec(QueryContext(), dummyDispatcher, dsRef, 0, filters, TimeRangeChunkScan(startTime, endTime)) val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue @@ -136,7 +135,7 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu import ZeroCopyUTF8String._ val filters = Seq (ColumnFilter("_metric_", Filter.Equals("not_a_metric!".utf8)), ColumnFilter("job", Filter.Equals("myCoolService".utf8))) - val execPlan = MultiSchemaPartitionsExec("someQueryId", now, numRawSamples, dummyDispatcher, + val execPlan = MultiSchemaPartitionsExec(QueryContext(), dummyDispatcher, dsRef, 0, filters, AllChunkScan) val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue @@ -150,12 +149,12 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu val filters = Seq(ColumnFilter("series", Filter.Equals("Series 1".utf8))) // read from an interval of 100000ms, resulting in 11 samples, count column - val execPlan = MultiSchemaPartitionsExec("someQueryId", now, numRawSamples, dummyDispatcher, MMD.dataset1.ref, 0, - filters, TimeRangeChunkScan(100000L, 150000L), colName=Some("count")) + 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 result = resp.asInstanceOf[QueryResult] - result.resultSchema.columns.map(_.colType) shouldEqual Seq(LongColumn, LongColumn) + result.resultSchema.columns.map(_.colType) shouldEqual Seq(TimestampColumn, LongColumn) result.result.size shouldEqual 1 val dataRead = result.result(0).rows.map(r=>(r.getLong(0), r.getLong(1))).toList dataRead shouldEqual mmdTuples.filter(_(5) == "Series 1").map(r => (r(0), r(4))).take(5) @@ -166,7 +165,7 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu val filters = Seq(ColumnFilter("dc", Filter.Equals("0".utf8)), ColumnFilter("_metric_", Filter.Equals("request-latency".utf8))) - val execPlan = MultiSchemaPartitionsExec("id1", now, numRawSamples, dummyDispatcher, dsRef, 0, + val execPlan = MultiSchemaPartitionsExec(QueryContext(), dummyDispatcher, dsRef, 0, filters, TimeRangeChunkScan(100000L, 150000L), colName=Some("h")) val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue @@ -182,12 +181,12 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu import ZeroCopyUTF8String._ val filters = Seq (ColumnFilter("_metric_", Filter.Equals("http_req_total".utf8)), ColumnFilter("job", Filter.Equals("myCoolService".utf8))) - val execPlan = MultiSchemaPartitionsExec("someQueryId", now, numRawSamples, dummyDispatcher, + val execPlan = MultiSchemaPartitionsExec(QueryContext(), dummyDispatcher, dsRef, 0, filters, AllChunkScan) val start = now - numRawSamples * reportingInterval - 100 // reduce by 100 to not coincide with reporting intervals val step = 20000 val end = now - (numRawSamples-100) * reportingInterval - execPlan.addRangeVectorTransformer(new PeriodicSamplesMapper(start, step, end, None, None, Nil)) + execPlan.addRangeVectorTransformer(new PeriodicSamplesMapper(start, step, end, None, None, QueryContext())) val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue val result = resp.asInstanceOf[QueryResult] @@ -216,18 +215,18 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu it("should read periodic samples from Long column") { import ZeroCopyUTF8String._ val filters = Seq(ColumnFilter("series", Filter.Equals("Series 1".utf8))) - val execPlan = MultiSchemaPartitionsExec("someQueryId", now, numRawSamples, dummyDispatcher, MMD.dataset1.ref, 0, - filters, AllChunkScan, colName=Some("count")) + val execPlan = MultiSchemaPartitionsExec(QueryContext(), dummyDispatcher, MMD.dataset1.ref, 0, + filters, AllChunkScan, colName = Some("count")) // Raw data like 101000, 111000, .... val start = 105000L val step = 20000L val end = 185000L - execPlan.addRangeVectorTransformer(new PeriodicSamplesMapper(start, step, end, None, None, Nil)) + execPlan.addRangeVectorTransformer(new PeriodicSamplesMapper(start, step, end, None, None, QueryContext())) val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue val result = resp.asInstanceOf[QueryResult] - result.resultSchema.columns.map(_.colType) shouldEqual Seq(LongColumn, DoubleColumn) + result.resultSchema.columns.map(_.colType) shouldEqual Seq(TimestampColumn, DoubleColumn) result.result.size shouldEqual 1 val dataRead = result.result(0).rows.map(r=>(r.getLong(0), r.getDouble(1))).toList dataRead.map(_._1) shouldEqual (start to end by step) @@ -238,13 +237,13 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu import ZeroCopyUTF8String._ val filters = Seq(ColumnFilter("dc", Filter.Equals("0".utf8)), ColumnFilter("_metric_", Filter.Equals("request-latency".utf8))) - val execPlan = MultiSchemaPartitionsExec("id1", now, numRawSamples, dummyDispatcher, dsRef, 0, + val execPlan = MultiSchemaPartitionsExec(QueryContext(), dummyDispatcher, dsRef, 0, filters, AllChunkScan) // should default to h column val start = 105000L val step = 20000L val end = 185000L - execPlan.addRangeVectorTransformer(new PeriodicSamplesMapper(start, step, end, None, None, Nil)) + execPlan.addRangeVectorTransformer(new PeriodicSamplesMapper(start, step, end, None, None, QueryContext())) val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue val result = resp.asInstanceOf[QueryResult] @@ -261,7 +260,7 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu import ZeroCopyUTF8String._ val filters = Seq(ColumnFilter("dc", Filter.Equals("0".utf8)), ColumnFilter("_metric_", Filter.Equals("request-latency".utf8))) - val execPlan = MultiSchemaPartitionsExec("id1", now, numRawSamples, dummyDispatcher, dsRef, 0, + val execPlan = MultiSchemaPartitionsExec(QueryContext(), dummyDispatcher, dsRef, 0, filters, AllChunkScan) // should default to h column val start = 105000L @@ -271,7 +270,7 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu InstantFunctionId.HistogramBucket, Seq(StaticFuncArgs(16.0, RangeParams(0,0,0))))) execPlan.addRangeVectorTransformer(new PeriodicSamplesMapper(start, step, end, Some(300 * 1000), // [5m] - Some(InternalRangeFunction.Rate), rawSource = false)) + Some(InternalRangeFunction.Rate), QueryContext(), rawSource = false)) val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue val result = resp.asInstanceOf[QueryResult] @@ -287,7 +286,7 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu } it("should return SchemaMismatch QueryError if multiple schemas found in query") { - val execPlan = MultiSchemaPartitionsExec("someQueryId", now, numRawSamples, dummyDispatcher, + val execPlan = MultiSchemaPartitionsExec(QueryContext(), dummyDispatcher, dsRef, 0, Nil, AllChunkScan) val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue @@ -296,8 +295,8 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu } it("should select only specified schema if schema option given even if multiple schemas match") { - val execPlan = MultiSchemaPartitionsExec("someQueryId", now, numRawSamples, dummyDispatcher, - dsRef, 0, Nil, AllChunkScan, schema=Some("prom-counter")) + val execPlan = MultiSchemaPartitionsExec(QueryContext(), dummyDispatcher, + dsRef, 0, Nil, AllChunkScan, schema = Some("prom-counter")) val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue val result = resp.asInstanceOf[QueryResult] @@ -310,14 +309,14 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu // A lower-level (below coordinator) end to end histogram with max ingestion and querying test it("should sum Histogram records with max correctly") { val filters = Seq(ColumnFilter("dc", Filter.Equals("0".utf8))) - val execPlan = MultiSchemaPartitionsExec("hMax", now, numRawSamples, dummyDispatcher, MMD.histMaxDS.ref, 0, - filters, AllChunkScan, colName=Some("h")) + val execPlan = MultiSchemaPartitionsExec(QueryContext(), dummyDispatcher, MMD.histMaxDS.ref, 0, + filters, AllChunkScan, colName = Some("h")) val start = 105000L val step = 20000L val end = 185000L execPlan.addRangeVectorTransformer(new PeriodicSamplesMapper(start, step, end, Some(300 * 1000), // [5m] - Some(InternalRangeFunction.SumOverTime), Nil)) + Some(InternalRangeFunction.SumOverTime), QueryContext())) execPlan.addRangeVectorTransformer(AggregateMapReduce(AggregationOperator.Sum, Nil, Nil, Nil)) val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue @@ -356,13 +355,13 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu it("should extract Histogram with max using Last/None function correctly") { val filters = Seq(ColumnFilter("dc", Filter.Equals("0".utf8))) - val execPlan = MultiSchemaPartitionsExec("hMax", now, numRawSamples, dummyDispatcher, MMD.histMaxDS.ref, 0, + val execPlan = MultiSchemaPartitionsExec(QueryContext(), dummyDispatcher, MMD.histMaxDS.ref, 0, filters, AllChunkScan) // should default to h column val start = 105000L val step = 20000L val end = 185000L - execPlan.addRangeVectorTransformer(new PeriodicSamplesMapper(start, step, end, None, None, Nil)) + execPlan.addRangeVectorTransformer(new PeriodicSamplesMapper(start, step, end, None, None, QueryContext())) val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue val result = resp.asInstanceOf[QueryResult] @@ -386,7 +385,7 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu val filters = Seq (ColumnFilter("_metric_", Filter.Equals("http_req_total".utf8)), ColumnFilter("job", Filter.Equals("myCoolService".utf8))) // TODO: SelectChunkInfos should not require a raw schema - val execPlan = SelectChunkInfosExec("someQueryId", now, numRawSamples, dummyDispatcher, + val execPlan = SelectChunkInfosExec(QueryContext(), dummyDispatcher, dsRef, 0, filters, AllChunkScan, colName = Some("timestamp")) val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue info(s"resp = $resp") @@ -417,7 +416,7 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu // Query returns n ("numRawSamples") samples - Applying Limit (n-1) to fail the query execution // with ResponseTooLargeException - val execPlan = MultiSchemaPartitionsExec("someQueryId", now, numRawSamples - 1, dummyDispatcher, + val execPlan = MultiSchemaPartitionsExec(QueryContext(sampleLimit = 999), dummyDispatcher, dsRef, 0, filters, AllChunkScan) val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue @@ -425,5 +424,20 @@ class MultiSchemaPartitionsExecSpec extends FunSpec with Matchers with ScalaFutu result.t.getClass shouldEqual classOf[BadQueryException] } + it("should throw QueryTimeoutException when query processing time is greater than timeout") { + import ZeroCopyUTF8String._ + val filters = Seq (ColumnFilter("_metric_", Filter.Equals("not_a_metric!".utf8)), + ColumnFilter("job", Filter.Equals("myCoolService".utf8))) + val execPlan = MultiSchemaPartitionsExec(QueryContext(submitTime = System.currentTimeMillis() - 180000), + dummyDispatcher, dsRef, 0, filters, AllChunkScan) + + val thrown = intercept[TestFailedException] { + val resp = execPlan.execute(memStore, queryConfig).runAsync.futureValue + val result = resp.asInstanceOf[QueryResult] + } + thrown.getCause.getClass shouldEqual classOf[QueryTimeoutException] + thrown.getCause.getMessage shouldEqual "Query timeout in filodb.query.exec.MultiSchemaPartitionsExec after 180 seconds" + } + } diff --git a/query/src/test/scala/filodb/query/exec/PeriodicSamplesMapperSpec.scala b/query/src/test/scala/filodb/query/exec/PeriodicSamplesMapperSpec.scala index ff3c5fddaf..fec15988a8 100644 --- a/query/src/test/scala/filodb/query/exec/PeriodicSamplesMapperSpec.scala +++ b/query/src/test/scala/filodb/query/exec/PeriodicSamplesMapperSpec.scala @@ -30,7 +30,7 @@ class PeriodicSamplesMapperSpec extends FunSpec with Matchers with ScalaFutures 400000L -> 200d, 500000L -> 200d ) - val periodicSamplesVectorFnMapper = exec.PeriodicSamplesMapper(100000L, 100000, 600000L, None, None) + val periodicSamplesVectorFnMapper = exec.PeriodicSamplesMapper(100000L, 100000, 600000L, None, None, QueryContext()) val resultObs = periodicSamplesVectorFnMapper(Observable.fromIterable(Seq(rv)), queryConfig, 1000, resultSchema, Nil) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map @@ -52,7 +52,7 @@ class PeriodicSamplesMapperSpec extends FunSpec with Matchers with ScalaFutures 500100L -> 200d ) - val periodicSamplesVectorFnMapper = exec.PeriodicSamplesMapper(100000L, 100000, 600000L, None, None, Nil, Some(100)) + val periodicSamplesVectorFnMapper = exec.PeriodicSamplesMapper(100000L, 100000, 600000L, None, None, QueryContext(), Nil, Some(100)) val resultObs = periodicSamplesVectorFnMapper(Observable.fromIterable(Seq(rv)), queryConfig, 1000, resultSchema, Nil) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map diff --git a/query/src/test/scala/filodb/query/exec/PromQlExecSpec.scala b/query/src/test/scala/filodb/query/exec/PromQlExecSpec.scala index d70b425e56..955711d834 100644 --- a/query/src/test/scala/filodb/query/exec/PromQlExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/PromQlExecSpec.scala @@ -1,17 +1,15 @@ package filodb.query.exec -import scala.concurrent.duration.FiniteDuration - import com.typesafe.config.ConfigFactory import kamon.Kamon import monix.eval.Task import monix.execution.Scheduler import org.scalatest.{FunSpec, Matchers} import org.scalatest.concurrent.ScalaFutures - import filodb.core.metadata.{Dataset, DatasetOptions} +import filodb.core.query.{PromQlQueryParams, QueryContext} import filodb.query -import filodb.query.{Data, PromQlInvocationParams, QueryResponse, QueryResult, Sampl} +import filodb.query.{Data, QueryResponse, QueryResult, Sampl} class PromQlExecSpec extends FunSpec with Matchers with ScalaFutures { val timeseriesDataset = Dataset.make("timeseries", @@ -21,13 +19,14 @@ class PromQlExecSpec extends FunSpec with Matchers with ScalaFutures { val dummyDispatcher = new PlanDispatcher { override def dispatch(plan: ExecPlan) - (implicit sched: Scheduler, - timeout: FiniteDuration): Task[QueryResponse] = ??? + (implicit sched: Scheduler): Task[QueryResponse] = ??? } + val queryContext = QueryContext() + val params = PromQlQueryParams(ConfigFactory.empty(), "", 0, 0 , 0) it ("should convert matrix Data to QueryResponse ") { val expectedResult = List((1000000, 1.0), (2000000, 2.0), (3000000, 3.0)) - val exec = PromQlExec("test", dummyDispatcher, timeseriesDataset.ref, PromQlInvocationParams(ConfigFactory.empty(), "", 0, 0 , 0)) + val exec = PromQlExec(queryContext, dummyDispatcher, timeseriesDataset.ref, params) val result = query.Result (Map("instance" ->"inst1"), Some(Seq(Sampl(1000, 1), Sampl(2000, 2), Sampl(3000, 3))), None) val res = exec.toQueryResponse(Data("vector", Seq(result)), "id", Kamon.currentSpan()) res.isInstanceOf[QueryResult] shouldEqual true @@ -40,7 +39,7 @@ class PromQlExecSpec extends FunSpec with Matchers with ScalaFutures { it ("should convert vector Data to QueryResponse ") { val expectedResult = List((1000000, 1.0)) - val exec = PromQlExec("test", dummyDispatcher, timeseriesDataset.ref, PromQlInvocationParams(ConfigFactory.empty(), "", 0, 0 , 0)) + val exec = PromQlExec(queryContext, dummyDispatcher, timeseriesDataset.ref, params) val result = query.Result (Map("instance" ->"inst1"), None, Some(Sampl(1000, 1))) val res = exec.toQueryResponse(Data("vector", Seq(result)), "id", Kamon.currentSpan()) res.isInstanceOf[QueryResult] shouldEqual true diff --git a/query/src/test/scala/filodb/query/exec/StitchRvsExecSpec.scala b/query/src/test/scala/filodb/query/exec/StitchRvsExecSpec.scala index 4c0b21807f..9f4818bf35 100644 --- a/query/src/test/scala/filodb/query/exec/StitchRvsExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/StitchRvsExecSpec.scala @@ -1,10 +1,9 @@ package filodb.query.exec -import filodb.core.query.{ColumnInfo, ResultSchema, TransientRow} -import scala.annotation.tailrec +import filodb.core.query.{ColumnInfo, QueryContext, ResultSchema, TransientRow} +import scala.annotation.tailrec import org.scalatest.{FunSpec, Matchers} - import filodb.core.metadata.Column.ColumnType.{DoubleColumn, TimestampColumn} import filodb.memory.format.UnsafeUtils import filodb.query.QueryResult @@ -98,7 +97,7 @@ class StitchRvsExecSpec extends FunSpec with Matchers { it ("should reduce result schemas with different fixedVecLengths without error") { // null needed below since there is a require in code that prevents empty children - val exec = StitchRvsExec("someId", InProcessPlanDispatcher, Seq(UnsafeUtils.ZeroPointer.asInstanceOf[ExecPlan])) + val exec = StitchRvsExec(QueryContext(), InProcessPlanDispatcher, Seq(UnsafeUtils.ZeroPointer.asInstanceOf[ExecPlan])) val rs1 = ResultSchema(List(ColumnInfo("timestamp", TimestampColumn), ColumnInfo("value", DoubleColumn)), 1, Map(), Some(430), List(0, 1)) 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 4d0990cf09..d523f69940 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala @@ -5,9 +5,9 @@ import scala.collection.mutable.ArrayBuffer import com.typesafe.config.ConfigFactory import org.scalatest.{BeforeAndAfterAll, FunSpec, Matchers} import filodb.core.memstore.{TimeSeriesPartition, TimeSeriesPartitionSpec, WriteBufferPool} -import filodb.core.query.{RangeParams, RawDataRangeVector, TransientHistMaxRow, TransientHistRow, TransientRow} +import filodb.core.query.{QueryContext, RangeParams, RawDataRangeVector, TransientHistMaxRow, TransientHistRow, TransientRow} import filodb.core.store.AllChunkScan -import filodb.core.{MetricsTestData, TestData, MachineMetricsData => MMD} +import filodb.core.{MetricsTestData, QueryTimeoutException, TestData, MachineMetricsData => MMD} import filodb.memory._ import filodb.memory.format.{TupleRowReader, vectors => bv} import filodb.query.QueryConfig @@ -34,10 +34,100 @@ trait RawDataWindowingSpec extends FunSpec with Matchers with BeforeAndAfterAll blockStore.releaseBlocks() } + // Below function computes summation of the squared values of the sequence + // Limitation : If the sequence contains combination of NaN and not NaN values, the computed summed squared value will be NaN def sumSquares(nn: Seq[Double]): Double = nn.map(n => n*n).sum.toDouble + + // Below function computes average of the sequence + // Limitation : If the sequence contains combination of NaN and not NaN values, the computed average will be NaN def avg(nn: Seq[Double]): Double = nn.sum.toDouble / nn.length + + // Below function computes stdVar of the sequence + // Limitation : If the sequence contains combination ofNaN and not NaN values, the standard variation will be NaN def stdVar(nn: Seq[Double]): Double = sumSquares(nn)/nn.length - avg(nn)*avg(nn) + // Below function count no. of non-nan values in the sequence + def countNonNaN(arr: Seq[Double]): Int = { + var count = 0 + var currPos = 0 + val length = arr.length + if (length > 0) { + while (currPos < length) { + if(!arr(currPos).isNaN) { + count += 1 + } + currPos += 1 + } + } + count + } + + // Below function computes sum of the sequence when the sequence contains combination of NaN and not NaN values + // sum = summation of non-nan values. + def sumWithNaN(arr: Seq[Double]): Double = { + var currPos = 0 + var length = arr.length + var sum = Double.NaN + var count = 0 + if (length > 0) { + while (currPos < length) { + if(!arr(currPos).isNaN) { + if (sum.isNaN) sum = 0d + sum += arr(currPos) + count +=1 + } + currPos += 1 + } + } + sum + } + + // Below function computes average of the sequence when the sequence contains combination of NaN and not NaN values + // avg = summation of non-nan values/no. of non-nan values. + def avgWithNaN(arr: Seq[Double]): Double = { + sumWithNaN(arr)/countNonNaN(arr) + } + + // Below function computes the summation of the squared values of the sequence when the sequence contains combination of NaN and not NaN values + // squaredSum = summation of squares of non-nan values. + def sumSquaresWithNaN(arr: Seq[Double]): Double = { + var currPos = 0 + var sumSquares = Double.NaN + val length = arr.length + if (length > 0) { + while (currPos < length) { + if(!arr(currPos).isNaN) { + if (sumSquares.isNaN) sumSquares = 0d + sumSquares += arr(currPos) * arr(currPos) + } + currPos += 1 + } + } + sumSquares + } + + + // Below function computes stdVar of the sequence when the sequence contains combination of NaN and non-NaN values + // standard variation = (summation of squares of non-nan values/no. of non-nan values) - square of avg of non-nan values + def stdVarWithNaN(arr: Seq[Double]): Double = { + val avg = avgWithNaN(arr) + (sumSquaresWithNaN(arr) / countNonNaN(arr)) - (avg * avg) + } + + // Below function computes zscore for the last Sample of the input sequence. + // zscore = (lastSampleValue - avg(sequence))/stddev(sequence) + def z_score(arr: Seq[Double]): Double = { + var zscore = Double.NaN + if (arr.length > 0) { + if (!arr.last.isNaN) { + val av = avgWithNaN(arr) + val sd = Math.sqrt(stdVarWithNaN(arr)) + zscore = (arr.last - av) / sd + } + } + zscore + } + val defaultStartTS = 100000L val pubFreq = 10000L @@ -116,7 +206,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, row) + func.asInstanceOf[ChunkedRangeFunction[TransientHistRow]], queryConfig, QueryContext(), row) } def chunkedWindowItHist(data: Seq[Seq[Any]], @@ -151,7 +241,7 @@ class AggrOverTimeFunctionsSpec extends RawDataWindowingSpec { (0 until numIterations).foreach { x => val windowSize = rand.nextInt(100) + 10 val step = rand.nextInt(75) + 5 - info(s" iteration $x windowSize=$windowSize step=$step") + info(s"iteration $x windowSize=$windowSize step=$step") val slidingIt = slidingWindowIt(data, rv, new SumOverTimeFunction(), windowSize, step) val aggregated = slidingIt.map(_.getDouble(1)).toBuffer // drop first sample because of exclusive start @@ -168,12 +258,12 @@ class AggrOverTimeFunctionsSpec extends RawDataWindowingSpec { (0 until numIterations).foreach { x => val windowSize = rand.nextInt(50) + 10 val step = rand.nextInt(50) + 5 - info(s" iteration $x windowSize=$windowSize step=$step") + info(s"iteration $x windowSize=$windowSize step=$step") val chunkedIt = chunkedWindowItHist(data, rv, new SumOverTimeChunkedFunctionH(), windowSize, step) chunkedIt.zip(data.sliding(windowSize, step).map(_.drop(1))).foreach { case (aggRow, rawDataWindow) => val aggHist = aggRow.getHistogram(1) - val sumRawHist = rawDataWindow.map(_(3).asInstanceOf[bv.MutableHistogram]) + val sumRawHist = rawDataWindow.map(_(3).asInstanceOf[bv.LongHistogram]) .foldLeft(emptyAggHist) { case (agg, h) => agg.add(h); agg } aggHist shouldEqual sumRawHist } @@ -187,13 +277,13 @@ class AggrOverTimeFunctionsSpec extends RawDataWindowingSpec { (0 until numIterations).foreach { x => val windowSize = rand.nextInt(50) + 10 val step = rand.nextInt(50) + 5 - info(s" iteration $x windowSize=$windowSize step=$step") + info(s"iteration $x windowSize=$windowSize step=$step") val row = new TransientHistMaxRow() val chunkedIt = chunkedWindowItHist(data, rv, new SumAndMaxOverTimeFuncHD(3), windowSize, step, row) chunkedIt.zip(data.sliding(windowSize, step).map(_.drop(1))).foreach { case (aggRow, rawDataWindow) => val aggHist = aggRow.getHistogram(1) - val sumRawHist = rawDataWindow.map(_(4).asInstanceOf[bv.MutableHistogram]) + val sumRawHist = rawDataWindow.map(_(4).asInstanceOf[bv.LongHistogram]) .foldLeft(emptyAggHist) { case (agg, h) => agg.add(h); agg } aggHist shouldEqual sumRawHist @@ -211,7 +301,7 @@ class AggrOverTimeFunctionsSpec extends RawDataWindowingSpec { (0 until numIterations).foreach { x => val windowSize = rand.nextInt(100) + 10 val step = rand.nextInt(75) + 5 - info(s" iteration $x windowSize=$windowSize step=$step") + info(s"iteration $x windowSize=$windowSize step=$step") val minSlidingIt = slidingWindowIt(data, rv, new MinMaxOverTimeFunction(Ordering[Double].reverse), windowSize, step) val aggregated = minSlidingIt.map(_.getDouble(1)).toBuffer @@ -241,7 +331,7 @@ class AggrOverTimeFunctionsSpec extends RawDataWindowingSpec { (0 until numIterations).foreach { x => val windowSize = rand.nextInt(100) + 10 val step = rand.nextInt(50) + 5 - info(s" iteration $x windowSize=$windowSize step=$step") + info(s"iteration $x windowSize=$windowSize step=$step") val countSliding = slidingWindowIt(data, rv, new CountOverTimeFunction(), windowSize, step) val aggregated1 = countSliding.map(_.getDouble(1)).toBuffer @@ -255,6 +345,7 @@ class AggrOverTimeFunctionsSpec extends RawDataWindowingSpec { val aggregated3 = avgSliding.map(_.getDouble(1)).toBuffer aggregated3 shouldEqual data.sliding(windowSize, step).map(a => avg(a drop 1)).toBuffer + // In sample_data2, there are no NaN's, that's why using avg function is fine val avgChunked = chunkedWindowIt(data, rv, new AvgOverTimeChunkedFunctionD(), windowSize, step) val aggregated4 = avgChunked.map(_.getDouble(1)).toBuffer aggregated4 shouldEqual data.sliding(windowSize, step).map(a => avg(a drop 1)).toBuffer @@ -273,7 +364,7 @@ class AggrOverTimeFunctionsSpec extends RawDataWindowingSpec { (0 until numIterations).foreach { x => val windowSize = rand.nextInt(100) + 10 val step = rand.nextInt(50) + 5 - info(s" iteration $x windowSize=$windowSize step=$step") + info(s"iteration $x windowSize=$windowSize step=$step") val varSlidingIt = slidingWindowIt(data, rv, new StdVarOverTimeFunction(), windowSize, step) val aggregated2 = varSlidingIt.map(_.getDouble(1)).toBuffer @@ -313,7 +404,7 @@ class AggrOverTimeFunctionsSpec extends RawDataWindowingSpec { val twoSampleData = Seq(0.0, 1.0) val threeSampleData = Seq(1.0, 0.0, 2.0) val unevenSampleData = Seq(0.0, 1.0, 4.0) - val rangeParams = RangeParams(100,20, 500) + val rangeParams = RangeParams(100, 20, 500) val quantiles = Seq(0, 0.5, 0.75, 0.8, 1, -1, 2).map(x => StaticFuncArgs(x, rangeParams)) val twoSampleDataResponses = Seq(0, 0.5, 0.75, 0.8, 1, Double.NegativeInfinity, Double.PositiveInfinity) @@ -348,7 +439,7 @@ class AggrOverTimeFunctionsSpec extends RawDataWindowingSpec { aggregatedEmpty(0) isNaN def median(s: Seq[Double]): Double = { - val (lower, upper) = s.sortWith(_<_).splitAt(s.size / 2) + val (lower, upper) = s.sortWith(_ < _).splitAt(s.size / 2) if (s.size % 2 == 0) (lower.last + upper.head) / 2.0 else upper.head } @@ -357,9 +448,9 @@ class AggrOverTimeFunctionsSpec extends RawDataWindowingSpec { (0 until numIterations).foreach { x => val windowSize = rand.nextInt(100) + 10 val step = rand.nextInt(50) + 5 - info(s" iteration $x windowSize=$windowSize step=$step") + info(s"iteration $x windowSize=$windowSize step=$step") - val minChunkedIt = chunkedWindowIt(data, rv2, new QuantileOverTimeChunkedFunctionD + val minChunkedIt = chunkedWindowIt(data, rv2, new QuantileOverTimeChunkedFunctionD (Seq(StaticFuncArgs(0.5, rangeParams))), windowSize, step) val aggregated2 = minChunkedIt.map(_.getDouble(1)).toBuffer @@ -369,13 +460,13 @@ class AggrOverTimeFunctionsSpec extends RawDataWindowingSpec { it("should correctly do changes for DoubleVectorDataReader and DeltaDeltaDataReader when window has more " + "than one chunks") { - val data1= (1 to 240).map(_.toDouble) - val data2 : Seq[Double]= Seq[Double]( 1.1, 1.5, 2.5, 3.5, 4.5, 5.5) + val data1 = (1 to 240).map(_.toDouble) + val data2: Seq[Double] = Seq[Double](1.1, 1.5, 2.5, 3.5, 4.5, 5.5) (0 until numIterations).foreach { x => val windowSize = rand.nextInt(100) + 10 val step = rand.nextInt(50) + 5 - info(s" iteration $x windowSize=$windowSize step=$step") + info(s"iteration $x windowSize=$windowSize step=$step") // Append double data and shuffle so that it becomes DoubleVectorDataReader val data = scala.util.Random.shuffle(data2 ++ data1) @@ -398,11 +489,11 @@ class AggrOverTimeFunctionsSpec extends RawDataWindowingSpec { it("should correctly calculate holt winters") { val positiveTrendData2 = Seq(15900.0, 15920.0, 15940.0, 15960.0, 15980.0, 16000.0) - val positiveTrendData3 = Seq(23850.0,23880.0,23910.0,23940.0,23970.0, 24000.0) + val positiveTrendData3 = Seq(23850.0, 23880.0, 23910.0, 23940.0, 23970.0, 24000.0) val positiveTrendData4 = Seq(31800.0, 31840.0, 31880.0, 31920.0, 31960.0, 32000.0) val negativeTrendData2 = Seq(-15900.0, -15920.0, -15940.0, -15960.0, -15980.0, -16000.0) - val params = Seq(StaticFuncArgs(0.01, RangeParams(100,20,500)), StaticFuncArgs(0.1, RangeParams(100,20,500))) + val params = Seq(StaticFuncArgs(0.01, RangeParams(100, 20, 500)), StaticFuncArgs(0.1, RangeParams(100, 20, 500))) def holt_winters(arr: Seq[Double]): Double = { val sf = 0.01 @@ -414,8 +505,8 @@ class AggrOverTimeFunctionsSpec extends RawDataWindowingSpec { if (n >= 2) { b0 = arr(1) - arr(0) for (i <- 1 until n) { - smoothedResult = sf*arr(i) + (1-sf)*(s0 + b0) - b0 = tf*(smoothedResult - s0) + (1-tf)*b0 + smoothedResult = sf * arr(i) + (1 - sf) * (s0 + b0) + b0 = tf * (smoothedResult - s0) + (1 - tf) * b0 s0 = smoothedResult } @@ -455,7 +546,7 @@ class AggrOverTimeFunctionsSpec extends RawDataWindowingSpec { (0 until numIterations).foreach { x => val windowSize = rand.nextInt(100) + 10 val step = rand.nextInt(75) + 5 - info(s" iteration $x windowSize=$windowSize step=$step") + info(s"iteration $x windowSize=$windowSize step=$step") val minChunkedIt = chunkedWindowIt(data, rv2, new HoltWintersChunkedFunctionD(params), windowSize, step) val aggregated2 = minChunkedIt.map(_.getDouble(1)).toBuffer @@ -475,7 +566,7 @@ class AggrOverTimeFunctionsSpec extends RawDataWindowingSpec { val data = (1 to 500).map(_.toDouble) val rv2 = timeValueRV(data) val duration = 50 - val params = Seq(StaticFuncArgs(50, RangeParams(100,20,500))) + val params = Seq(StaticFuncArgs(50, RangeParams(100, 20, 500))) def predict_linear(s: Seq[Double], interceptTime: Long, startTime: Long): Double = { val n = s.length.toDouble @@ -505,7 +596,7 @@ class AggrOverTimeFunctionsSpec extends RawDataWindowingSpec { val step = rand.nextInt(50) + 5 (0 until numIterations).foreach { x => val windowSize = rand.nextInt(100) + 10 - info(s" iteration $x windowSize=$windowSize step=$step") + info(s"iteration $x windowSize=$windowSize step=$step") val ChunkedIt = chunkedWindowIt(data, rv2, new PredictLinearChunkedFunctionD(params), windowSize, step) val aggregated2 = ChunkedIt.map(_.getDouble(1)).toBuffer var res = new ArrayBuffer[Double] @@ -516,32 +607,81 @@ class AggrOverTimeFunctionsSpec extends RawDataWindowingSpec { startTime += step * pubFreq endTime += step * pubFreq } - aggregated2 shouldEqual(res) + aggregated2 shouldEqual (res) } } it("it should correctly calculate zscore") { - val data = Seq(15900.0, 15920.0, 15940.0, 15960.0, 15980.0, 16000.0, 16020.0) - val data2 = Seq(-15900.0, -15920.0, -15940.0, -15960.0, -15980.0, -16000.0) - val data3 = Seq(15900.0, 15920.0, 15940.0, 15960.0, 15980.0, 16000.0, Double.NaN) - val params = Seq(StaticFuncArgs(0.01, RangeParams(100,20,500))) - - var rv = timeValueRV(data) - val chunkedIt = new ChunkedWindowIteratorD(rv, 160000, 100000, 180000, 100000, - new ZScoreChunkedFunctionD(), queryConfig) - val aggregated = chunkedIt.map(_.getDouble(1)).toBuffer - aggregated(0) shouldBe 1.5 - - var rv2 = timeValueRV(data2) - val chunkedIt2 = new ChunkedWindowIteratorD(rv2, 160000, 100000, 180000, 100000, - new ZScoreChunkedFunctionD(), queryConfig) - val aggregated2 = chunkedIt2.map(_.getDouble(1)).toBuffer - aggregated2(0) shouldBe (-1.463850109429032 +- 0.0000000001) + val data = (1 to 500).map(_.toDouble) + val rv = timeValueRV(data) - var rv3 = timeValueRV(data3) - val chunkedIt3 = new ChunkedWindowIteratorD(rv3, 160000, 100000, 180000, 100000, - new ZScoreChunkedFunctionD(), queryConfig) - val aggregated3 = chunkedIt3.map(_.getDouble(1)).toBuffer - aggregated3(0).isNaN shouldBe true + (0 until numIterations).foreach { x => + val windowSize = rand.nextInt(100) + 10 + val step = rand.nextInt(50) + 5 + info(s"iteration $x windowSize=$windowSize step=$step") + + val chunkedIt = chunkedWindowIt(data, rv, new ZScoreChunkedFunctionD(), windowSize, step) + val aggregated = chunkedIt.map(_.getDouble(1)).toBuffer + aggregated shouldEqual data.sliding(windowSize, step).map(d => z_score(d drop 1)).toBuffer + } + } + + 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), + Seq(15900.0, 15920.0, 15940.0, 15960.0, 15980.0, 16000.0, Double.NaN), + Seq(23850.0, 23880.0, 23910.0, 23940.0, 23970.0, 24000.0), + Seq(31800.0, 31840.0, 31880.0, 31920.0, 31960.0, 32000.0), + Seq(31800.0, 31840.0, 31880.0, Double.NaN, 31920.0, 31960.0, 32000.0), + Seq(Double.NaN, 31800.0, 31840.0, 31880.0, 31920.0, 31960.0, 32000.0), + Seq(Double.NaN, Double.NaN, Double.NaN, Double.NaN, Double.NaN, Double.NaN, Double.NaN), + Seq() + ) + + for (data <- test_data) { + val rv = timeValueRV(data) + + // sum_over_time + val chunkedItSumOverTime = new ChunkedWindowIteratorD(rv, 160000, 100000, 180000, 100000, new SumOverTimeChunkedFunctionD(), queryConfig) + val aggregatedSumOverTime = chunkedItSumOverTime.map(_.getDouble(1)).toBuffer + 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 aggregatedAvgOverTime = chunkedItAvgOverTime.map(_.getDouble(1)).toBuffer + 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 aggregatedStdVarOverTime = chunkedItStdVarOverTime.map(_.getDouble(1)).toBuffer + 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 aggregatedStdDevOverTime = chunkedItStdDevOverTime.map(_.getDouble(1)).toBuffer + 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 aggregatedZscore = chunkedItZscore.map(_.getDouble(1)).toBuffer + 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 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)) + 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/InstantFunctionSpec.scala b/query/src/test/scala/filodb/query/exec/rangefn/InstantFunctionSpec.scala index 5d956b1fa3..1b621619ab 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/InstantFunctionSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/InstantFunctionSpec.scala @@ -237,7 +237,7 @@ class InstantFunctionSpec extends RawDataWindowingSpec with ScalaFutures { // Calculating the quantile is quite complex... sigh val _max = row(3).asInstanceOf[Double] if ((i % 8) == 0) (_max * 0.9) else { - val _hist = row(4).asInstanceOf[bv.MutableHistogram] + val _hist = row(4).asInstanceOf[bv.LongHistogram] val rank = 0.9 * _hist.bucketValue(_hist.numBuckets - 1) val ratio = (rank - _hist.bucketValue((i-1) % 8)) / (_hist.bucketValue(i%8) - _hist.bucketValue((i-1) % 8)) _hist.bucketTop((i-1) % 8) + ratio * (_max - _hist.bucketTop((i-1) % 8)) 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 61e3e0396a..abb100917f 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/RateFunctionsSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/RateFunctionsSpec.scala @@ -5,7 +5,7 @@ import filodb.core.{MachineMetricsData, TestData} import filodb.core.memstore.{TimeSeriesPartition, WriteBufferPool} import filodb.core.metadata.Dataset import filodb.core.query.TransientRow -import filodb.memory.format.vectors.MutableHistogram +import filodb.memory.format.vectors.{LongHistogram, MutableHistogram} import filodb.query.exec.{ChunkedWindowIteratorD, ChunkedWindowIteratorH, QueueBasedWindow} import filodb.query.util.IndexedArrayQueue @@ -197,10 +197,10 @@ class RateFunctionsSpec extends RawDataWindowingSpec { val endTs = 161000L // just past 7th sample val lastTime = 160000L val headTime = 100000L - val headHist = data(0)(3).asInstanceOf[MutableHistogram] - val lastHist = data(6)(3).asInstanceOf[MutableHistogram] + val headHist = data(0)(3).asInstanceOf[LongHistogram] + val lastHist = data(6)(3).asInstanceOf[LongHistogram] val expectedRates = (0 until headHist.numBuckets).map { b => - (lastHist.bucketValue(b) - headHist.bucketValue(b)) / (lastTime - headTime) * 1000 + (lastHist.bucketValue(b).toDouble - headHist.bucketValue(b)) / (lastTime - headTime) * 1000 } val expected = MutableHistogram(MachineMetricsData.histBucketScheme, expectedRates.toArray) @@ -234,8 +234,8 @@ class RateFunctionsSpec extends RawDataWindowingSpec { val endTs = 171000L // just past 8th sample, the first dropped one val lastTime = 170000L val headTime = 100000L - val headHist = data(0)(3).asInstanceOf[MutableHistogram] - val corrHist = data(6)(3).asInstanceOf[MutableHistogram] + val headHist = data(0)(3).asInstanceOf[LongHistogram] + val corrHist = data(6)(3).asInstanceOf[LongHistogram] val lastHist = headHist.copy // 8th sample == first sample + correction lastHist.add(corrHist) val expectedRates = (0 until headHist.numBuckets).map { b => 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 5315ebee8f..244f08f46a 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/ScalarFunctionSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/ScalarFunctionSpec.scala @@ -6,13 +6,11 @@ import com.typesafe.config.{Config, ConfigFactory} import filodb.core.MetricsTestData import filodb.core.memstore.{FixedMaxPartitionsEvictionPolicy, TimeSeriesMemStore} import filodb.core.metadata.{Dataset, DatasetOptions} -import filodb.core.query.{CustomRangeVectorKey, HourScalar, RangeParams, RangeVector, RangeVectorKey, ResultSchema, ScalarFixedDouble, ScalarVaryingDouble, TimeScalar, TransientRow} +import filodb.core.query._ import filodb.core.store.{InMemoryMetaStore, NullColumnStore} import filodb.memory.format.{RowReader, ZeroCopyUTF8String} -import filodb.query.exec.{ExecPlan, PlanDispatcher, TimeScalarGeneratorExec} -import filodb.query.{QueryConfig, QueryResponse, QueryResult, ScalarFunctionId, exec} -import monix.eval.Task -import monix.execution.Scheduler +import filodb.query.exec.TimeScalarGeneratorExec +import filodb.query.{QueryConfig, QueryResult, ScalarFunctionId, exec} import monix.execution.Scheduler.Implicits.global import monix.reactive.Observable import org.scalatest.concurrent.ScalaFutures @@ -26,11 +24,6 @@ class ScalarFunctionSpec extends FunSpec with Matchers with ScalaFutures { Seq("timestamp:ts", "value:double:detectDrops=true"), options = DatasetOptions(Seq("__name__", "job"), "__name__")).get - val dummyDispatcher = new PlanDispatcher { - override def dispatch(plan: ExecPlan) - (implicit sched: Scheduler, - timeout: FiniteDuration): Task[QueryResponse] = ??? - } val config: Config = ConfigFactory.load("application_test.conf").getConfig("filodb") val queryConfig = new QueryConfig(config.getConfig("query")) val policy = new FixedMaxPartitionsEvictionPolicy(20) @@ -125,7 +118,7 @@ class ScalarFunctionSpec extends FunSpec with Matchers with ScalaFutures { } it("should generate time scalar") { - val execPlan = TimeScalarGeneratorExec("test", timeseriesDataset.ref, RangeParams(10, 10, 100), ScalarFunctionId.Time, 0) + 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 @@ -140,7 +133,7 @@ class ScalarFunctionSpec extends FunSpec with Matchers with ScalaFutures { } } it("should generate hour scalar") { - val execPlan = TimeScalarGeneratorExec("test", timeseriesDataset.ref, RangeParams(1565627710, 10, 1565627790), ScalarFunctionId.Hour, 0) + 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 @@ -155,4 +148,19 @@ class ScalarFunctionSpec extends FunSpec with Matchers with ScalaFutures { } } } + it("should generate DayOfWeek scalar") { + 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 result = resp match { + case QueryResult(id, _, response) => { + val rv = response(0) + rv.isInstanceOf[DayOfWeekScalar] shouldEqual(true) + val res = rv.rows.map(x=>(x.getLong(0), x.getDouble(1))).toList + List((1583682900000L,0.0), (1583683000000L,0.0), (1583683100000L,0.0), (1583683200000L,0.0), + (1583683300000L,0.0), (1583683400000L,0.0)).sameElements(res) shouldEqual(true) + } + } + } } diff --git a/scripts/prom_compare_timeseries.sh b/scripts/prom_compare_timeseries.sh new file mode 100755 index 0000000000..d5bfc8fab1 --- /dev/null +++ b/scripts/prom_compare_timeseries.sh @@ -0,0 +1,30 @@ +#!/bin/bash +# +# Script for comparing Prometheus query result output JSONs for histogram buckets. +# It filters out specific instances and le buckets, and converts math output too. +# Input are two JSON files from HTTP query_range API. +# +# Download JQ: https://stedolan.github.io/jq/ +# +# set -x -e +JSON_A=$1 +JSON_B=$2 +if [ -z "$JSON_A" ]; then + echo To run: $0 json_output_1 json_output_2 + exit 0 +fi + +LES=$(cat $JSON_A | jq -r '.data.result[].metric.le' | sort | uniq) +INSTANCES=$(cat $JSON_A | jq -r '.data.result[].metric.instance' | sort | uniq) + +# Loop over instances +for INST in $INSTANCES; do + # In data.result, select the timeseries matching le and instance, then produce one line per value, + # converting the value to a number for normalization as JSON array + for le in $LES; do + # NOTE: in fish shell, use (cat .... | psub) instead of <(cat... ) + echo Comparing le=$le and inst=$INST... + diff <(cat $JSON_A | jq -c '.data.result[] | select(.metric.le == "'$le'" and .metric.instance == "'$INST'") | .values[] | [ .[0], (.[1] | tonumber) ]') \ + <(cat $JSON_B | jq -c '.data.result[] | select(.metric.le == "'$le'" and .metric.instance == "'$INST'") | .values[] | [ .[0], (.[1] | tonumber) ]') + done +done diff --git a/spark-jobs/src/main/scala/filodb/downsampler/DownsamplerContext.scala b/spark-jobs/src/main/scala/filodb/downsampler/DownsamplerContext.scala new file mode 100644 index 0000000000..a4ebb550c3 --- /dev/null +++ b/spark-jobs/src/main/scala/filodb/downsampler/DownsamplerContext.scala @@ -0,0 +1,25 @@ +package filodb.downsampler + +import com.datastax.driver.core.Session +import com.typesafe.config.Config +import com.typesafe.scalalogging.{Logger, StrictLogging} +import monix.execution.Scheduler + +import filodb.cassandra.FiloSessionProvider +import filodb.core.concurrentCache + +object DownsamplerContext extends StrictLogging { + lazy protected[downsampler] val dsLogger: Logger = logger + lazy protected[downsampler] val sessionMap = concurrentCache[Config, Session](2) + + lazy protected[downsampler] val readSched = Scheduler.io("cass-read-sched") + lazy protected[downsampler] val writeSched = Scheduler.io("cass-write-sched") + + def getOrCreateCassandraSession(config: Config): Session = { + import filodb.core._ + sessionMap.getOrElseUpdate(config, { conf => + dsLogger.info(s"Creating new Cassandra session") + FiloSessionProvider.openSession(conf) + }) + } +} diff --git a/spark-jobs/src/main/scala/filodb/downsampler/DownsamplerMain.scala b/spark-jobs/src/main/scala/filodb/downsampler/DownsamplerMain.scala deleted file mode 100644 index 312517fed0..0000000000 --- a/spark-jobs/src/main/scala/filodb/downsampler/DownsamplerMain.scala +++ /dev/null @@ -1,116 +0,0 @@ -package filodb.downsampler - -import com.typesafe.scalalogging.StrictLogging -import kamon.Kamon -import org.apache.spark.SparkConf -import org.apache.spark.sql.SparkSession - -/** - * - * Goal: Downsample all real-time data. - * Goal: Align chunks when this job is run in multiple DCs so that cross-dc repairs can be done. - * Non-Goal: Downsampling of non-real time data or data with different epoch. - * - * Strategy is to run this spark job every 6 hours at 8am, 2pm, 8pm, 2am each day. - * - * Run at 8am: We query data with ingestionTime from 10pm to 8am. - * Then query and downsample data with userTime between 12am to 6am. - * Downsampled chunk would have an ingestionTime of 12am. - * Run at 2pm: We query data with ingestionTime from 8am to 2pm. - * Then query and downsample data with userTime between 6am to 12pm. - * Downsampled chunk would have an ingestionTime of 6am. - * Run at 8pm: We query data with ingestionTime from 10am to 8pm. - * Then query and downsample data with userTime between 12pm to 6pm. - * Downsampled chunk would have an ingestionTime of 12pm. - * Run at 2am: We query data with ingestionTime from 8pm to 2am. - * Then query and downsample data with userTime between 6pm to 12am. - * Downsampled chunk would have an ingestionTime of 6pm. - * - * This will cover all data with userTime 12am to 12am. - * Since we query for a broader ingestionTime, it will include data arriving early/late by 2 hours. - * - * Important Note: The reason non-real-time data is not included in goals is because we - * want chunk alignment between DCs in downsampled data to enable cross-dc repair without chunk surgery. - * Without chunk-alignment in raw data and consistency in behavior across DCs, it would be difficult - * to achieve chunk alignment in downsampled data. Once we solve that (deferred problem), we will - * lift the constraint. - */ -object DownsamplerMain extends App { - - val d = new Downsampler - val sparkConf = new SparkConf(loadDefaults = true) - d.run(sparkConf) - d.shutdown() -} - -class Downsampler extends StrictLogging { - - import BatchDownsampler._ - import DownsamplerSettings._ - - import java.time.Instant._ - - def shutdown(): Unit = { - rawCassandraColStore.shutdown() - downsampleCassandraColStore.shutdown() - } - - // Gotcha!! Need separate function (Cannot be within body of a class) - // to create a closure for spark to serialize and move to executors. - // Otherwise, config values below were not being sent over. - // scalastyle:off method.length - def run(conf: SparkConf): Unit = { - - val spark = SparkSession.builder() - .appName("FiloDBDownsampler") - .config(conf) - .getOrCreate() - - logger.info(s"Spark Job Properties: ${spark.sparkContext.getConf.toDebugString}") - - // Use the spark property spark.filodb.downsampler.user-time-override to override the - // userTime period for which downsampling should occur. - // Generally disabled, defaults the period that just ended prior to now. - // Specified during reruns for downsampling old data - val userTimeInPeriod: Long = spark.sparkContext.getConf.get("spark.filodb.downsampler.userTimeOverride", - s"${System.currentTimeMillis() - downsampleChunkDuration}").toLong - // by default assume a time in the previous downsample period - - val userTimeStart: Long = (userTimeInPeriod / downsampleChunkDuration) * downsampleChunkDuration - val userTimeEndExclusive: Long = userTimeStart + downsampleChunkDuration - val ingestionTimeStart: Long = userTimeStart - widenIngestionTimeRangeBy.toMillis - val ingestionTimeEnd: Long = userTimeEndExclusive + widenIngestionTimeRangeBy.toMillis - - logger.info(s"This is the Downsampling driver. Starting downsampling job " + - s"rawDataset=$rawDatasetName for userTimeInPeriod=${ofEpochMilli(userTimeInPeriod)} " + - s"ingestionTimeStart=${ofEpochMilli(ingestionTimeStart)} " + - s"ingestionTimeEnd=${ofEpochMilli(ingestionTimeEnd)} " + - s"userTimeStart=${ofEpochMilli(userTimeStart)} userTimeEndExclusive=${ofEpochMilli(userTimeEndExclusive)}") - - val splits = rawCassandraColStore.getScanSplits(rawDatasetRef, splitsPerNode) - logger.info(s"Cassandra split size: ${splits.size}. We will have this many spark partitions. " + - s"Tune splitsPerNode which was $splitsPerNode if parallelism is low") - - spark.sparkContext - .makeRDD(splits) - .mapPartitions { splitIter => - import filodb.core.Iterators._ - val rawDataSource = rawCassandraColStore - val batchReadSpan = Kamon.spanBuilder("cassandra-raw-data-read-latency").start() - val batchIter = rawDataSource.getChunksByIngestionTimeRange(datasetRef = rawDatasetRef, - splits = splitIter, ingestionTimeStart = ingestionTimeStart, - ingestionTimeEnd = ingestionTimeEnd, - userTimeStart = userTimeStart, endTimeExclusive = userTimeEndExclusive, - maxChunkTime = rawDatasetIngestionConfig.storeConfig.maxChunkTime.toMillis, - batchSize = batchSize, batchTime = batchTime).toIterator() - batchReadSpan.finish() - batchIter // iterator of batches - } - .foreach { rawPartsBatch => - downsampleBatch(rawPartsBatch, userTimeStart, userTimeEndExclusive) - } - - logger.info(s"Downsampling Driver completed successfully") - } - -} diff --git a/spark-jobs/src/main/scala/filodb/downsampler/DownsamplerSettings.scala b/spark-jobs/src/main/scala/filodb/downsampler/DownsamplerSettings.scala deleted file mode 100644 index 50fbef70f0..0000000000 --- a/spark-jobs/src/main/scala/filodb/downsampler/DownsamplerSettings.scala +++ /dev/null @@ -1,69 +0,0 @@ -package filodb.downsampler - -import scala.concurrent.duration._ - -import com.typesafe.config.ConfigFactory -import com.typesafe.scalalogging.StrictLogging -import kamon.Kamon -import net.ceedubs.ficus.Ficus._ - -import filodb.coordinator.{FilodbSettings, NodeClusterActor} -import filodb.core.store.{IngestionConfig, StoreConfig} - -/** - * DownsamplerSettings is always used in the context of an object so that it need not be serialized to a spark executor - * from the spark application driver. - */ -object DownsamplerSettings extends StrictLogging { - - Kamon.init() - - val filodbSettings = new FilodbSettings(ConfigFactory.empty) - - val filodbConfig = filodbSettings.allConfig.getConfig("filodb") - - val downsamplerConfig = filodbConfig.getConfig("downsampler") - logger.info(s"Loaded following downsampler config: ${downsamplerConfig.root().render()}" ) - - val cassandraConfig = filodbConfig.getConfig("cassandra") - - val rawDatasetName = downsamplerConfig.getString("raw-dataset-name") - - logger.info(s"Parsing dataset configs at ${filodbSettings.datasetConfPaths}") - - val rawDatasetIngestionConfig = filodbSettings.streamConfigs.map { config => - IngestionConfig(config, NodeClusterActor.noOpSource.streamFactoryClass).get - }.find(_.ref.toString == rawDatasetName).get - - logger.info(s"DatasetConfig for dataset $rawDatasetName was $rawDatasetIngestionConfig") - - val rawSchemaNames = rawDatasetIngestionConfig.downsampleConfig.schemas - - val downsampleResolutions = rawDatasetIngestionConfig.downsampleConfig.resolutions - - val downsampleTtls = rawDatasetIngestionConfig.downsampleConfig.ttls.map(_.toSeconds.toInt) - - val downsampledDatasetRefs = rawDatasetIngestionConfig.downsampleConfig.downsampleDatasetRefs(rawDatasetName) - - val downsampleStoreConfig = StoreConfig(downsamplerConfig.getConfig("downsample-store-config")) - - val ttlByResolution = downsampleResolutions.zip(downsampleTtls).toMap - - val batchSize = downsamplerConfig.getInt("cass-write-batch-size") - - val batchTime = downsamplerConfig.as[FiniteDuration]("cass-write-batch-time") - - val splitsPerNode = downsamplerConfig.getInt("splits-per-node") - - val cassWriteTimeout = downsamplerConfig.as[FiniteDuration]("cassandra-write-timeout") - - val widenIngestionTimeRangeBy = downsamplerConfig.as[FiniteDuration]("widen-ingestion-time-range-by") - - val downsampleChunkDuration = downsampleStoreConfig.flushInterval.toMillis - - val whitelist = downsamplerConfig.as[Seq[Map[String, String]]]("whitelist-filters").map(_.toSeq) - - val blacklist = downsamplerConfig.as[Seq[Map[String, String]]]("blacklist-filters").map(_.toSeq) - -} - diff --git a/spark-jobs/src/main/scala/filodb/downsampler/BatchDownsampler.scala b/spark-jobs/src/main/scala/filodb/downsampler/chunk/BatchDownsampler.scala similarity index 72% rename from spark-jobs/src/main/scala/filodb/downsampler/BatchDownsampler.scala rename to spark-jobs/src/main/scala/filodb/downsampler/chunk/BatchDownsampler.scala index f1f354af5f..71a6a8a94c 100644 --- a/spark-jobs/src/main/scala/filodb/downsampler/BatchDownsampler.scala +++ b/spark-jobs/src/main/scala/filodb/downsampler/chunk/BatchDownsampler.scala @@ -1,17 +1,13 @@ -package filodb.downsampler +package filodb.downsampler.chunk import scala.collection.mutable.{ArrayBuffer, Map => MMap} import scala.concurrent.Await import scala.concurrent.duration.FiniteDuration -import com.typesafe.scalalogging.StrictLogging -import java.util import kamon.Kamon -import monix.execution.Scheduler import monix.reactive.Observable import scalaxy.loops._ -import filodb.cassandra.FiloSessionProvider import filodb.cassandra.columnstore.CassandraColumnStore import filodb.core.{DatasetRef, ErrorResponse, Instance} import filodb.core.binaryrecord2.{RecordBuilder, RecordSchema} @@ -19,6 +15,7 @@ import filodb.core.downsample._ import filodb.core.memstore.{PagedReadablePartition, TimeSeriesPartition, TimeSeriesShardStats} import filodb.core.metadata.Schemas import filodb.core.store.{AllChunkScan, ChunkSet, RawPartData, ReadablePartition} +import filodb.downsampler.DownsamplerContext import filodb.memory.{BinaryRegionLarge, MemFactory} import filodb.memory.format.UnsafeUtils import filodb.query.exec.UnknownSchemaQueryErr @@ -36,71 +33,75 @@ import filodb.query.exec.UnknownSchemaQueryErr * * All of the necessary params for the behavior are loaded from DownsampleSettings. */ -object BatchDownsampler extends StrictLogging with Instance { - - val settings = DownsamplerSettings - - val numBatchesStarted = Kamon.counter("num-batches-started").withoutTags() - val numBatchesCompleted = Kamon.counter("num-batches-completed").withoutTags() - val numBatchesFailed = Kamon.counter("num-batches-failed").withoutTags() - val numPartitionsEncountered = Kamon.counter("num-partitions-encountered").withoutTags() - val numPartitionsBlacklisted = Kamon.counter("num-partitions-blacklisted").withoutTags() - val numPartitionsCompleted = Kamon.counter("num-partitions-completed").withoutTags() - val numPartitionsFailed = Kamon.counter("num-partitions-failed").withoutTags() - val numPartitionsSkipped = Kamon.counter("num-partitions-skipped").withoutTags() - val numChunksSkipped = Kamon.counter("num-chunks-skipped").withoutTags() - val numDownsampledChunksWritten = Kamon.counter("num-downsampled-chunks-written").withoutTags() - - private val readSched = Scheduler.io("cass-read-sched") - private val writeSched = Scheduler.io("cass-write-sched") - - private val session = FiloSessionProvider.openSession(settings.cassandraConfig) - - private[downsampler] val downsampleCassandraColStore = - new CassandraColumnStore(settings.filodbConfig, readSched, session, true)(writeSched) - - private[downsampler] val rawCassandraColStore = - new CassandraColumnStore(settings.filodbConfig, readSched, session, false)(writeSched) - - private val kamonTags = Map( "rawDataset" -> settings.rawDatasetName, +class BatchDownsampler(settings: DownsamplerSettings) extends Instance with Serializable { + + @transient lazy val numBatchesStarted = Kamon.counter("num-batches-started").withoutTags() + @transient lazy val numBatchesCompleted = Kamon.counter("num-batches-completed").withoutTags() + @transient lazy val numBatchesFailed = Kamon.counter("num-batches-failed").withoutTags() + @transient lazy val numPartitionsEncountered = Kamon.counter("num-partitions-encountered").withoutTags() + @transient lazy val numPartitionsBlacklisted = Kamon.counter("num-partitions-blacklisted").withoutTags() + @transient lazy val numPartitionsCompleted = Kamon.counter("num-partitions-completed").withoutTags() + @transient lazy val numPartitionsNoDownsampleSchema = Kamon.counter("num-partitions-no-downsample-schema") + .withoutTags() + @transient lazy val numPartitionsFailed = Kamon.counter("num-partitions-failed").withoutTags() + @transient lazy val numPartitionsSkipped = Kamon.counter("num-partitions-skipped").withoutTags() + @transient lazy val numRawChunksSkipped = Kamon.counter("num-raw-chunks-skipped").withoutTags() + @transient lazy val numRawChunksDownsampled = Kamon.counter("num-raw-chunks-downsampled").withoutTags() + @transient lazy val numDownsampledChunksWritten = Kamon.counter("num-downsampled-chunks-written").withoutTags() + + @transient lazy private val session = DownsamplerContext.getOrCreateCassandraSession(settings.cassandraConfig) + + @transient lazy private[downsampler] val downsampleCassandraColStore = + new CassandraColumnStore(settings.filodbConfig, DownsamplerContext.readSched, session, + true)(DownsamplerContext.writeSched) + + @transient lazy private[downsampler] val rawCassandraColStore = + new CassandraColumnStore(settings.filodbConfig, DownsamplerContext.readSched, session, + false)(DownsamplerContext.writeSched) + + @transient lazy private val kamonTags = Map( "rawDataset" -> settings.rawDatasetName, "owner" -> "BatchDownsampler") - private[downsampler] val schemas = Schemas.fromConfig(settings.filodbConfig).get + @transient lazy private[downsampler] val schemas = Schemas.fromConfig(settings.filodbConfig).get - private val rawSchemas = settings.rawSchemaNames.map { s => schemas.schemas(s)} + @transient lazy private val rawSchemas = settings.rawSchemaNames.map { s => schemas.schemas(s)} /** * Downsample Schemas */ - private val dsSchemas = settings.rawSchemaNames.map { s => schemas.schemas(s).downsample.get} + @transient lazy private val dsSchemas = settings.rawSchemaNames.flatMap { s => schemas.schemas(s).downsample } /** * Chunk Downsamplers by Raw Schema Id */ - private val chunkDownsamplersByRawSchemaId = debox.Map.empty[Int, scala.Seq[ChunkDownsampler]] - rawSchemas.foreach { s => chunkDownsamplersByRawSchemaId += s.schemaHash -> s.data.downsamplers } + @transient lazy private val chunkDownsamplersByRawSchemaId = { + val map = debox.Map.empty[Int, scala.Seq[ChunkDownsampler]] + rawSchemas.foreach { s => map += s.schemaHash -> s.data.downsamplers } + map + } - private val downsamplePeriodMarkersByRawSchemaId = debox.Map.empty[Int, DownsamplePeriodMarker] - rawSchemas.foreach { s => downsamplePeriodMarkersByRawSchemaId += s.schemaHash -> s.data.downsamplePeriodMarker } + @transient lazy private val downsamplePeriodMarkersByRawSchemaId = { + val map = debox.Map.empty[Int, DownsamplePeriodMarker] + rawSchemas.foreach { s => map += s.schemaHash -> s.data.downsamplePeriodMarker } + map + } /** * Raw dataset from which we downsample data */ - private[downsampler] val rawDatasetRef = DatasetRef(settings.rawDatasetName) + @transient lazy private[downsampler] val rawDatasetRef = DatasetRef(settings.rawDatasetName) // FIXME * 4 exists to workaround an issue where we see under-allocation for metaspan due to // possible mis-calculation of max block meta size. - private val maxMetaSize = dsSchemas.map(_.data.blockMetaSize).max * 4 + @transient lazy private val maxMetaSize = dsSchemas.map(_.data.blockMetaSize).max * 4 /** * Datasets to which we write downsampled data. Keyed by Downsample resolution. */ - private[downsampler] val downsampleRefsByRes = settings.downsampleResolutions + @transient lazy private[downsampler] val downsampleRefsByRes = settings.downsampleResolutions .zip(settings.downsampledDatasetRefs).toMap - private[downsampler] val shardStats = new TimeSeriesShardStats(rawDatasetRef, -1) // TODO fix - - import java.time.Instant._ + @transient lazy private[downsampler] val shardStats = new TimeSeriesShardStats(rawDatasetRef, -1) // TODO fix /** * Downsample batch of raw partitions, and store downsampled chunks to cassandra @@ -109,9 +110,10 @@ object BatchDownsampler extends StrictLogging with Instance { def downsampleBatch(rawPartsBatch: Seq[RawPartData], userTimeStart: Long, userTimeEndExclusive: Long): Unit = { - logger.info(s"Starting to downsample batchSize=${rawPartsBatch.size} partitions " + + DownsamplerContext.dsLogger.info(s"Starting to downsample batchSize=${rawPartsBatch.size} partitions " + s"rawDataset=${settings.rawDatasetName} for " + - s"userTimeStart=${ofEpochMilli(userTimeStart)} userTimeEndExclusive=${ofEpochMilli(userTimeEndExclusive)}") + s"userTimeStart=${java.time.Instant.ofEpochMilli(userTimeStart)} " + + s"userTimeEndExclusive=${java.time.Instant.ofEpochMilli(userTimeEndExclusive)}") numBatchesStarted.increment() val startedAt = System.currentTimeMillis() val downsampledChunksToPersist = MMap[FiniteDuration, Iterator[ChunkSet]]() @@ -120,7 +122,7 @@ object BatchDownsampler extends StrictLogging with Instance { } val pagedPartsToFree = ArrayBuffer[PagedReadablePartition]() val downsampledPartsToFree = ArrayBuffer[TimeSeriesPartition]() - val offHeapMem = new OffHeapMemory(rawSchemas.map(_.downsample.get), + val offHeapMem = new OffHeapMemory(rawSchemas.flatMap(_.downsample), kamonTags, maxMetaSize, settings.downsampleStoreConfig) var numDsChunks = 0 val dsRecordBuilder = new RecordBuilder(MemFactory.onHeapFactory) @@ -131,21 +133,22 @@ object BatchDownsampler extends StrictLogging with Instance { val schema = schemas(rawSchemaId) if (schema != Schemas.UnknownSchema) { val pkPairs = schema.partKeySchema.toStringPairs(rawPart.partitionKey, UnsafeUtils.arayOffset) - if (isEligibleForDownsample(pkPairs)) { + if (settings.isEligibleForDownsample(pkPairs)) { try { downsamplePart(offHeapMem, rawPart, pagedPartsToFree, downsampledPartsToFree, downsampledChunksToPersist, userTimeStart, userTimeEndExclusive, dsRecordBuilder) numPartitionsCompleted.increment() } catch { case e: Exception => - logger.error(s"Error occurred when downsampling partition $pkPairs", e) + DownsamplerContext.dsLogger.error(s"Error occurred when downsampling partition $pkPairs", e) numPartitionsFailed.increment() } } else { + DownsamplerContext.dsLogger.debug(s"Skipping blacklisted partition $pkPairs") numPartitionsBlacklisted.increment() } } else { numPartitionsSkipped.increment() - logger.warn(s"Skipping series with unknown schema ID $rawSchemaId") + DownsamplerContext.dsLogger.warn(s"Skipping series with unknown schema ID $rawSchemaId") } } numDsChunks = persistDownsampledChunks(downsampledChunksToPersist) @@ -159,7 +162,7 @@ object BatchDownsampler extends StrictLogging with Instance { } numBatchesCompleted.increment() val endedAt = System.currentTimeMillis() - logger.info(s"Finished iterating through and downsampling batchSize=${rawPartsBatch.size} " + + DownsamplerContext.dsLogger.info(s"Finished iterating through and downsampling batchSize=${rawPartsBatch.size} " + s"partitions in current executor timeTakenMs=${endedAt-startedAt} numDsChunks=$numDsChunks") } @@ -183,20 +186,25 @@ object BatchDownsampler extends StrictLogging with Instance { userTimeStart: Long, userTimeEndExclusive: Long, dsRecordBuilder: RecordBuilder) = { + val rawSchemaId = RecordSchema.schemaID(rawPart.partitionKey, UnsafeUtils.arayOffset) val rawPartSchema = schemas(rawSchemaId) if (rawPartSchema == Schemas.UnknownSchema) throw UnknownSchemaQueryErr(rawSchemaId) rawPartSchema.downsample match { case Some(downsampleSchema) => val rawReadablePart = new PagedReadablePartition(rawPartSchema, 0, 0, rawPart) - logger.debug(s"Downsampling partition ${rawReadablePart.stringPartition}") + DownsamplerContext.dsLogger.debug(s"Downsampling partition ${rawReadablePart.stringPartition}") val bufferPool = offHeapMem.bufferPools(rawPartSchema.downsample.get.schemaHash) val downsamplers = chunkDownsamplersByRawSchemaId(rawSchemaId) val periodMarker = downsamplePeriodMarkersByRawSchemaId(rawSchemaId) + val (_, partKeyPtr, _) = BinaryRegionLarge.allocateAndCopy(rawReadablePart.partKeyBase, rawReadablePart.partKeyOffset, offHeapMem.nativeMemoryManager) + // 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) @@ -217,8 +225,9 @@ object BatchDownsampler extends StrictLogging with Instance { } downsamplePartSpan.finish() case None => - logger.warn(s"Encountered partition ${rawPartSchema.partKeySchema.stringify(rawPart.partitionKey)}" + - s" which does not have a downsample schema") + numPartitionsNoDownsampleSchema.increment() + DownsamplerContext.dsLogger.debug(s"Skipping downsampling of partition " + + s"${rawPartSchema.partKeySchema.stringify(rawPart.partitionKey)} which does not have a downsample schema") } } @@ -262,7 +271,7 @@ object BatchDownsampler extends StrictLogging with Instance { val downsamplePeriods = periodMarker.periods(rawPartToDownsample, chunkset, resMillis, startRow, endRow).toArray() - util.Arrays.sort(downsamplePeriods) + java.util.Arrays.sort(downsamplePeriods) try { // for each downsample period @@ -294,7 +303,7 @@ object BatchDownsampler extends StrictLogging with Instance { } } catch { case e: Exception => - logger.error(s"Error downsampling partition " + + DownsamplerContext.dsLogger.error(s"Error downsampling partition " + s"hexPartKey=${rawPartToDownsample.hexPartKey} " + s"schema=${rawPartToDownsample.schema.name} " + s"resolution=$resolution " + @@ -307,9 +316,10 @@ object BatchDownsampler extends StrictLogging with Instance { } dsRecordBuilder.removeAndFreeContainers(dsRecordBuilder.allContainers.size) } + numRawChunksDownsampled.increment() } else { - numChunksSkipped.increment() - logger.warn(s"Not downsampling chunk of partition since startRow lessThan endRow " + + numRawChunksSkipped.increment() + DownsamplerContext.dsLogger.warn(s"Not downsampling chunk of partition since startRow lessThan endRow " + s"hexPartKey=${rawPartToDownsample.hexPartKey} " + s"startRow=$startRow " + s"endRow=$endRow " + @@ -340,27 +350,13 @@ object BatchDownsampler extends StrictLogging with Instance { writeFut.foreach { fut => val response = Await.result(fut, settings.cassWriteTimeout) - logger.debug(s"Got message $response for cassandra write call") + DownsamplerContext.dsLogger.debug(s"Got message $response for cassandra write call") if (response.isInstanceOf[ErrorResponse]) - logger.error(s"Got response $response when writing to Cassandra") + DownsamplerContext.dsLogger.error(s"Got response $response when writing to Cassandra") } numDownsampledChunksWritten.increment(numChunks) batchWriteSpan.finish() numChunks } - /** - * Two conditions should satisfy for eligibility: - * (a) If whitelist is nonEmpty partKey should match a filter in the whitelist. - * (b) It should not match any filter in blacklist - */ - private def isEligibleForDownsample(pkPairs: Seq[(String, String)]): Boolean = { - import DownsamplerSettings._ - if (whitelist.nonEmpty && !whitelist.exists(w => w.forall(pkPairs.contains))) { - false - } else { - blacklist.forall(w => !w.forall(pkPairs.contains)) - } - } - } diff --git a/spark-jobs/src/main/scala/filodb/downsampler/chunk/DownsamplerMain.scala b/spark-jobs/src/main/scala/filodb/downsampler/chunk/DownsamplerMain.scala new file mode 100644 index 0000000000..8f4fe5da6a --- /dev/null +++ b/spark-jobs/src/main/scala/filodb/downsampler/chunk/DownsamplerMain.scala @@ -0,0 +1,127 @@ +package filodb.downsampler.chunk + +import java.time.Instant +import java.time.format.DateTimeFormatter + +import kamon.Kamon +import org.apache.spark.SparkConf +import org.apache.spark.sql.SparkSession + +import filodb.downsampler.DownsamplerContext + +/** + * + * Goal: Downsample all real-time data. + * Goal: Align chunks when this job is run in multiple DCs so that cross-dc repairs can be done. + * Non-Goal: Downsampling of non-real time data or data with different epoch. + * + * Strategy is to run this spark job every 6 hours at 8am, 2pm, 8pm, 2am UTC each day. + * + * Run at 8am: We query data with ingestionTime from 10pm to 8am. + * Then query and downsample data with userTime between 12am to 6am. + * Downsampled chunk would have an ingestionTime of 12am. + * Run at 2pm: We query data with ingestionTime from 8am to 2pm. + * Then query and downsample data with userTime between 6am to 12pm. + * Downsampled chunk would have an ingestionTime of 6am. + * Run at 8pm: We query data with ingestionTime from 10am to 8pm. + * Then query and downsample data with userTime between 12pm to 6pm. + * Downsampled chunk would have an ingestionTime of 12pm. + * Run at 2am: We query data with ingestionTime from 8pm to 2am. + * Then query and downsample data with userTime between 6pm to 12am. + * Downsampled chunk would have an ingestionTime of 6pm. + * + * This will cover all data with userTime 12am to 12am. + * Since we query for a broader ingestionTime, it will include data arriving early/late by 2 hours. + * + * Important Note: The reason non-real-time data is not included in goals is because we + * want chunk alignment between DCs in downsampled data to enable cross-dc repair without chunk surgery. + * Without chunk-alignment in raw data and consistency in behavior across DCs, it would be difficult + * to achieve chunk alignment in downsampled data. Once we solve that (deferred problem), we will + * lift the constraint. + */ +object DownsamplerMain extends App { + + Kamon.init() // kamon init should be first thing in driver jvm + val settings = new DownsamplerSettings() + val batchDownsampler = new BatchDownsampler(settings) + + val d = new Downsampler(settings, batchDownsampler) + val sparkConf = new SparkConf(loadDefaults = true) + d.run(sparkConf) +} + +class Downsampler(settings: DownsamplerSettings, batchDownsampler: BatchDownsampler) extends Serializable { + + @transient lazy private val jobCompleted = Kamon.counter("chunk-migration-completed").withoutTags() + + // Gotcha!! Need separate function (Cannot be within body of a class) + // to create a closure for spark to serialize and move to executors. + // Otherwise, config values below were not being sent over. + // See https://medium.com/onzo-tech/serialization-challenges-with-spark-and-scala-a2287cd51c54 + // scalastyle:off method.length + def run(sparkConf: SparkConf): SparkSession = { + + val spark = SparkSession.builder() + .appName("FiloDBDownsampler") + .config(sparkConf) + .getOrCreate() + + DownsamplerContext.dsLogger.info(s"Spark Job Properties: ${spark.sparkContext.getConf.toDebugString}") + + // Use the spark property spark.filodb.downsampler.user-time-override to override the + // userTime period for which downsampling should occur. + // Generally disabled, defaults the period that just ended prior to now. + // Specified during reruns for downsampling old data + val userTimeInPeriod: Long = spark.sparkContext.getConf + .getOption("spark.filodb.downsampler.userTimeOverride") match { + // by default assume a time in the previous downsample period + case None => System.currentTimeMillis() - settings.downsampleChunkDuration + // examples: 2019-10-20T12:34:56Z or 2019-10-20T12:34:56-08:00 + case Some(str) => Instant.from(DateTimeFormatter.ISO_OFFSET_DATE_TIME.parse(str)).toEpochMilli() + } + + val userTimeStart: Long = (userTimeInPeriod / settings.downsampleChunkDuration) * settings.downsampleChunkDuration + val userTimeEndExclusive: Long = userTimeStart + settings.downsampleChunkDuration + val ingestionTimeStart: Long = userTimeStart - settings.widenIngestionTimeRangeBy.toMillis + val ingestionTimeEnd: Long = userTimeEndExclusive + settings.widenIngestionTimeRangeBy.toMillis + + DownsamplerContext.dsLogger.info(s"This is the Downsampling driver. Starting downsampling job " + + s"rawDataset=${settings.rawDatasetName} for " + + s"userTimeInPeriod=${java.time.Instant.ofEpochMilli(userTimeInPeriod)} " + + s"ingestionTimeStart=${java.time.Instant.ofEpochMilli(ingestionTimeStart)} " + + s"ingestionTimeEnd=${java.time.Instant.ofEpochMilli(ingestionTimeEnd)} " + + s"userTimeStart=${java.time.Instant.ofEpochMilli(userTimeStart)} " + + s"userTimeEndExclusive=${java.time.Instant.ofEpochMilli(userTimeEndExclusive)}") + + val splits = batchDownsampler.rawCassandraColStore.getScanSplits(batchDownsampler.rawDatasetRef, + settings.splitsPerNode) + 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") + + spark.sparkContext + .makeRDD(splits) + .mapPartitions { splitIter => + Kamon.init() // kamon init should be first thing in worker jvm + import filodb.core.Iterators._ + val rawDataSource = batchDownsampler.rawCassandraColStore + val batchReadSpan = Kamon.spanBuilder("cassandra-raw-data-read-latency").start() + val batchIter = rawDataSource.getChunksByIngestionTimeRange(datasetRef = batchDownsampler.rawDatasetRef, + splits = splitIter, ingestionTimeStart = ingestionTimeStart, + ingestionTimeEnd = ingestionTimeEnd, + userTimeStart = userTimeStart, endTimeExclusive = userTimeEndExclusive, + maxChunkTime = settings.rawDatasetIngestionConfig.storeConfig.maxChunkTime.toMillis, + batchSize = settings.batchSize, batchTime = settings.batchTime).toIterator() + batchReadSpan.finish() + batchIter // iterator of batches + } + .foreach { rawPartsBatch => + Kamon.init() // kamon init should be first thing in worker jvm + batchDownsampler.downsampleBatch(rawPartsBatch, userTimeStart, userTimeEndExclusive) + } + + DownsamplerContext.dsLogger.info(s"Downsampling Driver completed successfully") + jobCompleted.increment() + spark + } + +} diff --git a/spark-jobs/src/main/scala/filodb/downsampler/chunk/DownsamplerSettings.scala b/spark-jobs/src/main/scala/filodb/downsampler/chunk/DownsamplerSettings.scala new file mode 100644 index 0000000000..4e6fc01a9b --- /dev/null +++ b/spark-jobs/src/main/scala/filodb/downsampler/chunk/DownsamplerSettings.scala @@ -0,0 +1,85 @@ +package filodb.downsampler.chunk + +import scala.concurrent.duration._ + +import com.typesafe.config.{Config, ConfigFactory} +import net.ceedubs.ficus.Ficus._ + +import filodb.coordinator.{FilodbSettings, NodeClusterActor} +import filodb.core.store.{IngestionConfig, StoreConfig} +import filodb.downsampler.DownsamplerContext + + +/** + * DownsamplerSettings is always used in the context of an object so that it need not be serialized to a spark executor + * from the spark application driver. + */ +class DownsamplerSettings(conf: Config = ConfigFactory.empty()) extends Serializable { + + @transient lazy val filodbSettings = new FilodbSettings(conf) + + @transient lazy val filodbConfig = filodbSettings.allConfig.getConfig("filodb") + + @transient lazy val downsamplerConfig = { + val conf = filodbConfig.getConfig("downsampler") + DownsamplerContext.dsLogger.info(s"Loaded following downsampler config: ${conf.root().render()}" ) + conf + } + + @transient lazy val cassandraConfig = filodbConfig.getConfig("cassandra") + + @transient lazy val rawDatasetName = downsamplerConfig.getString("raw-dataset-name") + + @transient lazy val rawDatasetIngestionConfig = { + DownsamplerContext.dsLogger.info(s"Parsing dataset configs at ${filodbSettings.datasetConfPaths}") + val ingConf = filodbSettings.streamConfigs.map { config => + IngestionConfig(config, NodeClusterActor.noOpSource.streamFactoryClass).get + }.find(_.ref.toString == rawDatasetName).get + DownsamplerContext.dsLogger.info(s"DatasetConfig for dataset $rawDatasetName was $ingConf") + ingConf + } + + @transient lazy val rawSchemaNames = rawDatasetIngestionConfig.downsampleConfig.schemas + + @transient lazy val downsampleResolutions = rawDatasetIngestionConfig.downsampleConfig.resolutions + + @transient lazy val downsampleTtls = rawDatasetIngestionConfig.downsampleConfig.ttls.map(_.toSeconds.toInt) + + @transient lazy val downsampledDatasetRefs = + rawDatasetIngestionConfig.downsampleConfig.downsampleDatasetRefs(rawDatasetName) + + @transient lazy val downsampleStoreConfig = StoreConfig(downsamplerConfig.getConfig("downsample-store-config")) + + @transient lazy val ttlByResolution = downsampleResolutions.zip(downsampleTtls).toMap + + @transient lazy val batchSize = downsamplerConfig.getInt("cass-write-batch-size") + + @transient lazy val batchTime = downsamplerConfig.as[FiniteDuration]("cass-write-batch-time") + + @transient lazy val splitsPerNode = downsamplerConfig.getInt("splits-per-node") + + @transient lazy val cassWriteTimeout = downsamplerConfig.as[FiniteDuration]("cassandra-write-timeout") + + @transient lazy val widenIngestionTimeRangeBy = downsamplerConfig.as[FiniteDuration]("widen-ingestion-time-range-by") + + @transient lazy val downsampleChunkDuration = downsampleStoreConfig.flushInterval.toMillis + + @transient lazy val whitelist = downsamplerConfig.as[Seq[Map[String, String]]]("whitelist-filters").map(_.toSeq) + + @transient lazy val blacklist = downsamplerConfig.as[Seq[Map[String, String]]]("blacklist-filters").map(_.toSeq) + + /** + * Two conditions should satisfy for eligibility: + * (a) If whitelist is nonEmpty partKey should match a filter in the whitelist. + * (b) It should not match any filter in blacklist + */ + def isEligibleForDownsample(pkPairs: Seq[(String, String)]): Boolean = { + if (whitelist.nonEmpty && !whitelist.exists(w => w.forall(pkPairs.contains))) { + false + } else { + blacklist.forall(w => !w.forall(pkPairs.contains)) + } + } + +} + diff --git a/spark-jobs/src/main/scala/filodb/downsampler/OffHeapMemory.scala b/spark-jobs/src/main/scala/filodb/downsampler/chunk/OffHeapMemory.scala similarity index 98% rename from spark-jobs/src/main/scala/filodb/downsampler/OffHeapMemory.scala rename to spark-jobs/src/main/scala/filodb/downsampler/chunk/OffHeapMemory.scala index 6016adc89a..1b72bb3d11 100644 --- a/spark-jobs/src/main/scala/filodb/downsampler/OffHeapMemory.scala +++ b/spark-jobs/src/main/scala/filodb/downsampler/chunk/OffHeapMemory.scala @@ -1,4 +1,4 @@ -package filodb.downsampler +package filodb.downsampler.chunk import com.typesafe.scalalogging.StrictLogging @@ -16,7 +16,6 @@ class OffHeapMemory(schemas: Seq[Schema], private val blockMemSize = storeConfig.shardMemSize private val nativeMemSize = storeConfig.ingestionBufferMemSize - logger.info(s"Allocating OffHeap memory $this with nativeMemManagerSize=$nativeMemSize " + s"and blockMemorySize=$blockMemSize") val blockStore = new PageAlignedBlockManager(blockMemSize, 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 d146bdb696..2c724a6396 100644 --- a/spark-jobs/src/main/scala/filodb/downsampler/index/DSIndexJob.scala +++ b/spark-jobs/src/main/scala/filodb/downsampler/index/DSIndexJob.scala @@ -2,120 +2,125 @@ package filodb.downsampler.index import scala.concurrent.Await -import com.typesafe.scalalogging.StrictLogging import kamon.Kamon -import monix.execution.Scheduler import monix.reactive.Observable -import filodb.cassandra.FiloSessionProvider import filodb.cassandra.columnstore.CassandraColumnStore import filodb.core.{DatasetRef, Instance} +import filodb.core.binaryrecord2.{RecordBuilder, RecordSchema} import filodb.core.metadata.Schemas import filodb.core.store.PartKeyRecord -import filodb.downsampler.DownsamplerSettings -import filodb.downsampler.DownsamplerSettings.rawDatasetIngestionConfig -import filodb.downsampler.index.DSIndexJobSettings.cassWriteTimeout +import filodb.downsampler.DownsamplerContext +import filodb.downsampler.chunk.DownsamplerSettings import filodb.memory.format.UnsafeUtils -object DSIndexJob extends StrictLogging with Instance { +class DSIndexJob(dsSettings: DownsamplerSettings, + dsJobsettings: DSIndexJobSettings) extends Instance with Serializable { - val settings = DownsamplerSettings - val dsJobsettings = DownsamplerSettings + @transient lazy private val sparkTasksStarted = Kamon.counter("spark-tasks-started").withoutTags() + @transient lazy private val sparkForeachTasksCompleted = Kamon.counter("spark-foreach-tasks-completed") + .withoutTags() + @transient lazy private val sparkTasksFailed = Kamon.counter("spark-tasks-failed").withoutTags() + @transient lazy private val numPartKeysNoDownsampleSchema = Kamon.counter("num-partkeys-no-downsample").withoutTags() + @transient lazy private val numPartKeysMigrated = Kamon.counter("num-partkeys-migrated").withoutTags() + @transient lazy private val numPartKeysBlacklisted = Kamon.counter("num-partkeys-blacklisted").withoutTags() - private val readSched = Scheduler.io("cass-index-read-sched") - private val writeSched = Scheduler.io("cass-index-write-sched") - - val sparkTasksStarted = Kamon.counter("spark-tasks-started").withoutTags() - val sparkForeachTasksCompleted = Kamon.counter("spark-foreach-tasks-completed").withoutTags() - val sparkTasksFailed = Kamon.counter("spark-tasks-failed").withoutTags() - val totalPartkeysUpdated = Kamon.counter("total-partkeys-updated").withoutTags() + @transient lazy private[downsampler] val schemas = Schemas.fromConfig(dsSettings.filodbConfig).get /** * Datasets to which we write downsampled data. Keyed by Downsample resolution. */ - private[downsampler] val downsampleRefsByRes = settings.downsampleResolutions - .zip(settings.downsampledDatasetRefs).toMap - - - private[downsampler] val schemas = Schemas.fromConfig(settings.filodbConfig).get + @transient lazy private[downsampler] val downsampleRefsByRes = dsSettings.downsampleResolutions + .zip(dsSettings.downsampledDatasetRefs).toMap /** * Raw dataset from which we downsample data */ - private[downsampler] val rawDatasetRef = DatasetRef(settings.rawDatasetName) + @transient lazy private[downsampler] val rawDatasetRef = DatasetRef(dsSettings.rawDatasetName) - private val session = FiloSessionProvider.openSession(settings.cassandraConfig) + @transient lazy private val session = DownsamplerContext.getOrCreateCassandraSession(dsSettings.cassandraConfig) - private[index] val downsampleCassandraColStore = - new CassandraColumnStore(dsJobsettings.filodbConfig, readSched, session, true)(writeSched) + @transient lazy private[index] val downsampleCassandraColStore = + new CassandraColumnStore(dsJobsettings.filodbConfig, DownsamplerContext.readSched, + session, true)(DownsamplerContext.writeSched) - private[index] val rawCassandraColStore = - new CassandraColumnStore(dsJobsettings.filodbConfig, readSched, session, false)(writeSched) + @transient lazy private[index] val rawCassandraColStore = + new CassandraColumnStore(dsJobsettings.filodbConfig, DownsamplerContext.readSched, session, + false)(DownsamplerContext.writeSched) - val dsDatasource = downsampleCassandraColStore - val highestDSResolution = rawDatasetIngestionConfig.downsampleConfig.resolutions.last // data retained longest - val dsDatasetRef = downsampleRefsByRes(highestDSResolution) - - def updateDSPartKeyIndex(shard: Int, fromHour: Long, toHour: Long): Unit = { - import DSIndexJobSettings._ + @transient lazy private val dsDatasource = downsampleCassandraColStore + // data retained longest + @transient lazy private val highestDSResolution = + dsSettings.rawDatasetIngestionConfig.downsampleConfig.resolutions.last + @transient lazy private val dsDatasetRef = downsampleRefsByRes(highestDSResolution) + def updateDSPartKeyIndex(shard: Int, fromHour: Long, toHourExcl: Long, fullIndexMigration: Boolean): Unit = { sparkTasksStarted.increment - - val span = Kamon.spanBuilder("per-shard-index-migration-latency") - .asChildOf(Kamon.currentSpan()) - .tag("shard", shard) - .start val rawDataSource = rawCassandraColStore - @volatile var count = 0 try { - if (migrateRawIndex) { - logger.info("migrating complete partkey index") + val span = Kamon.spanBuilder("per-shard-index-migration-latency") + .asChildOf(Kamon.currentSpan()) + .tag("shard", shard) + .start + if (fullIndexMigration) { + DownsamplerContext.dsLogger.info("migrating complete partkey index") val partKeys = rawDataSource.scanPartKeys(ref = rawDatasetRef, shard = shard.toInt) - count += updateDSPartkeys(partKeys, shard) - logger.info(s"Complete Partkey index migration successful for shard=$shard count=$count") + count += migrateWithDownsamplePartKeys(partKeys, shard) + DownsamplerContext.dsLogger.info(s"Complete PartKey index migration successful for shard=$shard count=$count") } else { - for (epochHour <- fromHour to toHour) { + for (epochHour <- fromHour until toHourExcl) { val partKeys = rawDataSource.getPartKeysByUpdateHour(ref = rawDatasetRef, shard = shard.toInt, updateHour = epochHour) - count += updateDSPartkeys(partKeys, shard) + count += migrateWithDownsamplePartKeys(partKeys, shard) } - logger.info(s"Partial Partkey index migration successful for shard=$shard count=$count" + - s" from=$fromHour to=$toHour") + DownsamplerContext.dsLogger.info(s"Partial PartKey index migration successful for shard=$shard count=$count" + + s" fromHour=$fromHour toHourExcl=$toHourExcl") } sparkForeachTasksCompleted.increment() - totalPartkeysUpdated.increment(count) - } catch { - case e: Exception => - logger.error(s"Exception in task count=$count " + - s"shard=$shard from=$fromHour to=$toHour", e) - sparkTasksFailed.increment - throw e - } finally { span.finish() + } catch { case e: Exception => + DownsamplerContext.dsLogger.error(s"Exception in task count=$count " + + s"shard=$shard fromHour=$fromHour toHourExcl=$toHourExcl fullIndexMigration=$fullIndexMigration", e) + sparkTasksFailed.increment + throw e } } - def updateDSPartkeys(partKeys: Observable[PartKeyRecord], shard: Int): Int = { + def migrateWithDownsamplePartKeys(partKeys: Observable[PartKeyRecord], shard: Int): Int = { @volatile var count = 0 - val pkRecords = partKeys.map(toPartkeyRecordWithHash).map{pkey => - count += 1 - logger.debug(s"migrating partition pkstring=${schemas.part.binSchema.stringify(pkey.partKey)}" + - s" start=${pkey.startTime} end=${pkey.endTime}") - pkey - } + val pkRecords = partKeys.filter { pk => + val rawSchemaId = RecordSchema.schemaID(pk.partKey, UnsafeUtils.arayOffset) + val schema = schemas(rawSchemaId) + val pkPairs = schema.partKeySchema.toStringPairs(pk.partKey, UnsafeUtils.arayOffset) + val blacklisted = !dsSettings.isEligibleForDownsample(pkPairs) + val hasDownsampleSchema = schema.downsample.isDefined + if (blacklisted) numPartKeysBlacklisted.increment() + if (!hasDownsampleSchema) numPartKeysNoDownsampleSchema.increment() + DownsamplerContext.dsLogger.info(s"Migrating partition partKey=$pkPairs schema=${schema.name} " + + s"startTime=${pk.startTime} endTime=${pk.endTime} blacklisted=$blacklisted " + + s"hasDownsampleSchema=$hasDownsampleSchema") + val eligible = hasDownsampleSchema && !blacklisted + if (eligible) count += 1 + eligible + }.map(toDownsamplePkrWithHash) + val updateHour = System.currentTimeMillis() / 1000 / 60 / 60 Await.result(dsDatasource.writePartKeys(ref = dsDatasetRef, shard = shard.toInt, partKeys = pkRecords, - diskTTLSeconds = dsJobsettings.ttlByResolution(highestDSResolution), - writeToPkUTTable = false), cassWriteTimeout) - + diskTTLSeconds = dsSettings.ttlByResolution(highestDSResolution), updateHour, + writeToPkUTTable = false), dsSettings.cassWriteTimeout) + numPartKeysMigrated.increment(count) count } - def toPartkeyRecordWithHash(pkRecord: PartKeyRecord): PartKeyRecord = { - val hash = Option(schemas.part.binSchema.partitionHash(pkRecord.partKey, UnsafeUtils.arayOffset)) - PartKeyRecord(pkRecord.partKey, pkRecord.startTime, pkRecord.endTime, hash) + /** + * Builds a new PartKeyRecord with downsample schema. + * This method will throw an exception if schema of part key does not have downsample schema + */ + private def toDownsamplePkrWithHash(pkRecord: PartKeyRecord): PartKeyRecord = { + val dsPartKey = RecordBuilder.buildDownsamplePartKey(pkRecord.partKey, schemas) + val hash = Option(schemas.part.binSchema.partitionHash(dsPartKey, UnsafeUtils.arayOffset)) + PartKeyRecord(dsPartKey.get, pkRecord.startTime, pkRecord.endTime, hash) } - } 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 ebae836b5c..dfc49cd7e3 100644 --- a/spark-jobs/src/main/scala/filodb/downsampler/index/DSIndexJobMain.scala +++ b/spark-jobs/src/main/scala/filodb/downsampler/index/DSIndexJobMain.scala @@ -1,60 +1,108 @@ package filodb.downsampler.index -import com.typesafe.scalalogging.StrictLogging +import java.time.Instant +import java.time.format.DateTimeFormatter + import kamon.Kamon import org.apache.spark.SparkConf import org.apache.spark.sql.SparkSession -object DSIndexJobMain extends App { - import DSIndexJobSettings._ - val migrateUpto: Long = hour() - 1 - val iu = new IndexJobDriver(migrateUpto - batchLookbackInHours, migrateUpto) //migrate partkeys between these hours - val sparkConf = new SparkConf(loadDefaults = true) - iu.run(sparkConf) - iu.shutdown() -} +import filodb.downsampler.DownsamplerContext +import filodb.downsampler.chunk.DownsamplerSettings /** - * Migrate index updates from Raw dataset to Downsampled dataset. - * Updates get applied only to the dataset with highest ttl. * - * Updates are applied sequentially between the provided hours inclusive. As the updates are incremental, if a job run - * fails and successive runs complete successfully, migration still needs to happen from the failed batch upto the - * latest hour. This is to ensure that subsequent mutations were not overwritten. Hence job will be submitted once to - * fix the failed cases. + * Goal: Migrate Part keys into downsample cassandra tables. + * + * Strategy is to run this spark job every 6 hours at 7:15am, 1:15pm, 7:15pm, 1:15am UTC each day. + * + * Run at 7:15am: Will migrate all entries added for update hours 12am, 1am ... and 5am. + * Run at 1:15pm: Will migrate all entries added for update hours 6am, 7am ... and 11am. + * Run at 7:15pm: Will migrate all entries added for update hours 12pm, 1pm ... and 5pm. + * Run at 1:15am: Will migrate all entries added for update hours 6pm, 7pm ... and 11pm. * - * For e.g if there was a failure 12 hours ago. Job will be submitted to run once with 12 hours as lookback time to - * fix the indexes before resuming the regular schedule. + * Job behavior can be overridden/controlled in two ways: + * 1. If `spark.filodb.downsampler.index.doFullMigration` is set to true, full migration is done + * 2. If `spark.filodb.downsampler.index.timeInPeriodOverride` is set to an ISO timestamp, index migration + * for that period will be done. For example: setting to `2020-03-13T15:44:56` will cause migration to + * be run for hours 12pm, 1pm ... and 5pm on 2020-03-13 * - * @param fromHour from epoch hour - inclusive - * @param toHour to epoch hour - inclusive */ -class IndexJobDriver(fromHour: Long, toHour: Long) extends StrictLogging { - import DSIndexJobSettings._ +object DSIndexJobMain extends App { + + Kamon.init() // kamon init should be first thing in driver jvm + val dsSettings = new DownsamplerSettings() + val dsIndexJobSettings = new DSIndexJobSettings(dsSettings) + + //migrate partkeys between these hours + val iu = new IndexJobDriver(dsSettings, dsIndexJobSettings) + val sparkConf = new SparkConf(loadDefaults = true) + iu.run(sparkConf) + +} - def run(conf: SparkConf): Unit = { - import DSIndexJob._ +class IndexJobDriver(dsSettings: DownsamplerSettings, dsIndexJobSettings: DSIndexJobSettings) extends Serializable { + + @transient lazy private val jobCompleted = Kamon.counter("index-migration-completed").withoutTags() + + // scalastyle:off method.length + def run(conf: SparkConf): SparkSession = { val spark = SparkSession.builder() - .appName("FiloDB_DS_IndexUpdater") + .appName("FiloDB_Index_Downsampler") .config(conf) .getOrCreate() - logger.info(s"Spark Job Properties: ${spark.sparkContext.getConf.toDebugString}") - val startHour = fromHour - val endHour = toHour - val rdd = spark.sparkContext - .makeRDD(0 until numShards) - .foreach(updateDSPartKeyIndex(_, startHour, endHour)) + def hour(millis: Long) = millis / 1000 / 60 / 60 - Kamon.counter("index-migration-completed").withoutTags().increment + val timeInMigrationPeriod: Long = spark.sparkContext.getConf + .getOption("spark.filodb.downsampler.index.timeInPeriodOverride") match { + // by default assume a time in the previous downsample period + case None => System.currentTimeMillis() - dsSettings.downsampleChunkDuration + // examples: 2019-10-20T12:34:56Z or 2019-10-20T12:34:56-08:00 + case Some(str) => Instant.from(DateTimeFormatter.ISO_OFFSET_DATE_TIME.parse(str)).toEpochMilli() + } - logger.info(s"IndexUpdater Driver completed successfully") - } + val hourInMigrationPeriod = hour(timeInMigrationPeriod) + val jobIntervalInHours = dsIndexJobSettings.batchLookbackInHours + val fromHour = hourInMigrationPeriod / jobIntervalInHours * jobIntervalInHours + + // Index migration cannot be rerun just for specific hours, since there could have been + // subsequent updates. Perform migration or all hours until last downsample period's hour. + val currentHour = hour(System.currentTimeMillis()) + val toHourExclDefault = currentHour / jobIntervalInHours * jobIntervalInHours + + // this override should almost never used by operators - only for unit testing + val toHourExcl = spark.sparkContext.getConf + .getLong("spark.filodb.downsampler.index.toHourExclOverride", toHourExclDefault) - def shutdown(): Unit = { - import DSIndexJob._ - rawCassandraColStore.shutdown() - downsampleCassandraColStore.shutdown() + // This is required in the following scenarios + // 1. Initial refresh of partkey index to downsampler cluster + // 2. For fixing corrupt downsampler index + val doFullMigration = spark.sparkContext.getConf + .getBoolean("spark.filodb.downsampler.index.doFullMigration", false) + + val job = new DSIndexJob(dsSettings, dsIndexJobSettings) + + DownsamplerContext.dsLogger.info(s"This is the Downsampling Index Migration driver. Starting job... " + + s"fromHour=$fromHour " + + s"toHourExcl=$toHourExcl " + + s"timeInMigrationPeriod=$timeInMigrationPeriod " + + s"doFullMigration=$doFullMigration") + + val numShards = dsIndexJobSettings.numShards + + 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 + job.updateDSPartKeyIndex(shard, startHour, endHourExcl, doFullMigration) + } + DownsamplerContext.dsLogger.info(s"IndexUpdater Driver completed successfully") + jobCompleted.increment() + spark } -} \ No newline at end of file +} diff --git a/spark-jobs/src/main/scala/filodb/downsampler/index/DSIndexJobSettings.scala b/spark-jobs/src/main/scala/filodb/downsampler/index/DSIndexJobSettings.scala index 2950015cab..a4d410e062 100644 --- a/spark-jobs/src/main/scala/filodb/downsampler/index/DSIndexJobSettings.scala +++ b/spark-jobs/src/main/scala/filodb/downsampler/index/DSIndexJobSettings.scala @@ -3,44 +3,42 @@ package filodb.downsampler.index import scala.concurrent.duration._ import com.typesafe.config.ConfigFactory -import com.typesafe.scalalogging.StrictLogging import net.ceedubs.ficus.Ficus._ -import filodb.coordinator.FilodbSettings -import filodb.downsampler.DownsamplerSettings -import filodb.downsampler.DownsamplerSettings.downsampleStoreConfig +import filodb.downsampler.DownsamplerContext +import filodb.downsampler.chunk.DownsamplerSettings /** * DownsamplerSettings is always used in the context of an object so that it need not be serialized to a spark executor * from the spark application driver. */ -object DSIndexJobSettings extends StrictLogging { +class DSIndexJobSettings(settings: DownsamplerSettings) extends Serializable { - val filodbSettings = new FilodbSettings(ConfigFactory.empty) + @transient lazy val filodbSettings = settings.filodbSettings - val filodbConfig = filodbSettings.config + @transient lazy val filodbConfig = filodbSettings.config - val dsIndexJobConfig = filodbConfig.getConfig("ds-index-job") - logger.info(s"Loaded following downsampler config: ${dsIndexJobConfig.root().render()}" ) + @transient lazy val dsIndexJobConfig = { + val conf = filodbConfig.getConfig("ds-index-job") + DownsamplerContext.dsLogger.info(s"Loaded following downsampler config: ${conf.root().render()}" ) + conf + } - val batchSize = dsIndexJobConfig.getInt("cass-write-batch-size") + @transient lazy val batchSize = dsIndexJobConfig.getInt("cass-write-batch-size") - val splitsPerNode = dsIndexJobConfig.getInt("splits-per-node") + @transient lazy val splitsPerNode = dsIndexJobConfig.getInt("splits-per-node") - val cassWriteTimeout = dsIndexJobConfig.as[FiniteDuration]("cassandra-write-timeout") - - val migrateRawIndex = dsIndexJobConfig.getBoolean("migrate-full-index") + @transient lazy val cassWriteTimeout = dsIndexJobConfig.as[FiniteDuration]("cassandra-write-timeout") // Longer lookback-time is needed to account for failures in the job runs. // As the updates need to be applied incrementally, migration needs to happen from the failed batch until the // latest hour. This is to ensure that subsequent mutations were not overwritten. - val batchLookbackInHours = dsIndexJobConfig.as[Option[Long]]("batch-lookback-in-hours") - .getOrElse(downsampleStoreConfig.flushInterval.toHours) + @transient lazy val batchLookbackInHours = dsIndexJobConfig.as[Option[Long]]("batch-lookback-in-hours") + .getOrElse(settings.downsampleStoreConfig.flushInterval.toHours) - val numShards = filodbSettings.streamConfigs - .find(_.getString("dataset") == DownsamplerSettings.rawDatasetName) - .headOption.getOrElse(ConfigFactory.empty()) - .as[Option[Int]]("num-shards").getOrElse(0) + @transient lazy val numShards = filodbSettings.streamConfigs + .find(_.getString("dataset") == settings.rawDatasetName) + .getOrElse(ConfigFactory.empty()) + .as[Option[Int]]("num-shards").get - def hour(millis: Long = System.currentTimeMillis()): Long = millis / 1000 / 60 / 60 } diff --git a/spark-jobs/src/main/scala/filodb/repair/ChunkCopier.scala b/spark-jobs/src/main/scala/filodb/repair/ChunkCopier.scala new file mode 100644 index 0000000000..9b7468471b --- /dev/null +++ b/spark-jobs/src/main/scala/filodb/repair/ChunkCopier.scala @@ -0,0 +1,134 @@ +package filodb.repair + +import java.io.File +import java.time.Instant +import java.time.format.DateTimeFormatter + +import com.typesafe.config.ConfigFactory +import com.typesafe.scalalogging.StrictLogging +import monix.execution.Scheduler +import org.apache.spark.SparkConf +import org.apache.spark.sql.SparkSession + +import filodb.cassandra.FiloSessionProvider +import filodb.cassandra.columnstore.CassandraColumnStore +import filodb.core.{DatasetRef, GlobalConfig} +import filodb.core.store.ScanSplit + +/** + * Contains all the objects necessary for performing worker tasks. Is constructed from SparkConf + * and is cached. + */ +class ChunkCopier(conf: SparkConf) { + private def openConfig(str: String) = { + val sysConfig = GlobalConfig.systemConfig.getConfig("filodb") + ConfigFactory.parseFile(new File(conf.get(str))).getConfig("filodb").withFallback(sysConfig) + } + + // Both "source" and "target" refer to file paths which define config files that have a + // top-level "filodb" section and a "cassandra" subsection. + val sourceConfig = openConfig("spark.filodb.chunkcopier.source.configFile") + val targetConfig = openConfig("spark.filodb.chunkcopier.target.configFile") + + val sourceCassConfig = sourceConfig.getConfig("cassandra") + val targetCassConfig = targetConfig.getConfig("cassandra") + + val sourceDatasetRef = DatasetRef.fromDotString(conf.get("spark.filodb.chunkcopier.source.dataset")) + val targetDatasetRef = DatasetRef.fromDotString(conf.get("spark.filodb.chunkcopier.target.dataset")) + + // Examples: 2019-10-20T12:34:56Z or 2019-10-20T12:34:56-08:00 + private def parseDateTime(str: String) = Instant.from(DateTimeFormatter.ISO_OFFSET_DATE_TIME.parse(str)) + + val ingestionTimeStart = parseDateTime(conf.get("spark.filodb.chunkcopier.ingestionTimeStart")) + val ingestionTimeEnd = parseDateTime(conf.get("spark.filodb.chunkcopier.ingestionTimeEnd")) + + val diskTimeToLiveSeconds = conf.getTimeAsSeconds("spark.filodb.chunkcopier.diskTimeToLive") + val splitsPerNode = conf.getInt("spark.filodb.chunkcopier.splitsPerNode", 1) + val batchSize = conf.getInt("spark.filodb.chunkcopier.batchSize", 10000) + + val readSched = Scheduler.io("cass-read-sched") + val writeSched = Scheduler.io("cass-write-sched") + + val sourceSession = FiloSessionProvider.openSession(sourceCassConfig) + val targetSession = FiloSessionProvider.openSession(targetCassConfig) + + val sourceCassandraColStore = new CassandraColumnStore(sourceConfig, readSched, sourceSession)(writeSched) + val targetCassandraColStore = new CassandraColumnStore(targetConfig, readSched, targetSession)(writeSched) + + private[repair] def getScanSplits = sourceCassandraColStore.getScanSplits(sourceDatasetRef, splitsPerNode) + + def run(splitIter: Iterator[ScanSplit]): Unit = { + sourceCassandraColStore.copyChunksByIngestionTimeRange( + sourceDatasetRef, + splitIter, + ingestionTimeStart.toEpochMilli(), + ingestionTimeEnd.toEpochMilli(), + batchSize, + targetCassandraColStore, + targetDatasetRef, + diskTimeToLiveSeconds.toInt) + } + + def shutdown(): Unit = { + sourceCassandraColStore.shutdown() + targetCassandraColStore.shutdown() + } +} + +object ChunkCopier { + class ByteComparator extends java.util.Comparator[Array[Byte]] { + def compare(a: Array[Byte], b: Array[Byte]): Int = java.util.Arrays.compareUnsigned(a, b) + } + + val cache = new java.util.TreeMap[Array[Byte], ChunkCopier](new ByteComparator) + + // scalastyle: off null + def lookup(conf: SparkConf): ChunkCopier = synchronized { + // SparkConf cannot be used as a key, so serialize it instead. + val bout = new java.io.ByteArrayOutputStream() + val oout = new java.io.ObjectOutputStream(bout) + oout.writeObject(conf) + oout.close() + val key = bout.toByteArray() + + var copier = cache.get(key) + if (copier == null) { + copier = new ChunkCopier(conf) + cache.put(key, copier) + } + copier + } + // scalastyle: on +} + +/** + * For launching the Spark job. + */ +object ChunkCopierMain extends App with StrictLogging { + run(new SparkConf(loadDefaults = true)) + + def run(conf: SparkConf): SparkSession = { + logger.info(s"ChunkCopier Spark Job Properties: ${conf.toDebugString}") + + val copier = ChunkCopier.lookup(conf) + + val spark = SparkSession.builder() + .appName("FiloDBChunkCopier") + .config(conf) + .getOrCreate() + + val splits = copier.getScanSplits + + logger.info(s"Cassandra split size: ${splits.size}. We will have this many spark partitions. " + + s"Tune splitsPerNode which was ${copier.splitsPerNode} if parallelism is low") + + spark.sparkContext + .makeRDD(splits) + .foreachPartition(splitIter => ChunkCopier.lookup(conf).run(splitIter)) + + logger.info(s"ChunkCopier Driver completed successfully") + + copier.shutdown() + spark + } +} diff --git a/spark-jobs/src/test/scala/filodb/downsampler/DownsamplerMainSpec.scala b/spark-jobs/src/test/scala/filodb/downsampler/DownsamplerMainSpec.scala index 4f57035fd3..a16998ebdd 100644 --- a/spark-jobs/src/test/scala/filodb/downsampler/DownsamplerMainSpec.scala +++ b/spark-jobs/src/test/scala/filodb/downsampler/DownsamplerMainSpec.scala @@ -1,5 +1,8 @@ package filodb.downsampler +import java.io.File +import java.time.Instant + import scala.concurrent.Await import scala.concurrent.duration._ @@ -13,20 +16,19 @@ import org.scalatest.time.{Millis, Seconds, Span} import filodb.core.GlobalScheduler._ import filodb.core.MachineMetricsData -import filodb.core.binaryrecord2.{BinaryRecordRowReader, RecordBuilder} +import filodb.core.binaryrecord2.{BinaryRecordRowReader, RecordBuilder, RecordSchema} 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, RawDataRangeVector} +import filodb.core.query.{ColumnFilter, CustomRangeVectorKey, QueryContext, RawDataRangeVector} import filodb.core.query.Filter.Equals import filodb.core.store.{AllChunkScan, PartKeyRecord, SinglePartitionScan, StoreConfig} -import filodb.downsampler.BatchDownsampler.{schemas, shardStats} -import filodb.downsampler.index.DSIndexJobSettings._ -import filodb.downsampler.index.IndexJobDriver +import filodb.downsampler.chunk.{BatchDownsampler, Downsampler, DownsamplerSettings, OffHeapMemory} +import filodb.downsampler.index.{DSIndexJobSettings, IndexJobDriver} import filodb.memory.format.{PrimitiveVectorReader, UnsafeUtils} import filodb.memory.format.ZeroCopyUTF8String._ -import filodb.memory.format.vectors.{CustomBuckets, MutableHistogram} +import filodb.memory.format.vectors.{CustomBuckets, LongHistogram} import filodb.query.{QueryConfig, QueryResult} import filodb.query.exec.{InProcessPlanDispatcher, MultiSchemaPartitionsExec} @@ -38,20 +40,29 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w implicit val defaultPatience = PatienceConfig(timeout = Span(30, Seconds), interval = Span(250, Millis)) - System.setProperty("config.file", "conf/timeseries-filodb-server.conf") - val seriesTags = Map("_ws_".utf8 -> "my_ws".utf8, - "_ns_".utf8 -> "my_ns".utf8) + val conf = ConfigFactory.parseFile(new File("conf/timeseries-filodb-server.conf")) + + val settings = new DownsamplerSettings(conf) + val dsIndexJobSettings = new DSIndexJobSettings(settings) + val batchDownsampler = new BatchDownsampler(settings) + + val seriesTags = Map("_ws_".utf8 -> "my_ws".utf8, "_ns_".utf8 -> "my_ns".utf8) - val rawColStore = BatchDownsampler.rawCassandraColStore - val downsampleColStore = BatchDownsampler.downsampleCassandraColStore + val rawColStore = batchDownsampler.rawCassandraColStore + val downsampleColStore = batchDownsampler.downsampleCassandraColStore val rawDataStoreConfig = StoreConfig(ConfigFactory.parseString( """ |flush-interval = 1h |shard-mem-size = 1MB + |ingestion-buffer-mem-size = 30MB """.stripMargin)) - val offheapMem = new OffHeapMemory(Seq(Schemas.gauge, Schemas.promCounter, Schemas.promHistogram), + val offheapMem = new OffHeapMemory(Seq(Schemas.gauge, Schemas.promCounter, Schemas.promHistogram, Schemas.untyped), Map.empty, 100, rawDataStoreConfig) + + val untypedName = "my_untyped" + var untypedPartKeyBytes: Array[Byte] = _ + val gaugeName = "my_gauge" var gaugePartKeyBytes: Array[Byte] = _ @@ -64,29 +75,70 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w val gaugeLowFreqName = "my_gauge_low_freq" var gaugeLowFreqPartKeyBytes: Array[Byte] = _ - val lastSampleTime = 1574273042000L - val downsampler = new Downsampler + val lastSampleTime = 1574373042000L + val pkUpdateHour = hour(lastSampleTime) - //Index migration job, runs for current 2hours for testing. actual job migrates last 6 hour's index updates - val currentHour = hour() - val indexUpdater = new IndexJobDriver(currentHour - 2, currentHour) + val metricNames = Seq(gaugeName, gaugeLowFreqName, counterName, histName, untypedName) - def partKeyReader(pkr: PartKeyRecord): Seq[(String, String)] = { - schemas.part.binSchema.toStringPairs(pkr.partKey, UnsafeUtils.arayOffset) - } + val downsampler = new Downsampler(settings, batchDownsampler) + + def hour(millis: Long = System.currentTimeMillis()): Long = millis / 1000 / 60 / 60 + + val indexUpdater = new IndexJobDriver(settings, dsIndexJobSettings) override def beforeAll(): Unit = { - BatchDownsampler.downsampleRefsByRes.values.foreach { ds => + batchDownsampler.downsampleRefsByRes.values.foreach { ds => downsampleColStore.initialize(ds, 4).futureValue downsampleColStore.truncate(ds, 4).futureValue } - rawColStore.initialize(BatchDownsampler.rawDatasetRef, 4).futureValue - rawColStore.truncate(BatchDownsampler.rawDatasetRef, 4).futureValue + rawColStore.initialize(batchDownsampler.rawDatasetRef, 4).futureValue + rawColStore.truncate(batchDownsampler.rawDatasetRef, 4).futureValue } override def afterAll(): Unit = { offheapMem.free() - downsampler.shutdown() + } + + it ("should write untyped data to cassandra") { + + val rawDataset = Dataset("prometheus", Schemas.untyped) + + val partBuilder = new RecordBuilder(offheapMem.nativeMemoryManager) + val partKey = partBuilder.partKeyFromObjects(Schemas.untyped, untypedName, seriesTags) + + val part = new TimeSeriesPartition(0, Schemas.untyped, partKey, + 0, offheapMem.bufferPools(Schemas.untyped.schemaHash), batchDownsampler.shardStats, + offheapMem.nativeMemoryManager, 1) + + untypedPartKeyBytes = part.partKeyBytes + + val rawSamples = Stream( + Seq(1574372801000L, 3d, untypedName, seriesTags), + Seq(1574372802000L, 5d, untypedName, seriesTags), + + Seq(1574372861000L, 9d, untypedName, seriesTags), + Seq(1574372862000L, 11d, untypedName, seriesTags), + + Seq(1574372921000L, 13d, untypedName, seriesTags), + Seq(1574372922000L, 15d, untypedName, seriesTags), + + Seq(1574372981000L, 17d, untypedName, seriesTags), + Seq(1574372982000L, 15d, untypedName, seriesTags), + + Seq(1574373041000L, 13d, untypedName, seriesTags), + Seq(1574373042000L, 11d, untypedName, seriesTags) + ) + + MachineMetricsData.records(rawDataset, rawSamples).records.foreach { case (base, offset) => + val rr = new BinaryRecordRowReader(Schemas.untyped.ingestionSchema, base, offset) + part.ingest( lastSampleTime, rr, offheapMem.blockMemFactory) + } + part.switchBuffers(offheapMem.blockMemFactory, true) + val chunks = part.makeFlushChunks(offheapMem.blockMemFactory) + + rawColStore.write(rawDataset.ref, Observable.fromIterator(chunks)).futureValue + val pk = PartKeyRecord(untypedPartKeyBytes, 1574372801000L, 1574373042000L, Some(150)) + rawColStore.writePartKeys(rawDataset.ref, 0, Observable.now(pk), 259200, pkUpdateHour).futureValue } it ("should write gauge data to cassandra") { @@ -97,26 +149,26 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w val partKey = partBuilder.partKeyFromObjects(Schemas.gauge, gaugeName, seriesTags) val part = new TimeSeriesPartition(0, Schemas.gauge, partKey, - 0, offheapMem.bufferPools(Schemas.gauge.schemaHash), shardStats, + 0, offheapMem.bufferPools(Schemas.gauge.schemaHash), batchDownsampler.shardStats, offheapMem.nativeMemoryManager, 1) gaugePartKeyBytes = part.partKeyBytes val rawSamples = Stream( - Seq(1574272801000L, 3d, gaugeName, seriesTags), - Seq(1574272802000L, 5d, gaugeName, seriesTags), + Seq(1574372801000L, 3d, gaugeName, seriesTags), + Seq(1574372802000L, 5d, gaugeName, seriesTags), - Seq(1574272861000L, 9d, gaugeName, seriesTags), - Seq(1574272862000L, 11d, gaugeName, seriesTags), + Seq(1574372861000L, 9d, gaugeName, seriesTags), + Seq(1574372862000L, 11d, gaugeName, seriesTags), - Seq(1574272921000L, 13d, gaugeName, seriesTags), - Seq(1574272922000L, 15d, gaugeName, seriesTags), + Seq(1574372921000L, 13d, gaugeName, seriesTags), + Seq(1574372922000L, 15d, gaugeName, seriesTags), - Seq(1574272981000L, 17d, gaugeName, seriesTags), - Seq(1574272982000L, 15d, gaugeName, seriesTags), + Seq(1574372981000L, 17d, gaugeName, seriesTags), + Seq(1574372982000L, 15d, gaugeName, seriesTags), - Seq(1574273041000L, 13d, gaugeName, seriesTags), - Seq(1574273042000L, 11d, gaugeName, seriesTags) + Seq(1574373041000L, 13d, gaugeName, seriesTags), + Seq(1574373042000L, 11d, gaugeName, seriesTags) ) MachineMetricsData.records(rawDataset, rawSamples).records.foreach { case (base, offset) => @@ -127,8 +179,8 @@ 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, 1574272801000L, 1574273042000L, Some(150)) - rawColStore.writePartKeys(rawDataset.ref, 0, Observable.now(pk), 259200).futureValue + val pk = PartKeyRecord(gaugePartKeyBytes, 1574372801000L, 1574373042000L, Some(150)) + rawColStore.writePartKeys(rawDataset.ref, 0, Observable.now(pk), 259200, pkUpdateHour).futureValue } it ("should write low freq gauge data to cassandra") { @@ -139,24 +191,24 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w val partKey = partBuilder.partKeyFromObjects(Schemas.gauge, gaugeLowFreqName, seriesTags) val part = new TimeSeriesPartition(0, Schemas.gauge, partKey, - 0, offheapMem.bufferPools(Schemas.gauge.schemaHash), shardStats, + 0, offheapMem.bufferPools(Schemas.gauge.schemaHash), batchDownsampler.shardStats, offheapMem.nativeMemoryManager, 1) gaugeLowFreqPartKeyBytes = part.partKeyBytes val rawSamples = Stream( - Seq(1574272801000L, 3d, gaugeName, seriesTags), - Seq(1574272802000L, 5d, gaugeName, seriesTags), + Seq(1574372801000L, 3d, gaugeName, seriesTags), + Seq(1574372802000L, 5d, gaugeName, seriesTags), // skip next minute - Seq(1574272921000L, 13d, gaugeName, seriesTags), - Seq(1574272922000L, 15d, gaugeName, seriesTags), + Seq(1574372921000L, 13d, gaugeName, seriesTags), + Seq(1574372922000L, 15d, gaugeName, seriesTags), // skip next minute - Seq(1574273041000L, 13d, gaugeName, seriesTags), - Seq(1574273042000L, 11d, gaugeName, seriesTags) + Seq(1574373041000L, 13d, gaugeName, seriesTags), + Seq(1574373042000L, 11d, gaugeName, seriesTags) ) MachineMetricsData.records(rawDataset, rawSamples).records.foreach { case (base, offset) => @@ -167,8 +219,8 @@ 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, 1574272801000L, 1574273042000L, Some(150)) - rawColStore.writePartKeys(rawDataset.ref, 0, Observable.now(pk), 259200).futureValue + val pk = PartKeyRecord(gaugeLowFreqPartKeyBytes, 1574372801000L, 1574373042000L, Some(150)) + rawColStore.writePartKeys(rawDataset.ref, 0, Observable.now(pk), 259200, pkUpdateHour).futureValue } it ("should write prom counter data to cassandra") { @@ -179,30 +231,30 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w val partKey = partBuilder.partKeyFromObjects(Schemas.promCounter, counterName, seriesTags) val part = new TimeSeriesPartition(0, Schemas.promCounter, partKey, - 0, offheapMem.bufferPools(Schemas.promCounter.schemaHash), shardStats, + 0, offheapMem.bufferPools(Schemas.promCounter.schemaHash), batchDownsampler.shardStats, offheapMem.nativeMemoryManager, 1) counterPartKeyBytes = part.partKeyBytes val rawSamples = Stream( - Seq(1574272801000L, 3d, counterName, seriesTags), - Seq(1574272801500L, 4d, counterName, seriesTags), - Seq(1574272802000L, 5d, counterName, seriesTags), + Seq(1574372801000L, 3d, counterName, seriesTags), + Seq(1574372801500L, 4d, counterName, seriesTags), + Seq(1574372802000L, 5d, counterName, seriesTags), - Seq(1574272861000L, 9d, counterName, seriesTags), - Seq(1574272861500L, 10d, counterName, seriesTags), - Seq(1574272862000L, 11d, counterName, seriesTags), + Seq(1574372861000L, 9d, counterName, seriesTags), + Seq(1574372861500L, 10d, counterName, seriesTags), + Seq(1574372862000L, 11d, counterName, seriesTags), - Seq(1574272921000L, 2d, counterName, seriesTags), - Seq(1574272921500L, 7d, counterName, seriesTags), - Seq(1574272922000L, 15d, counterName, seriesTags), + Seq(1574372921000L, 2d, counterName, seriesTags), + Seq(1574372921500L, 7d, counterName, seriesTags), + Seq(1574372922000L, 15d, counterName, seriesTags), - Seq(1574272981000L, 17d, counterName, seriesTags), - Seq(1574272981500L, 1d, counterName, seriesTags), - Seq(1574272982000L, 15d, counterName, seriesTags), + Seq(1574372981000L, 17d, counterName, seriesTags), + Seq(1574372981500L, 1d, counterName, seriesTags), + Seq(1574372982000L, 15d, counterName, seriesTags), - Seq(1574273041000L, 18d, counterName, seriesTags), - Seq(1574273042000L, 20d, counterName, seriesTags) + Seq(1574373041000L, 18d, counterName, seriesTags), + Seq(1574373042000L, 20d, counterName, seriesTags) ) MachineMetricsData.records(rawDataset, rawSamples).records.foreach { case (base, offset) => @@ -213,8 +265,8 @@ 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, 1574272801000L, 1574273042000L, Some(1)) - rawColStore.writePartKeys(rawDataset.ref, 0, Observable.now(pk), 259200).futureValue + val pk = PartKeyRecord(counterPartKeyBytes, 1574372801000L, 1574373042000L, Some(1)) + rawColStore.writePartKeys(rawDataset.ref, 0, Observable.now(pk), 259200, pkUpdateHour).futureValue } it ("should write prom histogram data to cassandra") { @@ -225,31 +277,31 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w val partKey = partBuilder.partKeyFromObjects(Schemas.promHistogram, histName, seriesTags) val part = new TimeSeriesPartition(0, Schemas.promHistogram, partKey, - 0, offheapMem.bufferPools(Schemas.promHistogram.schemaHash), shardStats, + 0, offheapMem.bufferPools(Schemas.promHistogram.schemaHash), batchDownsampler.shardStats, offheapMem.nativeMemoryManager, 1) histPartKeyBytes = part.partKeyBytes val bucketScheme = CustomBuckets(Array(3d, 10d, Double.PositiveInfinity)) val rawSamples = Stream( // time, sum, count, hist, name, tags - Seq(1574272801000L, 0d, 1d, MutableHistogram(bucketScheme, Array(0d, 0d, 1d)), histName, seriesTags), - Seq(1574272801500L, 2d, 3d, MutableHistogram(bucketScheme, Array(0d, 2d, 3d)), histName, seriesTags), - Seq(1574272802000L, 5d, 6d, MutableHistogram(bucketScheme, Array(2d, 5d, 6d)), histName, seriesTags), + 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(1574272861000L, 9d, 9d, MutableHistogram(bucketScheme, Array(2d, 5d, 9d)), histName, seriesTags), - Seq(1574272861500L, 10d, 10d, MutableHistogram(bucketScheme, Array(2d, 5d, 10d)), histName, seriesTags), - Seq(1574272862000L, 11d, 14d, MutableHistogram(bucketScheme, Array(2d, 8d, 14d)), 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(1574272921000L, 2d, 2d, MutableHistogram(bucketScheme, Array(0d, 0d, 2d)), histName, seriesTags), - Seq(1574272921500L, 7d, 9d, MutableHistogram(bucketScheme, Array(1d, 7d, 9d)), histName, seriesTags), - Seq(1574272922000L, 15d, 19d, MutableHistogram(bucketScheme, Array(1d, 15d, 19d)), 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(1574272981000L, 17d, 21d, MutableHistogram(bucketScheme, Array(2d, 16d, 21d)), histName, seriesTags), - Seq(1574272981500L, 1d, 1d, MutableHistogram(bucketScheme, Array(0d, 1d, 1d)), histName, seriesTags), - Seq(1574272982000L, 15d, 15d, MutableHistogram(bucketScheme, Array(0d, 15d, 15d)), 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(1574273041000L, 18d, 19d, MutableHistogram(bucketScheme, Array(1d, 16d, 19d)), histName, seriesTags), - Seq(1574273042000L, 20d, 25d, MutableHistogram(bucketScheme, Array(4d, 20d, 25d)), 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) ) MachineMetricsData.records(rawDataset, rawSamples).records.foreach { case (base, offset) => @@ -260,8 +312,31 @@ 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, 1574272801000L, 1574273042000L, Some(199)) - rawColStore.writePartKeys(rawDataset.ref, 0, Observable.now(pk), 259200).futureValue + val pk = PartKeyRecord(histPartKeyBytes, 1574372801000L, 1574373042000L, Some(199)) + rawColStore.writePartKeys(rawDataset.ref, 0, Observable.now(pk), 259200, pkUpdateHour).futureValue + } + + val numShards = dsIndexJobSettings.numShards + val bulkPkUpdateHours = { + val start = pkUpdateHour / 6 * 6 // 6 is number of hours per downsample chunk + start until start + 6 + } + it("should simulate bulk part key records being written 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", seriesTags) + val bytes = schema.partKeySchema.asByteArray(UnsafeUtils.ZeroPointer, partKey) + PkToWrite(PartKeyRecord(bytes, 0L, 1000L, Some(-i)), i % numShards, bulkPkUpdateHours(i % bulkPkUpdateHours.size)) + } + + val rawDataset = Dataset("prometheus", Schemas.promHistogram) + pks.groupBy(k => (k.shard, k.updateHour)).foreach { case ((shard, updHour), shardPks) => + rawColStore.writePartKeys(rawDataset.ref, shard, Observable.fromIterable(shardPks).map(_.pkr), + 259200, updHour).futureValue + } } it ("should free all offheap memory") { @@ -271,35 +346,64 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w it ("should downsample raw data into the downsample dataset tables in cassandra using spark job") { val sparkConf = new SparkConf(loadDefaults = true) sparkConf.setMaster("local[2]") - sparkConf.set("spark.filodb.downsampler.userTimeOverride", lastSampleTime.toString) - downsampler.run(sparkConf) + sparkConf.set("spark.filodb.downsampler.userTimeOverride", Instant.ofEpochMilli(lastSampleTime).toString) + downsampler.run(sparkConf).close() } it ("should migrate partKey data into the downsample dataset tables in cassandra using spark job") { val sparkConf = new SparkConf(loadDefaults = true) sparkConf.setMaster("local[2]") - indexUpdater.run(sparkConf) + sparkConf.set("spark.filodb.downsampler.index.timeInPeriodOverride", Instant.ofEpochMilli(lastSampleTime).toString) + sparkConf.set("spark.filodb.downsampler.index.toHourExclOverride", (pkUpdateHour + 6 + 1).toString) + indexUpdater.run(sparkConf).close() } - it ("should recover migrated partKey data") { - val metrics = Seq(counterName, gaugeName, gaugeLowFreqName, histName) - val partKeys = downsampleColStore.scanPartKeys(BatchDownsampler.downsampleRefsByRes(FiniteDuration(5, "min")), 0) - val partition = Await.result(partKeys.map(partKeyReader).toListL.runAsync, 1 minutes) - .map(tags => tags.find(_._1 == "_metric_").get._2).sorted - partition shouldEqual metrics + it ("should recover migrated partKey data and match the downsampled schema") { + + def pkMetricSchemaReader(pkr: PartKeyRecord): (String, String) = { + val schemaId = RecordSchema.schemaID(pkr.partKey, UnsafeUtils.arayOffset) + val partSchema = batchDownsampler.schemas(schemaId) + val strPairs = batchDownsampler.schemas.part.binSchema.toStringPairs(pkr.partKey, UnsafeUtils.arayOffset) + (strPairs.find(p => p._1 == "_metric_").get._2, partSchema.data.name) + } + + val metrics = Set((counterName, Schemas.promCounter.name), + (gaugeName, Schemas.dsGauge.name), + (gaugeLowFreqName, Schemas.dsGauge.name), + (histName, Schemas.promHistogram.name)) + val partKeys = downsampleColStore.scanPartKeys(batchDownsampler.downsampleRefsByRes(FiniteDuration(5, "min")), 0) + val tsSchemaMetric = Await.result(partKeys.map(pkMetricSchemaReader).toListL.runAsync, 1 minutes) + tsSchemaMetric.filter(k => metricNames.contains(k._1)).toSet shouldEqual metrics + } + + it("should verify bulk part key record migration and validate completeness of PK migration") { + + 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 + + // 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) } it("should read and verify gauge data in cassandra using PagedReadablePartition for 1-min downsampled data") { + val dsGaugePartKeyBytes = RecordBuilder.buildDownsamplePartKey(gaugePartKeyBytes, batchDownsampler.schemas).get val downsampledPartData1 = downsampleColStore.readRawPartitions( - BatchDownsampler.downsampleRefsByRes(FiniteDuration(1, "min")), + batchDownsampler.downsampleRefsByRes(FiniteDuration(1, "min")), 0, - SinglePartitionScan(gaugePartKeyBytes)) + SinglePartitionScan(dsGaugePartKeyBytes)) .toListL.runAsync.futureValue.head val downsampledPart1 = new PagedReadablePartition(Schemas.gauge.downsample.get, 0, 0, downsampledPartData1) - downsampledPart1.partKeyBytes shouldEqual gaugePartKeyBytes + downsampledPart1.partKeyBytes shouldEqual dsGaugePartKeyBytes val rv1 = RawDataRangeVector(CustomRangeVectorKey.empty, downsampledPart1, AllChunkScan, Array(0, 1, 2, 3, 4, 5)) @@ -309,25 +413,27 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w // time, min, max, sum, count, avg downsampledData1 shouldEqual Seq( - (1574272802000L, 3.0, 5.0, 8.0, 2.0, 4.0), - (1574272862000L, 9.0, 11.0, 20.0, 2.0, 10.0), - (1574272922000L, 13.0, 15.0, 28.0, 2.0, 14.0), - (1574272982000L, 15.0, 17.0, 32.0, 2.0, 16.0), - (1574273042000L, 11.0, 13.0, 24.0, 2.0, 12.0) + (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) ) } it("should read and verify low freq gauge in cassandra using PagedReadablePartition for 1-min downsampled data") { + val dsGaugeLowFreqPartKeyBytes = RecordBuilder.buildDownsamplePartKey(gaugeLowFreqPartKeyBytes, + batchDownsampler.schemas).get val downsampledPartData1 = downsampleColStore.readRawPartitions( - BatchDownsampler.downsampleRefsByRes(FiniteDuration(1, "min")), + batchDownsampler.downsampleRefsByRes(FiniteDuration(1, "min")), 0, - SinglePartitionScan(gaugeLowFreqPartKeyBytes)) + SinglePartitionScan(dsGaugeLowFreqPartKeyBytes)) .toListL.runAsync.futureValue.head val downsampledPart1 = new PagedReadablePartition(Schemas.gauge.downsample.get, 0, 0, downsampledPartData1) - downsampledPart1.partKeyBytes shouldEqual gaugeLowFreqPartKeyBytes + downsampledPart1.partKeyBytes shouldEqual dsGaugeLowFreqPartKeyBytes val rv1 = RawDataRangeVector(CustomRangeVectorKey.empty, downsampledPart1, AllChunkScan, Array(0, 1, 2, 3, 4, 5)) @@ -337,16 +443,16 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w // time, min, max, sum, count, avg downsampledData1 shouldEqual Seq( - (1574272802000L, 3.0, 5.0, 8.0, 2.0, 4.0), - (1574272922000L, 13.0, 15.0, 28.0, 2.0, 14.0), - (1574273042000L, 11.0, 13.0, 24.0, 2.0, 12.0) + (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) ) } it("should read and verify prom counter data in cassandra using PagedReadablePartition for 1-min downsampled data") { val downsampledPartData1 = downsampleColStore.readRawPartitions( - BatchDownsampler.downsampleRefsByRes(FiniteDuration(1, "min")), + batchDownsampler.downsampleRefsByRes(FiniteDuration(1, "min")), 0, SinglePartitionScan(counterPartKeyBytes)) .toListL.runAsync.futureValue.head @@ -366,19 +472,19 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w // time, counter downsampledData1 shouldEqual Seq( - (1574272801000L, 3d), - (1574272802000L, 5d), + (1574372801000L, 3d), + (1574372802000L, 5d), - (1574272862000L, 11d), + (1574372862000L, 11d), - (1574272921000L, 2d), - (1574272922000L, 15d), + (1574372921000L, 2d), + (1574372922000L, 15d), - (1574272981000L, 17d), - (1574272981500L, 1d), - (1574272982000L, 15d), + (1574372981000L, 17d), + (1574372981500L, 1d), + (1574372982000L, 15d), - (1574273042000L, 20d) + (1574373042000L, 20d) ) } @@ -387,7 +493,7 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w "PagedReadablePartition for 1-min downsampled data") { val downsampledPartData1 = downsampleColStore.readRawPartitions( - BatchDownsampler.downsampleRefsByRes(FiniteDuration(1, "min")), + batchDownsampler.downsampleRefsByRes(FiniteDuration(1, "min")), 0, SinglePartitionScan(histPartKeyBytes)) .toListL.runAsync.futureValue.head @@ -411,32 +517,33 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w // time, sum, count, histogram downsampledData1 shouldEqual Seq( - (1574272801000L, 0d, 1d, Seq(0d, 0d, 1d)), - (1574272802000L, 5d, 6d, Seq(2d, 5d, 6d)), + (1574372801000L, 0d, 1d, Seq(0d, 0d, 1d)), + (1574372802000L, 5d, 6d, Seq(2d, 5d, 6d)), - (1574272862000L, 11d, 14d, Seq(2d, 8d, 14d)), + (1574372862000L, 11d, 14d, Seq(2d, 8d, 14d)), - (1574272921000L, 2d, 2d, Seq(0d, 0d, 2d)), - (1574272922000L, 15d, 19d, Seq(1d, 15d, 19d)), + (1574372921000L, 2d, 2d, Seq(0d, 0d, 2d)), + (1574372922000L, 15d, 19d, Seq(1d, 15d, 19d)), - (1574272981000L, 17d, 21d, Seq(2d, 16d, 21d)), - (1574272981500L, 1d, 1d, Seq(0d, 1d, 1d)), - (1574272982000L, 15d, 15d, Seq(0d, 15d, 15d)), + (1574372981000L, 17d, 21d, Seq(2d, 16d, 21d)), + (1574372981500L, 1d, 1d, Seq(0d, 1d, 1d)), + (1574372982000L, 15d, 15d, Seq(0d, 15d, 15d)), - (1574273042000L, 20d, 25d, Seq(4d, 20d, 25d)) + (1574373042000L, 20d, 25d, Seq(4d, 20d, 25d)) ) } it("should read and verify gauge data in cassandra using PagedReadablePartition for 5-min downsampled data") { + val dsGaugePartKeyBytes = RecordBuilder.buildDownsamplePartKey(gaugePartKeyBytes, batchDownsampler.schemas).get val downsampledPartData2 = downsampleColStore.readRawPartitions( - BatchDownsampler.downsampleRefsByRes(FiniteDuration(5, "min")), + batchDownsampler.downsampleRefsByRes(FiniteDuration(5, "min")), 0, - SinglePartitionScan(gaugePartKeyBytes)) + SinglePartitionScan(dsGaugePartKeyBytes)) .toListL.runAsync.futureValue.head val downsampledPart2 = new PagedReadablePartition(Schemas.gauge.downsample.get, 0, 0, downsampledPartData2) - downsampledPart2.partKeyBytes shouldEqual gaugePartKeyBytes + downsampledPart2.partKeyBytes shouldEqual dsGaugePartKeyBytes val rv2 = RawDataRangeVector(CustomRangeVectorKey.empty, downsampledPart2, AllChunkScan, Array(0, 1, 2, 3, 4, 5)) @@ -446,15 +553,15 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w // time, min, max, sum, count, avg downsampledData2 shouldEqual Seq( - (1574273042000L, 3.0, 17.0, 112.0, 10.0, 11.2) + (1574372982000L, 3.0, 17.0, 88.0, 8.0, 11.0), + (1574373042000L, 11.0, 13.0, 24.0, 2.0, 12.0) ) } - it("should read and verify prom counter data in cassandra using PagedReadablePartition for 5-min downsampled data") { val downsampledPartData1 = downsampleColStore.readRawPartitions( - BatchDownsampler.downsampleRefsByRes(FiniteDuration(5, "min")), + batchDownsampler.downsampleRefsByRes(FiniteDuration(5, "min")), 0, SinglePartitionScan(counterPartKeyBytes)) .toListL.runAsync.futureValue.head @@ -474,16 +581,18 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w // time, counter downsampledData1 shouldEqual Seq( - (1574272801000L, 3d), + (1574372801000L, 3d), - (1574272862000L, 11d), + (1574372862000L, 11d), - (1574272921000L, 2d), + (1574372921000L, 2d), - (1574272981000L, 17d), - (1574272981500L, 1d), + (1574372981000L, 17d), + (1574372981500L, 1d), - (1574273042000L, 20d) + (1574372982000L, 15.0d), + + (1574373042000L, 20.0d) ) } @@ -491,7 +600,7 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w "PagedReadablePartition for 5-min downsampled data") { val downsampledPartData1 = downsampleColStore.readRawPartitions( - BatchDownsampler.downsampleRefsByRes(FiniteDuration(5, "min")), + batchDownsampler.downsampleRefsByRes(FiniteDuration(5, "min")), 0, SinglePartitionScan(histPartKeyBytes)) .toListL.runAsync.futureValue.head @@ -515,37 +624,37 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w // time, sum, count, histogram downsampledData1 shouldEqual Seq( - (1574272801000L, 0d, 1d, Seq(0d, 0d, 1d)), - (1574272862000L, 11d, 14d, Seq(2d, 8d, 14d)), - (1574272921000L, 2d, 2d, Seq(0d, 0d, 2d)), - (1574272981000L, 17d, 21d, Seq(2d, 16d, 21d)), - (1574272981500L, 1d, 1d, Seq(0d, 1d, 1d)), - (1574273042000L, 20d, 25d, Seq(4d, 20d, 25d)) + (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)) ) - } it("should bring up DownsampledTimeSeriesShard and be able to read data using SelectRawPartitionsExec") { val downsampleTSStore = new DownsampledTimeSeriesStore(downsampleColStore, rawColStore, - DownsamplerSettings.filodbConfig) + settings.filodbConfig) - downsampleTSStore.setup(BatchDownsampler.rawDatasetRef, DownsamplerSettings.filodbSettings.schemas, - 0, rawDataStoreConfig, DownsamplerSettings.rawDatasetIngestionConfig.downsampleConfig) + downsampleTSStore.setup(batchDownsampler.rawDatasetRef, settings.filodbSettings.schemas, + 0, rawDataStoreConfig, settings.rawDatasetIngestionConfig.downsampleConfig) - downsampleTSStore.recoverIndex(BatchDownsampler.rawDatasetRef, 0).futureValue + downsampleTSStore.recoverIndex(batchDownsampler.rawDatasetRef, 0).futureValue val colFilters = seriesTags.map { case (t, v) => ColumnFilter(t.toString, Equals(v.toString)) }.toSeq Seq(gaugeName, gaugeLowFreqName, counterName, histName).foreach { metricName => val queryFilters = colFilters :+ ColumnFilter("_metric_", Equals(metricName)) - val exec = MultiSchemaPartitionsExec("someId", System.currentTimeMillis(), - 1000, InProcessPlanDispatcher, BatchDownsampler.rawDatasetRef, 0, queryFilters, AllChunkScan) + val exec = MultiSchemaPartitionsExec(QueryContext(sampleLimit = 1000), InProcessPlanDispatcher, + batchDownsampler.rawDatasetRef, 0, queryFilters, AllChunkScan) - val queryConfig = new QueryConfig(DownsamplerSettings.filodbConfig.getConfig("query")) + val queryConfig = new QueryConfig(settings.filodbConfig.getConfig("query")) val queryScheduler = Scheduler.fixedPool(s"$QuerySchedName", 3) - val res = exec.execute(downsampleTSStore, queryConfig)(queryScheduler, 1 minute) - .runAsync(queryScheduler).futureValue.asInstanceOf[QueryResult] + val res = exec.execute(downsampleTSStore, queryConfig)(queryScheduler) + .runAsync(queryScheduler).futureValue.asInstanceOf[QueryResult] queryScheduler.shutdown() res.result.size shouldEqual 1 @@ -553,4 +662,53 @@ class DownsamplerMainSpec extends FunSpec with Matchers with BeforeAndAfterAll w } } + + it("should bring up DownsampledTimeSeriesShard and NOT be able to read untyped data using SelectRawPartitionsExec") { + + val downsampleTSStore = new DownsampledTimeSeriesStore(downsampleColStore, rawColStore, + settings.filodbConfig) + + downsampleTSStore.setup(batchDownsampler.rawDatasetRef, settings.filodbSettings.schemas, + 0, rawDataStoreConfig, settings.rawDatasetIngestionConfig.downsampleConfig) + + downsampleTSStore.recoverIndex(batchDownsampler.rawDatasetRef, 0).futureValue + + val colFilters = seriesTags.map { case (t, v) => ColumnFilter(t.toString, Equals(v.toString)) }.toSeq + + val queryFilters = colFilters :+ ColumnFilter("_metric_", Equals(untypedName)) + val exec = MultiSchemaPartitionsExec(QueryContext(sampleLimit = 1000), InProcessPlanDispatcher, + batchDownsampler.rawDatasetRef, 0, queryFilters, AllChunkScan) + + val queryConfig = new QueryConfig(settings.filodbConfig.getConfig("query")) + val queryScheduler = Scheduler.fixedPool(s"$QuerySchedName", 3) + val res = exec.execute(downsampleTSStore, queryConfig)(queryScheduler) + .runAsync(queryScheduler).futureValue.asInstanceOf[QueryResult] + queryScheduler.shutdown() + + res.result.size shouldEqual 0 + } + + it("should bring up DownsampledTimeSeriesShard and be able to read specific columns " + + "from gauge using MultiSchemaPartitionsExec") { + + val downsampleTSStore = new DownsampledTimeSeriesStore(downsampleColStore, rawColStore, + settings.filodbConfig) + downsampleTSStore.setup(batchDownsampler.rawDatasetRef, settings.filodbSettings.schemas, + 0, rawDataStoreConfig, settings.rawDatasetIngestionConfig.downsampleConfig) + downsampleTSStore.recoverIndex(batchDownsampler.rawDatasetRef, 0).futureValue + val colFilters = seriesTags.map { case (t, v) => ColumnFilter(t.toString, Equals(v.toString)) }.toSeq + val queryFilters = colFilters :+ ColumnFilter("_metric_", Equals(gaugeName)) + val exec = MultiSchemaPartitionsExec(QueryContext(sampleLimit = 1000), InProcessPlanDispatcher, + batchDownsampler.rawDatasetRef, 0, queryFilters, AllChunkScan, + colName = Option("sum")) + val queryConfig = new QueryConfig(settings.filodbConfig.getConfig("query")) + val queryScheduler = Scheduler.fixedPool(s"$QuerySchedName", 3) + val res = exec.execute(downsampleTSStore, queryConfig)(queryScheduler) + .runAsync(queryScheduler).futureValue.asInstanceOf[QueryResult] + 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)) + } + } diff --git a/spark-jobs/src/test/scala/filodb/repair/ChunkCopierSpec.scala b/spark-jobs/src/test/scala/filodb/repair/ChunkCopierSpec.scala new file mode 100644 index 0000000000..bbde629696 --- /dev/null +++ b/spark-jobs/src/test/scala/filodb/repair/ChunkCopierSpec.scala @@ -0,0 +1,60 @@ +package filodb.repair + +import org.apache.spark.SparkConf +import org.scalatest.{BeforeAndAfterAll, FunSpec, Matchers} +import org.scalatest.concurrent.ScalaFutures +import org.scalatest.time.{Millis, Seconds, Span} +import com.typesafe.config.ConfigFactory + +import filodb.cassandra.DefaultFiloSessionProvider +import filodb.cassandra.columnstore.CassandraColumnStore +import filodb.core.GlobalConfig +import filodb.core.metadata.{Dataset, Schemas} + +class ChunkCopierSpec extends FunSpec with Matchers with BeforeAndAfterAll with ScalaFutures { + implicit val defaultPatience = + PatienceConfig(timeout = Span(15, Seconds), interval = Span(250, Millis)) + + implicit val s = monix.execution.Scheduler.Implicits.global + + val configPath = "conf/timeseries-filodb-server.conf" + + val sysConfig = GlobalConfig.systemConfig.getConfig("filodb") + val config = ConfigFactory.parseFile(new java.io.File(configPath)) + .getConfig("filodb").withFallback(sysConfig) + + lazy val session = new DefaultFiloSessionProvider(config.getConfig("cassandra")).session + lazy val colStore = new CassandraColumnStore(config, s, session) + + it ("should run a simple Spark job") { + // This test verifies that the configuration can be read and that Spark runs. A full test + // that verifies chunks are copied correctly is found in CassandraColumnStoreSpec. + + val dataset = Dataset("source", Schemas.gauge) + val targetDataset = Dataset("target", Schemas.gauge) + + colStore.initialize(dataset.ref, 1).futureValue + colStore.truncate(dataset.ref, 1).futureValue + + colStore.initialize(targetDataset.ref, 1).futureValue + colStore.truncate(targetDataset.ref, 1).futureValue + + val sparkConf = new SparkConf(loadDefaults = true) + sparkConf.setMaster("local[2]") + + sparkConf.set("spark.filodb.chunkcopier.source.configFile", configPath) + sparkConf.set("spark.filodb.chunkcopier.source.dataset", "source") + + sparkConf.set("spark.filodb.chunkcopier.target.configFile", configPath) + sparkConf.set("spark.filodb.chunkcopier.target.dataset", "target") + + sparkConf.set("spark.filodb.chunkcopier.ingestionTimeStart", "2020-02-26T13:00:00Z") + sparkConf.set("spark.filodb.chunkcopier.ingestionTimeEnd", "2020-02-26T23:00:00Z") + sparkConf.set("spark.filodb.chunkcopier.diskTimeToLive", "7d") + + ChunkCopierMain.run(sparkConf).close() + + colStore.truncate(dataset.ref, 1).futureValue + colStore.truncate(targetDataset.ref, 1).futureValue + } +} diff --git a/standalone/src/multi-jvm/scala/filodb/standalone/StandaloneMultiJvmSpec.scala b/standalone/src/multi-jvm/scala/filodb/standalone/StandaloneMultiJvmSpec.scala index 1ad3f1b35e..3835e5cf2b 100644 --- a/standalone/src/multi-jvm/scala/filodb/standalone/StandaloneMultiJvmSpec.scala +++ b/standalone/src/multi-jvm/scala/filodb/standalone/StandaloneMultiJvmSpec.scala @@ -24,8 +24,9 @@ import filodb.core.DatasetRef import filodb.core.store.StoreConfig import filodb.prometheus.ast.TimeStepParams import filodb.prometheus.parse.Parser -import filodb.query.{QueryError, QueryResult => QueryResult2} +import filodb.query.{QueryError, QueryResult => QueryResult2, Sampl} import filodb.query.PromCirceSupport +import filodb.query.Sampl import filodb.query.SuccessResponse /** @@ -204,7 +205,7 @@ abstract class StandaloneMultiJvmSpec(config: MultiNodeConfig) extends MultiNode val url = uri"http://localhost:8080/promql/prometheus/api/v1/query?query=$query&time=${queryTimestamp/1000}" info(s"Querying: $url") val result1 = sttp.get(url).response(asJson[SuccessResponse]).send().futureValue.unsafeBody.right.get.data.result - val result = result1.flatMap(_.values.get.map { d => (d.timestamp, d.value) }) + val result = result1.flatMap(_.values.get.collect { case d: Sampl => (d.timestamp, d.value) }) info(s"result values were $result") result.length should be > 0 val sum = result.map(_._2).sum diff --git a/version.sbt b/version.sbt index 12be1a9106..1d2b815284 100644 --- a/version.sbt +++ b/version.sbt @@ -1 +1 @@ -version in ThisBuild := "0.9.4" \ No newline at end of file +version in ThisBuild := "0.9.5"