From 708a6c88f73937f858699fdbabf30ef52961b4d3 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Wed, 27 Feb 2019 22:44:42 -0800 Subject: [PATCH 01/41] feat(memory, core, query): Initial end to end HistogramColumn (#211) A big feature: a HistogramColumn column type which stores entire histograms together for storage efficiency and much faster queries --- README.md | 11 +- cli/src/main/scala/filodb.cli/CliMain.scala | 4 +- .../queryengine2/QueryEngine.scala | 8 +- .../FilodbClusterNodeSpec.scala | 30 +- .../ShardSubscriptionsSpec.scala | 4 +- core/src/main/scala/filodb.core/KeyType.scala | 8 +- .../filodb.core/binaryrecord/FieldType.scala | 1 - .../binaryrecord2/RecordBuilder.scala | 52 ++- .../binaryrecord2/RecordComparator.scala | 25 +- .../binaryrecord2/RecordSchema.scala | 48 ++- .../scala/filodb.core/memstore/MemStore.scala | 2 + .../memstore/TimeSeriesPartition.scala | 21 +- .../memstore/TimeSeriesShard.scala | 5 +- .../scala/filodb.core/metadata/Column.scala | 7 +- .../metadata/SimpleComputations.scala | 2 +- .../query/PartitionTimeRangeReader.scala | 3 +- .../filodb.core/store/ChunkSetInfo.scala | 48 ++- core/src/test/resources/application_test.conf | 1 + .../src/test/scala/filodb.core/TestData.scala | 45 +++ .../binaryrecord/BinaryRecordSpec.scala | 6 +- .../binaryrecord2/BinaryRecordSpec.scala | 36 +- .../downsample/ShardDownsamplerSpec.scala | 10 +- .../memstore/TimeSeriesMemStoreSpec.scala | 31 +- doc/FAQ.md | 9 + doc/binaryrecord-spec.md | 36 +- doc/compression.md | 20 +- doc/downsampling.md | 12 + doc/indexing.md | 6 +- doc/mermaid/chunked-iteration.mermaid | 15 + doc/mermaid/chunked-iteration.png | Bin 0 -> 43199 bytes doc/query-engine.md | 16 +- doc/sharding.md | 1 + doc/spark-oldold.md | 13 + .../filodb.jmh/HistogramIngestBenchmark.scala | 120 ++++++ .../filodb.jmh/HistogramQueryBenchmark.scala | 126 ++++++ .../scala/filodb.jmh/SparkReadBenchmark.scala | 120 ------ .../scala/filodb.memory/BinaryRegion.scala | 2 +- .../src/main/scala/filodb.memory/Block.scala | 3 + .../filodb.memory/format/BinaryVector.scala | 5 + .../filodb.memory/format/RowReader.scala | 26 ++ .../format/RowToVectorBuilder.scala | 1 + .../filodb.memory/format/UnsafeUtils.scala | 10 + .../format/vectors/Histogram.scala | 273 +++++++++++++ .../format/vectors/HistogramCompressor.scala | 66 ++++ .../format/vectors/HistogramVector.scala | 370 ++++++++++++++++++ .../format/vectors/IntBinaryVector.scala | 4 +- .../format/vectors/HistogramTest.scala | 91 +++++ .../format/vectors/HistogramVectorTest.scala | 111 ++++++ .../format/vectors/IntBinaryVectorTest.scala | 23 ++ .../query/exec/HistogramQuantileMapper.scala | 41 +- .../query/exec/PeriodicSamplesMapper.scala | 78 ++-- .../query/exec/RangeVectorTransformer.scala | 81 +++- .../filodb/query/exec/TransientRow.scala | 35 +- .../exec/rangefn/AggrOverTimeFunctions.scala | 45 ++- .../query/exec/rangefn/InstantFunction.scala | 83 ++-- .../query/exec/rangefn/RangeFunction.scala | 159 ++++---- .../query/exec/LastSampleFunctionSpec.scala | 12 +- .../exec/SelectRawPartitionsExecSpec.scala | 47 ++- .../query/exec/WindowIteratorSpec.scala | 79 ++-- .../rangefn/AggrOverTimeFunctionsSpec.scala | 62 ++- .../exec/rangefn/InstantFunctionSpec.scala | 35 +- .../scala/filodb.spark/TypeConverters.scala | 1 - 62 files changed, 2152 insertions(+), 493 deletions(-) create mode 100644 doc/mermaid/chunked-iteration.mermaid create mode 100644 doc/mermaid/chunked-iteration.png create mode 100644 jmh/src/main/scala/filodb.jmh/HistogramIngestBenchmark.scala create mode 100644 jmh/src/main/scala/filodb.jmh/HistogramQueryBenchmark.scala delete mode 100644 jmh/src/main/scala/filodb.jmh/SparkReadBenchmark.scala create mode 100644 memory/src/main/scala/filodb.memory/format/vectors/Histogram.scala create mode 100644 memory/src/main/scala/filodb.memory/format/vectors/HistogramCompressor.scala create mode 100644 memory/src/main/scala/filodb.memory/format/vectors/HistogramVector.scala create mode 100644 memory/src/test/scala/filodb.memory/format/vectors/HistogramTest.scala create mode 100644 memory/src/test/scala/filodb.memory/format/vectors/HistogramVectorTest.scala diff --git a/README.md b/README.md index aae0b52e78..0524c30133 100644 --- a/README.md +++ b/README.md @@ -29,7 +29,7 @@ and [filodb-discuss](https://groups.google.com/forum/#!forum/filodb-discuss) goo - [Getting Started](#getting-started) - [End to End Kafka Developer Setup](#end-to-end-kafka-developer-setup) - [Using the Gateway to stream Application Metrics](#using-the-gateway-to-stream-application-metrics) - - [Multiple Servers](#multiple-servers) + - [Multiple Servers using Consul](#multiple-servers-using-consul) - [Local Scale Testing](#local-scale-testing) - [Understanding the FiloDB Data Model](#understanding-the-filodb-data-model) - [Prometheus FiloDB Schema for Operational Metrics](#prometheus-filodb-schema-for-operational-metrics) @@ -38,6 +38,7 @@ and [filodb-discuss](https://groups.google.com/forum/#!forum/filodb-discuss) goo - [Sharding](#sharding) - [Querying FiloDB](#querying-filodb) - [FiloDB PromQL Extensions](#filodb-promql-extensions) + - [First-Class Histogram Support](#first-class-histogram-support) - [Using the FiloDB HTTP API](#using-the-filodb-http-api) - [Grafana setup](#grafana-setup) - [Using the CLI](#using-the-cli) @@ -383,6 +384,14 @@ Example of debugging chunk metadata using the CLI: ./filo-cli --host 127.0.0.1 --dataset prometheus --promql '_filodb_chunkmeta_all(heap_usage{_ns="App-0"})' --start XX --end YY +### First-Class Histogram Support + +One major difference FiloDB has from the Prometheus data model is that FiloDB supports histograms as a first-class entity. In Prometheus, histograms are stored with each bucket in its own time series differentiated by the `le` tag. In FiloDB, there is a `HistogramColumn` which stores all the buckets together for significantly improved compression, especially over the wire during ingestion, as well as significantly faster query speeds. Here are the differences users need to be aware of when using `HistogramColumn`: + +* There is no need to append `_bucket` to the metric name. +* However, you need to select the histogram column like `__col__="hist"` +* TODO: document ways of selecting the histogram bucket + ### Using the FiloDB HTTP API Please see the [HTTP API](doc/http_api.md) doc. diff --git a/cli/src/main/scala/filodb.cli/CliMain.scala b/cli/src/main/scala/filodb.cli/CliMain.scala index e8de300f98..790909cfb4 100644 --- a/cli/src/main/scala/filodb.cli/CliMain.scala +++ b/cli/src/main/scala/filodb.cli/CliMain.scala @@ -408,11 +408,11 @@ object CliMain extends ArgMain[Arguments] with CsvImportExport with FilodbCluste case LongColumn => rowReader.getLong(position).toString case DoubleColumn => rowReader.getDouble(position).toString case StringColumn => rowReader.filoUTF8String(position).toString - case BitmapColumn => rowReader.getBoolean(position).toString case TimestampColumn => rowReader.as[Timestamp](position).toString + case HistogramColumn => rowReader.getHistogram(position).toString case _ => throw new UnsupportedOperationException("Unsupported type: " + columns(position).columnType) } - content.update(position,value) + content.update(position, value) position += 1 } content diff --git a/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala b/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala index 2c9682a842..61906f5508 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala @@ -19,7 +19,6 @@ import filodb.core.metadata.Dataset import filodb.core.query.{ColumnFilter, Filter} import filodb.prometheus.ast.Vectors.PromMetricLabel import filodb.query.{exec, _} -import filodb.query.InstantFunctionId.HistogramQuantile import filodb.query.exec._ /** @@ -165,12 +164,7 @@ class QueryEngine(dataset: Dataset, options: QueryOptions, lp: ApplyInstantFunction): Seq[ExecPlan] = { val vectors = walkLogicalPlanTree(lp.vectors, queryId, submitTime, options) - lp.function match { - case HistogramQuantile => - vectors.foreach(_.addRangeVectorTransformer(HistogramQuantileMapper(lp.functionArgs))) - case _ => - vectors.foreach(_.addRangeVectorTransformer(InstantVectorFunctionMapper(lp.function, lp.functionArgs))) - } + vectors.foreach(_.addRangeVectorTransformer(InstantVectorFunctionMapper(lp.function, lp.functionArgs))) vectors } diff --git a/coordinator/src/test/scala/filodb.coordinator/FilodbClusterNodeSpec.scala b/coordinator/src/test/scala/filodb.coordinator/FilodbClusterNodeSpec.scala index 98503377b5..03fb8001c9 100644 --- a/coordinator/src/test/scala/filodb.coordinator/FilodbClusterNodeSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/FilodbClusterNodeSpec.scala @@ -1,5 +1,7 @@ package filodb.coordinator +import java.net.Socket + import akka.testkit.TestProbe import com.typesafe.config.{Config, ConfigFactory} import org.scalatest.concurrent.ScalaFutures @@ -8,6 +10,29 @@ import org.scalatest.time.{Millis, Seconds, Span} import filodb.coordinator.client.MiscCommands import filodb.core.{AbstractSpec, Success} +trait SocketChecker { + def waitSocketOpen(port: Int): Unit = { + while (!isPortAvail(port)) { + println(s"Port $port is not available, waiting and retrying...") + Thread sleep 1000 + } + } + + def isPortAvail(port: Int): Boolean = { + var s: Socket = null + try { + s = new Socket("localhost", port) + // If the code makes it this far without an exception it means + // something is using the port and has responded. + return false + } catch { + case e: Exception => return true; + } finally { + if (s != null) s.close() + } + } +} + trait FilodbClusterNodeSpec extends AbstractSpec with FilodbClusterNode with ScalaFutures { // Ensure that CoordinatedShutdown does not shutdown the whole test JVM, otherwise Travis CI/CD fails override protected lazy val roleConfig = ConfigFactory.parseString( @@ -108,8 +133,7 @@ class ClusterNodeServerSpec extends FilodbClusterNodeSpec { * Initiates cluster singleton recovery sequence by populating guardian with some initial non-empty * shardmap and subscription state, and checking that it is recovered properly on startup */ -class ClusterNodeRecoverySpec extends FilodbClusterNodeSpec { - +class ClusterNodeRecoverySpec extends FilodbClusterNodeSpec with SocketChecker { import scala.collection.immutable import scala.concurrent.duration._ @@ -121,6 +145,8 @@ class ClusterNodeRecoverySpec extends FilodbClusterNodeSpec { import NodeClusterActor._ import NodeProtocol._ + waitSocketOpen(2552) + override val role = ClusterRole.Server override protected lazy val roleConfig: Config = AkkaSpec.settings.allConfig diff --git a/coordinator/src/test/scala/filodb.coordinator/ShardSubscriptionsSpec.scala b/coordinator/src/test/scala/filodb.coordinator/ShardSubscriptionsSpec.scala index 6f4b1ed8f6..ef0cf90a8b 100644 --- a/coordinator/src/test/scala/filodb.coordinator/ShardSubscriptionsSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/ShardSubscriptionsSpec.scala @@ -4,7 +4,7 @@ import akka.testkit.TestProbe import filodb.core.DatasetRef -class ShardSubscriptionsSpec extends AkkaSpec { +class ShardSubscriptionsSpec extends AkkaSpec with SocketChecker { private val extension = FilodbCluster(system) @@ -13,6 +13,8 @@ class ShardSubscriptionsSpec extends AkkaSpec { private val subscribers = Set(TestProbe().ref, TestProbe().ref) + waitSocketOpen(2552) + "ShardSubscription" must { "add unseen subscribers, not add seen subscribers" in { var subscription = ShardSubscription(dataset1, Set.empty) diff --git a/core/src/main/scala/filodb.core/KeyType.scala b/core/src/main/scala/filodb.core/KeyType.scala index 9b2c3ce2b4..6ab1c7af2c 100644 --- a/core/src/main/scala/filodb.core/KeyType.scala +++ b/core/src/main/scala/filodb.core/KeyType.scala @@ -8,7 +8,7 @@ import scala.math.Ordering import org.joda.time.DateTime import scalaxy.loops._ -import filodb.memory.format.{RowReader, ZeroCopyUTF8String} +import filodb.memory.format.{vectors => bv, RowReader, ZeroCopyUTF8String} import filodb.memory.format.RowReader._ /** @@ -177,4 +177,10 @@ object SingleKeyTypes { } } } + + // Order histograms by top bucket value + implicit val histOrdering = Ordering.by((h: bv.Histogram) => h.bucketValue(h.numBuckets - 1)) + implicit case object HistogramKeyType extends SingleKeyTypeBase[bv.Histogram] { + def fromString(str: String): bv.Histogram = ??? + } } diff --git a/core/src/main/scala/filodb.core/binaryrecord/FieldType.scala b/core/src/main/scala/filodb.core/binaryrecord/FieldType.scala index 173e802a36..b29b2ff959 100644 --- a/core/src/main/scala/filodb.core/binaryrecord/FieldType.scala +++ b/core/src/main/scala/filodb.core/binaryrecord/FieldType.scala @@ -64,7 +64,6 @@ object FieldType { IntColumn -> IntFieldType, LongColumn -> LongFieldType, StringColumn -> UTF8StringFieldType, - BitmapColumn -> BooleanFieldType, DoubleColumn -> DoubleFieldType, TimestampColumn -> LongFieldType, // default to long handling due to BinaryRecord MapColumn -> UTF8MapFieldType diff --git a/core/src/main/scala/filodb.core/binaryrecord2/RecordBuilder.scala b/core/src/main/scala/filodb.core/binaryrecord2/RecordBuilder.scala index 35daf99efd..808495336e 100644 --- a/core/src/main/scala/filodb.core/binaryrecord2/RecordBuilder.scala +++ b/core/src/main/scala/filodb.core/binaryrecord2/RecordBuilder.scala @@ -1,6 +1,7 @@ package filodb.core.binaryrecord2 import com.typesafe.scalalogging.StrictLogging +import org.agrona.concurrent.UnsafeBuffer import scalaxy.loops._ import filodb.core.metadata.{Column, Dataset} @@ -8,6 +9,7 @@ import filodb.core.metadata.Column.ColumnType.{DoubleColumn, LongColumn, MapColu import filodb.core.query.ColumnInfo import filodb.memory._ import filodb.memory.format.{RowReader, SeqRowReader, UnsafeUtils, ZeroCopyUTF8String => ZCUTF8} +import filodb.memory.format.vectors.Histogram // scalastyle:off number.of.methods @@ -134,6 +136,16 @@ final class RecordBuilder(memFactory: MemFactory, fieldNo += 1 } + final def addBlob(strPtr: ZCUTF8): Unit = addBlob(strPtr.base, strPtr.offset, strPtr.numBytes) + + // Adds a blob from another buffer which already has the length bytes as the first two bytes + // For example: buffers created by BinaryHistograms. OR, a UTF8String medium. + final def addBlob(buf: UnsafeBuffer): Unit = { + val numBytes = buf.getShort(0).toInt + require(numBytes < buf.capacity) + addBlob(buf.byteArray, buf.addressOffset + 2, numBytes) + } + /** * IMPORTANT: Internal method, does not update hash values for the map key/values individually. * If this method is used, then caller needs to also update the partitionHash manually. @@ -191,8 +203,6 @@ final class RecordBuilder(memFactory: MemFactory, recHash = partKeySchema.partitionHash(base, offset) } - final def addBlob(strPtr: ZCUTF8): Unit = addBlob(strPtr.base, strPtr.offset, strPtr.numBytes) - import Column.ColumnType._ /** @@ -209,11 +219,8 @@ final class RecordBuilder(memFactory: MemFactory, case (StringColumn, s: String) => addString(s) case (StringColumn, a: Array[Byte]) => addString(a) case (StringColumn, z: ZCUTF8) => addBlob(z) - case (MapColumn, m: Map[ZCUTF8, ZCUTF8] @unchecked) => - val pairs = new java.util.ArrayList[(String, String)] - m.toSeq.foreach { case (k, v) => pairs.add((k.toString, v.toString)) } - val hashes = sortAndComputeHashes(pairs) - addSortedPairsAsMap(pairs, hashes) + case (MapColumn, m: Map[ZCUTF8, ZCUTF8] @unchecked) => addMap(m) + case (HistogramColumn, h: Histogram) => addBlob(h.serialize()) case (other: Column.ColumnType, v) => throw new UnsupportedOperationException(s"Column type of $other and value of class ${v.getClass}") } @@ -235,17 +242,12 @@ final class RecordBuilder(memFactory: MemFactory, def addFromObjects(parts: Any*): Long = addFromReader(SeqRowReader(parts.toSeq)) /** - * High level function to add a sorted list of unique key-value pairs as a map in the BinaryRecord. - * @param sortedPairs sorted list of key-value pairs, as modified by sortAndComputeHashes - * @param hashes an array of hashes, one for each k-v pair, as returned by sortAndComputeHashes + * Sorts and adds keys and values from a map. The easiest way to add a map to a BinaryRecord. */ - def addSortedPairsAsMap(sortedPairs: java.util.ArrayList[(String, String)], - hashes: Array[Int]): Unit = { + def addMap(map: Map[ZCUTF8, ZCUTF8]): Unit = { startMap() - for { i <- 0 until sortedPairs.size optimized } { - val (k, v) = sortedPairs.get(i) - addMapKeyValue(k.getBytes, v.getBytes) - recHash = combineHash(recHash, hashes(i)) + map.toSeq.sortBy(_._1).foreach { case (k, v) => + addMapKeyValue(k.bytes, v.bytes) } endMap() } @@ -312,6 +314,7 @@ final class RecordBuilder(memFactory: MemFactory, * An alternative to above for adding a known key with precomputed key hash * along with a value, to the map, while updating the hash too. * Saves computing the key hash twice. + * TODO: deprecate this. We are switching to computing a hash for all keys at the same time. */ final def addMapKeyValueHash(keyBytes: Array[Byte], keyHash: Int, valueBytes: Array[Byte], valueOffset: Int, valueLen: Int): Unit = { @@ -321,9 +324,11 @@ final class RecordBuilder(memFactory: MemFactory, } /** - * Ends creation of a map field + * Ends creation of a map field. Recompute the hash for all fields at once. */ final def endMap(): Unit = { + val mapHash = BinaryRegion.hash32(curBase, mapOffset, (curRecEndOffset - mapOffset).toInt) + updatePartitionHash(mapHash) mapOffset = -1L fieldNo += 1 } @@ -359,7 +364,7 @@ final class RecordBuilder(memFactory: MemFactory, * Used only internally by RecordComparator etc. to shortcut create a new BR by copying bytes from an existing BR. * You BETTER know what you are doing. */ - private[binaryrecord2] def copyNewRecordFrom(base: Any, offset: Long, numBytes: Int): Unit = { + private[binaryrecord2] def copyFixedAreasFrom(base: Any, offset: Long, numBytes: Int): Unit = { require(curRecEndOffset == curRecordOffset, s"Illegal state: $curRecEndOffset != $curRecordOffset") requireBytes(numBytes + 4) @@ -369,6 +374,15 @@ final class RecordBuilder(memFactory: MemFactory, curRecEndOffset = curRecordOffset + numBytes + 4 } + // Extend current variable area with stuff from somewhere else + private[binaryrecord2] def copyVarAreasFrom(base: Any, offset: Long, numBytes: Int): Unit = { + requireBytes(numBytes) + UnsafeUtils.unsafe.copyMemory(base, offset, curBase, curRecEndOffset, numBytes) + // Increase length of current BR. Then bump curRecEndOffset so we are consistent + setInt(curBase, curRecordOffset, getInt(curBase, curRecordOffset) + numBytes) + curRecEndOffset += numBytes + } + private[binaryrecord2] def adjustFieldOffset(fieldNo: Int, adjustment: Int): Unit = { val offset = curRecordOffset + schema.fieldOffset(fieldNo) UnsafeUtils.setInt(curBase, offset, UnsafeUtils.getInt(curBase, offset) + adjustment) @@ -464,7 +478,7 @@ final class RecordBuilder(memFactory: MemFactory, curRecEndOffset = curRecordOffset + recordNumBytes } - private def newContainer(): Unit = { + private[filodb] def newContainer(): Unit = { val (newBase, newOff, _) = memFactory.allocate(containerSize) val container = new RecordContainer(newBase, newOff, containerSize) containers += container diff --git a/core/src/main/scala/filodb.core/binaryrecord2/RecordComparator.scala b/core/src/main/scala/filodb.core/binaryrecord2/RecordComparator.scala index 95bf14f7e0..bab5c697f0 100644 --- a/core/src/main/scala/filodb.core/binaryrecord2/RecordComparator.scala +++ b/core/src/main/scala/filodb.core/binaryrecord2/RecordComparator.scala @@ -28,8 +28,7 @@ final class RecordComparator(ingestSchema: RecordSchema) { // the ingest BR offset of the first partition fixed area field private final val ingestPartOffset = ingestSchema.fieldOffset(ingestSchema.partitionFieldStart.get) - // the offsets of the variable areas for ingest and partition keys (from beginning of BR) - private final val ingestVarAreaOffset = ingestSchema.variableAreaStart + // the offsets of the variable areas for partition keys (from beginning of BR) private final val partVarAreaOffset = partitionKeySchema.variableAreaStart // The number of bytes of the partition fields fixed area @@ -53,6 +52,21 @@ final class RecordComparator(ingestSchema: RecordSchema) { private final val anyPrimitiveFieldsToCompare = compareBitmap != 0 + // Find the fixed area of the first non-primitive (variable area pointer) partition key field + private val ingestPartNonPrimOffset = + if (!anyPrimitiveFieldsToCompare) ingestPartOffset else { + // NOTE: this algorithm fails if the partition key has all primitive fields. This should not happen. + var nonPrimOffset = ingestPartOffset + var wordsBitmap = compareBitmap + while ((wordsBitmap & 0x01) == 1) { nonPrimOffset += 4; wordsBitmap >>= 1 } + nonPrimOffset + } + + // Finds the offset from beginning of ingest record to the var area of first partition field. + // We have to take into account that data columns might have variable areas too. + private def ingestVarOffset(ingestBase: Any, ingestOffset: Long): Int = + UnsafeUtils.getInt(ingestBase, ingestOffset + ingestPartNonPrimOffset) + /** * Returns true if the partition part of the ingest BinaryRecord matches (all partition fields match identically) * to the partition key BR. Uses optimized byte comparisons which is faster than field by field comparison. @@ -67,6 +81,7 @@ final class RecordComparator(ingestSchema: RecordSchema) { val partKeyNumBytes = UnsafeUtils.getInt(partKeyBase, partKeyOffset) // compare lengths of variable areas (map tags & strings) + val ingestVarAreaOffset = ingestVarOffset(ingestBase, ingestOffset) val ingestVarSize = ingestNumBytes + 4 - ingestVarAreaOffset val partKeyVarSize = partKeyNumBytes + 4 - partVarAreaOffset if (ingestVarSize != partKeyVarSize) return false @@ -103,9 +118,11 @@ final class RecordComparator(ingestSchema: RecordSchema) { final def buildPartKeyFromIngest(ingestBase: Any, ingestOffset: Long, builder: RecordBuilder): Long = { require(builder.schema == partitionKeySchema, s"${builder.schema} is not part key schema $partitionKeySchema") - // Copy the entire fixed + hash + variable sized areas over + // Copy fixed area + hash over, then variable areas val ingestNumBytes = UnsafeUtils.getInt(ingestBase, ingestOffset) - builder.copyNewRecordFrom(ingestBase, ingestOffset + ingestPartOffset, ingestNumBytes + 4 - ingestPartOffset) + val ingestVarAreaOffset = ingestVarOffset(ingestBase, ingestOffset) + builder.copyFixedAreasFrom(ingestBase, ingestOffset + ingestPartOffset, fixedAreaNumBytes + 4) + builder.copyVarAreasFrom(ingestBase, ingestOffset + ingestVarAreaOffset, ingestNumBytes + 4 - ingestVarAreaOffset) // adjust offsets to var fields val adjustment = partVarAreaOffset - ingestVarAreaOffset diff --git a/core/src/main/scala/filodb.core/binaryrecord2/RecordSchema.scala b/core/src/main/scala/filodb.core/binaryrecord2/RecordSchema.scala index 9dd312ab16..ff7a01161f 100644 --- a/core/src/main/scala/filodb.core/binaryrecord2/RecordSchema.scala +++ b/core/src/main/scala/filodb.core/binaryrecord2/RecordSchema.scala @@ -2,6 +2,8 @@ package filodb.core.binaryrecord2 import scala.collection.mutable.ArrayBuffer +import org.agrona.concurrent.UnsafeBuffer + import filodb.core.metadata.{Column, Dataset} import filodb.core.metadata.Column.ColumnType.{LongColumn, TimestampColumn} import filodb.core.query.ColumnInfo @@ -60,14 +62,16 @@ final class RecordSchema(val columns: Seq[ColumnInfo], // Typed, efficient functions, one for each field/column, to add to a RecordBuilder efficiently from a RowReader // with no boxing or extra allocations involved val builderAdders = columnTypes.zipWithIndex.map { - case (Column.ColumnType.IntColumn, colNo) => - (row: RowReader, builder: RecordBuilder) => builder.addInt(row.getInt(colNo)) case (Column.ColumnType.LongColumn, colNo) => (row: RowReader, builder: RecordBuilder) => builder.addLong(row.getLong(colNo)) case (Column.ColumnType.TimestampColumn, colNo) => (row: RowReader, builder: RecordBuilder) => builder.addLong(row.getLong(colNo)) case (Column.ColumnType.DoubleColumn, colNo) => (row: RowReader, builder: RecordBuilder) => builder.addDouble(row.getDouble(colNo)) + case (Column.ColumnType.HistogramColumn, colNo) => + (row: RowReader, builder: RecordBuilder) => builder.addBlob(row.getHistogram(colNo).serialize()) + case (Column.ColumnType.IntColumn, colNo) => + (row: RowReader, builder: RecordBuilder) => builder.addInt(row.getInt(colNo)) case (Column.ColumnType.StringColumn, colNo) => // TODO: we REALLY need a better API than ZeroCopyUTF8String as it creates so much garbage (row: RowReader, builder: RecordBuilder) => builder.addBlob(row.filoUTF8String(colNo)) @@ -139,6 +143,26 @@ final class RecordSchema(val columns: Seq[ColumnInfo], def blobNumBytes(base: Any, offset: Long, index: Int): Int = UTF8StringMedium.numBytes(base, offset + UnsafeUtils.getInt(base, offset + offsets(index))) + /** + * Sets an existing UnsafeBuffer to wrap around the given blob/UTF8/Histogram bytes, including the + * 2-byte length prefix. Since the UnsafeBuffer is already allocated, this results in no new allocations. + * Could be used to efficiently retrieve blobs or histograms again and again. + */ + def blobAsBuffer(base: Any, offset: Long, index: Int, buf: UnsafeBuffer): Unit = base match { + case a: Array[Byte] => + buf.wrap(a, utf8StringOffset(base, offset, index).toInt - UnsafeUtils.arayOffset, + blobNumBytes(base, offset, index) + 2) + case UnsafeUtils.ZeroPointer => + buf.wrap(utf8StringOffset(base, offset, index), blobNumBytes(base, offset, index) + 2) + } + + // Same as above but allocates a new UnsafeBuffer wrapping the blob as a reference + def blobAsBuffer(base: Any, offset: Long, index: Int): UnsafeBuffer = { + val newBuf = new UnsafeBuffer(Array.empty[Byte]) + blobAsBuffer(base, offset, index, newBuf) + newBuf + } + /** * Used for extracting the offset for a UTF8StringMedium. * Note that blobOffset method is for the offset to the actual blob bytes, not including length header. @@ -173,11 +197,12 @@ final class RecordSchema(val columns: Seq[ColumnInfo], case (DoubleColumn, i) => result += s"${colNames(i)}= ${getDouble(base, offset, i)}" case (StringColumn, i) => result += s"${colNames(i)}= ${asJavaString(base, offset, i)}" case (TimestampColumn, i) => result += s"${colNames(i)}= ${getLong(base, offset, i)}" - case (BitmapColumn, i) => result += s"${colNames(i)}= ${getInt(base, offset, i) != 0}" case (MapColumn, i) => val consumer = new StringifyMapItemConsumer consumeMapItems(base, offset, i, consumer) result += s"${colNames(i)}= ${consumer.prettyPrint}" case (BinaryRecordColumn, i) => result += s"${colNames(i)}= ${brSchema(i).stringify(base, offset)}" + case (HistogramColumn, i) => + result += s"${colNames(i)}= ${bv.BinaryHistogram.BinHistogram(blobAsBuffer(base, offset, i))}" } s"b2[${result.mkString(",")}]" } @@ -197,11 +222,12 @@ final class RecordSchema(val columns: Seq[ColumnInfo], case (DoubleColumn, i) => resultMap += ((colNames(i), getDouble(base, offset, i).toString)) case (StringColumn, i) => resultMap += ((colNames(i), asJavaString(base, offset, i).toString)) case (TimestampColumn, i) => resultMap += ((colNames(i), getLong(base, offset, i).toString)) - case (BitmapColumn, i) => resultMap += ((colNames(i), (getInt(base, offset, i) != 0).toString)) case (MapColumn, i) => val consumer = new StringifyMapItemConsumer consumeMapItems(base, offset, i, consumer) resultMap ++= consumer.stringPairs case (BinaryRecordColumn, i) => resultMap ++= brSchema(i).toStringPairs(base, offset) + case (HistogramColumn, i) => + resultMap += ((colNames(i), bv.BinaryHistogram.BinHistogram(blobAsBuffer(base, offset, i)).toString)) } resultMap } @@ -332,7 +358,8 @@ object RecordSchema { TimestampColumn -> 8, // Just a long ms timestamp StringColumn -> 4, BinaryRecordColumn -> 4, - MapColumn -> 4) + MapColumn -> 4, + HistogramColumn -> 4) /** * Creates a "unique" Long key for each incoming predefined key for quick lookup. This will not be perfect @@ -395,11 +422,20 @@ final class BinaryRecordRowReader(schema: RecordSchema, def getDouble(columnNo: Int): Double = schema.getDouble(recordBase, recordOffset, columnNo) def getFloat(columnNo: Int): Float = ??? def getString(columnNo: Int): String = filoUTF8String(columnNo).toString + override def getHistogram(columnNo: Int): bv.Histogram = + bv.BinaryHistogram.BinHistogram(blobAsBuffer(columnNo)).toHistogram def getAny(columnNo: Int): Any = schema.columnTypes(columnNo).keyType.extractor.getField(this, columnNo) override def filoUTF8String(i: Int): ZeroCopyUTF8String = schema.asZCUTF8Str(recordBase, recordOffset, i) def getBlobBase(columnNo: Int): Any = schema.blobBase(recordBase, recordOffset, columnNo) def getBlobOffset(columnNo: Int): Long = schema.blobOffset(recordBase, recordOffset, columnNo) def getBlobNumBytes(columnNo: Int): Int = schema.blobNumBytes(recordBase, recordOffset, columnNo) -// def getUtf8MediumOffset(columnNo: Int): Long = schema.utf8StringOffset(recordBase, recordOffset, columnNo) + + val buf = new UnsafeBuffer(Array.empty[Byte]) + // NOTE: this method reuses the same buffer to avoid allocations. + override def blobAsBuffer(columnNo: Int): UnsafeBuffer = { + UnsafeUtils.wrapUnsafeBuf(recordBase, schema.utf8StringOffset(recordBase, recordOffset, columnNo), + getBlobNumBytes(columnNo) + 2, buf) + buf + } } \ No newline at end of file diff --git a/core/src/main/scala/filodb.core/memstore/MemStore.scala b/core/src/main/scala/filodb.core/memstore/MemStore.scala index e1c764039f..fc8e9abb70 100644 --- a/core/src/main/scala/filodb.core/memstore/MemStore.scala +++ b/core/src/main/scala/filodb.core/memstore/MemStore.scala @@ -207,6 +207,7 @@ trait MemStore extends ChunkSource { object MemStore { // TODO: make the max string vector size configurable. val MaxUTF8VectorSize = 8192 + val MaxHistogramBuckets = 64 /** * Figures out the AppendableVectors for each column, depending on type and whether it is a static/ @@ -224,6 +225,7 @@ object MemStore { case DoubleColumn => bv.DoubleVector.appendingVectorNoNA(memFactory, maxElements) case TimestampColumn => bv.LongBinaryVector.timestampVector(memFactory, maxElements) case StringColumn => bv.UTF8Vector.appendingVector(memFactory, maxElements, MaxUTF8VectorSize) + case HistogramColumn => bv.HistogramVector.appendingColumnar(memFactory, MaxHistogramBuckets, maxElements) case other: Column.ColumnType => ??? } }.toArray diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala index 198a02be6b..41e4b0a0bf 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala @@ -16,6 +16,7 @@ object TimeSeriesPartition extends StrictLogging { val nullChunks = UnsafeUtils.ZeroPointer.asInstanceOf[AppenderArray] val nullInfo = ChunkSetInfo(UnsafeUtils.ZeroPointer.asInstanceOf[BinaryRegion.NativePointer]) + // Use global logger so we can save a few fields for each of millions of TSPartitions :) val _log = logger def partKeyString(dataset: Dataset, partKeyBase: Any, partKeyOffset: Long): String = { @@ -132,8 +133,13 @@ extends ChunkMap(memFactory, initMapSize) with ReadablePartition { for { col <- 0 until numColumns optimized } { currentChunks(col).addFromReaderNoNA(row, col) match { case r: VectorTooSmall => - switchBuffers(blockHolder, encode=true) - ingest(row, blockHolder) // re-ingest every element, allocating new WriteBuffers + // NOTE: a very bad infinite loop is possible if switching buffers fails (if the # rows is 0) but one of the + // vectors fills up. This is possible if one vector fills up but the other one does not for some reason. + // So we do not call ingest again unless switcing buffers succeeds. + // re-ingest every element, allocating new WriteBuffers + if (switchBuffers(blockHolder, encode=true)) { ingest(row, blockHolder) } + else { _log.warn("EMPTY WRITEBUFFERS when switchBuffers called! Likely a severe bug!!! " + + s"Part=$stringPartition ts=$ts col=$col numRows=${currentInfo.numRows}") } return case other: AddResponse => } @@ -159,8 +165,8 @@ extends ChunkMap(memFactory, initMapSize) with ReadablePartition { * To guarantee no more writes happen when switchBuffers is called, have ingest() and switchBuffers() be * called from a single thread / single synchronous stream. */ - final def switchBuffers(blockHolder: BlockMemFactory, encode: Boolean = false): Unit = - if (nonEmptyWriteBuffers) { + final def switchBuffers(blockHolder: BlockMemFactory, encode: Boolean = false): Boolean = + nonEmptyWriteBuffers && { val oldInfo = currentInfo val oldAppenders = currentChunks @@ -171,6 +177,7 @@ extends ChunkMap(memFactory, initMapSize) with ReadablePartition { if (encode) { encodeOneChunkset(oldInfo, oldAppenders, blockHolder) } else { appenders = InfoAppenders(oldInfo, oldAppenders) :: appenders } + true } /** @@ -373,6 +380,12 @@ extends ChunkMap(memFactory, initMapSize) with ReadablePartition { private def infoPut(info: ChunkSetInfo): Unit = { chunkmapWithExclusive(chunkmapDoPut(info.infoAddr)) } + + // Free memory (esp offheap) attached to this TSPartition and return buffers to common pool + def shutdown(): Unit = { + chunkmapFree() + if (currentInfo != nullInfo) bufferPool.release(currentInfo.infoAddr, currentChunks) + } } final case class PartKeyRowReader(records: Iterator[TimeSeriesPartition]) extends Iterator[RowReader] { diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala index 5d742fdc74..fab4a6a294 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala @@ -968,7 +968,8 @@ class TimeSeriesShard(val dataset: Dataset, if (part == OutOfMemPartition) { disableAddPartitions() } else { part.asInstanceOf[TimeSeriesPartition].ingest(binRecordReader, overflowBlockFactory) } } catch { - case e: OutOfOffheapMemoryException => disableAddPartitions() + case e: OutOfOffheapMemoryException => logger.error(s"Out of offheap memory in dataset=${dataset.ref} " + + s"shard=$shardNum", e); disableAddPartitions() case e: Exception => logger.error(s"Unexpected ingestion err in dataset=${dataset.ref} " + s"shard=$shardNum", e); disableAddPartitions() } @@ -1094,7 +1095,7 @@ class TimeSeriesShard(val dataset: Dataset, } finally { partSetLock.unlockWrite(stamp) } - partitionObj.chunkmapFree() + partitionObj.shutdown() bufferMemoryManager.freeMemory(partitionObj.partKeyOffset) partitions.remove(partitionObj.partID) } diff --git a/core/src/main/scala/filodb.core/metadata/Column.scala b/core/src/main/scala/filodb.core/metadata/Column.scala index 578408d849..c612deee07 100644 --- a/core/src/main/scala/filodb.core/metadata/Column.scala +++ b/core/src/main/scala/filodb.core/metadata/Column.scala @@ -9,7 +9,7 @@ import org.scalactic._ import filodb.core._ import filodb.core.SingleKeyTypes._ import filodb.core.Types._ -import filodb.memory.format.{VectorInfo, ZeroCopyUTF8String} +import filodb.memory.format.{vectors => bv, VectorInfo, ZeroCopyUTF8String} import filodb.memory.format.RowReader.TypedFieldExtractor /** @@ -73,10 +73,13 @@ object Column extends StrictLogging { case object LongColumn extends RichColumnType[Long]("long") case object DoubleColumn extends RichColumnType[Double]("double") case object StringColumn extends RichColumnType[ZeroCopyUTF8String]("string") - case object BitmapColumn extends RichColumnType[Boolean]("bitmap") case object TimestampColumn extends RichColumnType[Long]("ts") case object MapColumn extends RichColumnType[UTF8Map]("map") case object BinaryRecordColumn extends RichColumnType[ZeroCopyUTF8String]("br") + // TODO: find a way to annotate histograms as rate-based (Prometheus increasing over time) or + // non-increasing over time + // These histograms for now are non-increasing over time (but increasing from bucket to bucket) + case object HistogramColumn extends RichColumnType[bv.Histogram]("hist") } val typeNameToColType = ColumnType.values.map { colType => colType.typeName -> colType }.toMap diff --git a/core/src/main/scala/filodb.core/metadata/SimpleComputations.scala b/core/src/main/scala/filodb.core/metadata/SimpleComputations.scala index 17dc2dcee2..f9803dd1f5 100644 --- a/core/src/main/scala/filodb.core/metadata/SimpleComputations.scala +++ b/core/src/main/scala/filodb.core/metadata/SimpleComputations.scala @@ -132,10 +132,10 @@ object SimpleComputations { case LongColumn => wrap((l: Long) => Math.abs(l % numBuckets).toInt) case DoubleColumn => wrap((d: Double) => Math.abs(d % numBuckets).toInt) case StringColumn => wrap((s: UTF8Str) => Math.abs(s.hashCode % numBuckets)) - case BitmapColumn => wrap((b: Boolean) => (if (b) 1 else 0) % numBuckets) case TimestampColumn => wrap((l: Long) => Math.abs(l % numBuckets).toInt) case MapColumn => wrap((m: UTF8Map) => Math.abs(m.hashCode % numBuckets)) case BinaryRecordColumn => wrap((s: UTF8Str) => Math.abs(s.hashCode % numBuckets)) + case HistogramColumn => wrap((s: UTF8Str) => Math.abs(s.hashCode % numBuckets)) } computedColumn(expr, dataset, info, IntColumn, extractor) } diff --git a/core/src/main/scala/filodb.core/query/PartitionTimeRangeReader.scala b/core/src/main/scala/filodb.core/query/PartitionTimeRangeReader.scala index b351726aec..4756da5a60 100644 --- a/core/src/main/scala/filodb.core/query/PartitionTimeRangeReader.scala +++ b/core/src/main/scala/filodb.core/query/PartitionTimeRangeReader.scala @@ -4,7 +4,7 @@ import scalaxy.loops._ import filodb.core.metadata.Dataset import filodb.core.store.{ChunkInfoIterator, ChunkSetInfo, ReadablePartition} -import filodb.memory.format.{RowReader, TypedIterator, UnsafeUtils, ZeroCopyUTF8String} +import filodb.memory.format.{vectors => bv, RowReader, TypedIterator, UnsafeUtils, ZeroCopyUTF8String} /** * A RowReader iterator which iterates over a time range in the ReadablePartition. Designed to be relatively memory @@ -33,6 +33,7 @@ final class PartitionTimeRangeReader(part: ReadablePartition, def getDouble(columnNo: Int): Double = vectorIts(columnNo).asDoubleIt.next def getFloat(columnNo: Int): Float = ??? def getString(columnNo: Int): String = ??? + override def getHistogram(columnNo: Int): bv.Histogram = vectorIts(columnNo).asHistIt.next def getAny(columnNo: Int): Any = ??? override def filoUTF8String(columnNo: Int): ZeroCopyUTF8String = vectorIts(columnNo).asUTF8It.next diff --git a/core/src/main/scala/filodb.core/store/ChunkSetInfo.scala b/core/src/main/scala/filodb.core/store/ChunkSetInfo.scala index 9867f18c6c..3de4c53bfa 100644 --- a/core/src/main/scala/filodb.core/store/ChunkSetInfo.scala +++ b/core/src/main/scala/filodb.core/store/ChunkSetInfo.scala @@ -8,6 +8,7 @@ import debox.Buffer import filodb.core.Types._ import filodb.core.metadata.{Column, Dataset} +import filodb.core.query.RawDataRangeVector import filodb.memory.BinaryRegion.NativePointer import filodb.memory.MemFactory import filodb.memory.data.ElementIterator @@ -317,26 +318,45 @@ class FilteredChunkInfoIterator(base: ChunkInfoIterator, final def unlock(): Unit = base.unlock() } +/** + * Stores, for each chunk in the WindowedChunkIterator, the reader and vector pointer for both + * the timestamp and value columns. + * NOTE: to prevent object bloat for value classes, the pointer to the info is stored. Use info member + * for easy access. + */ +final case class ChunkQueryInfo(infoPtr: NativePointer, + tsVector: BinaryVector.BinaryVectorPtr, + tsReader: vectors.LongVectorDataReader, + valueVector: BinaryVector.BinaryVectorPtr, + valueReader: VectorDataReader) { + // ChunkSetInfo is a value class, use this for typed and efficient access without allocations + def info: ChunkSetInfo = ChunkSetInfo(infoPtr) +} + /** * A sliding window based iterator over the chunks needed to be read from for each window. * Assumes the ChunkInfos are in increasing time order. - * The sliding window goes from (start-window, start] -> (end-window, end] in step increments, and for - * each window, this class may be used as a ChunkInfoIterator. Excludes start, includes end. + * The sliding window goes from (start-window, start] -> (end-window, end] in step increments. + * Excludes start, includes end. + * Also parses the readers for each new info for query efficiency. - * @param infos the base ChunkInfoIterator to perform windowing over + * @param rv a RawDataRangeVector to iterate over * @param start the starting window end timestamp, must have same units as the ChunkSetInfo start/end times * @param step the increment the window goes forward by * @param end the ending window end timestamp. If it does not line up on multiple of start + n * step, then * the windows will slide until the window end is beyond end. * @param window the # of millis/time units that define the length of each window */ -class WindowedChunkIterator(infos: ChunkInfoIterator, start: Long, step: Long, end: Long, window: Long, +class WindowedChunkIterator(rv: RawDataRangeVector, start: Long, step: Long, end: Long, window: Long, // internal vars, put it here for better performance var curWindowEnd: Long = -1L, var curWindowStart: Long = -1L, private var readIndex: Int = 0, - windowInfos: Buffer[NativePointer] = Buffer.empty[NativePointer]) -extends ChunkInfoIterator { + windowInfos: Buffer[ChunkQueryInfo] = Buffer.empty[ChunkQueryInfo]) +extends Iterator[ChunkQueryInfo] { + private val infos = rv.chunkInfos(start - window, end) + private val tsColID = rv.timestampColID + final def close(): Unit = infos.close() /** @@ -359,11 +379,11 @@ extends ChunkInfoIterator { readIndex = 0 // drop initial chunksets of window that are no longer part of the window - while (windowInfos.nonEmpty && ChunkSetInfo(windowInfos(0)).endTime < curWindowStart) { + while (windowInfos.nonEmpty && windowInfos(0).info.endTime < curWindowStart) { windowInfos.remove(0) } - var lastEndTime = if (windowInfos.isEmpty) -1L else ChunkSetInfo(windowInfos(windowInfos.length - 1)).endTime + var lastEndTime = if (windowInfos.isEmpty) -1L else windowInfos(windowInfos.length - 1).info.endTime // if new window end is beyond end of most recent chunkset, add more chunksets (if there are more) while (curWindowEnd > lastEndTime && infos.hasNext) { @@ -371,7 +391,11 @@ extends ChunkInfoIterator { require(next.infoAddr != 0, s"NULL nextInfo; curWindowEnd=$curWindowEnd, windowInfos=$windowInfos") // Add if next chunkset is within window and not empty. Otherwise keep going if (curWindowStart <= next.endTime && next.numRows > 0) { - windowInfos += next.infoAddr + val tsVector = next.vectorPtr(tsColID) + val tsReader = vectors.LongBinaryVector(tsVector) + val valueVector = next.vectorPtr(rv.valueColID) + val valueReader = rv.partition.chunkReader(rv.valueColID, valueVector) + windowInfos += ChunkQueryInfo(next.infoAddr, tsVector, tsReader, valueVector, valueReader) lastEndTime = Math.max(next.endTime, lastEndTime) } } @@ -385,11 +409,11 @@ extends ChunkInfoIterator { /** * Returns the next ChunkSetInfo for the current window */ - final def nextInfo: ChunkSetInfo = { + final def next: ChunkQueryInfo = { val next = windowInfos(readIndex) - require(next != 0, s"ERROR: info==null, windowInfos=$windowInfos readIndex=$readIndex") + require(next.infoPtr != 0, s"ERROR: info==null, windowInfos=$windowInfos readIndex=$readIndex") readIndex += 1 - ChunkSetInfo(next) + next } final def lock(): Unit = infos.lock() diff --git a/core/src/test/resources/application_test.conf b/core/src/test/resources/application_test.conf index 6e505909bf..650256c6ca 100644 --- a/core/src/test/resources/application_test.conf +++ b/core/src/test/resources/application_test.conf @@ -88,6 +88,7 @@ akka { send-buffer-size = 512000b receive-buffer-size = 512000b maximum-frame-size = 10 MiB + connection-timeout = 30s } } diff --git a/core/src/test/scala/filodb.core/TestData.scala b/core/src/test/scala/filodb.core/TestData.scala index c71ab4a979..bd8f4a25c7 100644 --- a/core/src/test/scala/filodb.core/TestData.scala +++ b/core/src/test/scala/filodb.core/TestData.scala @@ -15,6 +15,7 @@ import filodb.core.store._ import filodb.core.Types.{PartitionKey, UTF8Map} import filodb.memory.format._ import filodb.memory.format.ZeroCopyUTF8String._ +import filodb.memory.format.{vectors => bv} import filodb.memory.{BinaryRegionLarge, MemFactory, NativeMemoryManager} object TestData { @@ -292,6 +293,29 @@ object MachineMetricsData { val tagsDiffSameLen = extraTags ++ Map("region".utf8 -> "AWS-USEast".utf8) val extraTagsLen = extraTags.map { case (k, v) => k.numBytes + v.numBytes }.sum + + val histDataset = Dataset("histogram", Seq("tags:map"), + Seq("timestamp:ts", "count:long", "sum:long", "h:hist")) + + var histBucketScheme: bv.HistogramBuckets = _ + def linearHistSeries(startTs: Long = 100000L, numSeries: Int = 10, timeStep: Int = 1000, numBuckets: Int = 8): + Stream[Seq[Any]] = { + histBucketScheme = bv.GeometricBuckets(2.0, 2.0, numBuckets) + val buckets = new Array[Double](numBuckets) + def updateBuckets(bucketNo: Int): Unit = { + for { b <- bucketNo until numBuckets } { + buckets(b) += 1 + } + } + Stream.from(0).map { n => + updateBuckets(n % numBuckets) + Seq(startTs + n * timeStep, + (1 + n).toLong, + buckets.sum.toLong, + bv.MutableHistogram(histBucketScheme, buckets.map(x => x)), + extraTags ++ Map("__name__".utf8 -> "http_requests_total".utf8, "dc".utf8 -> s"${n % numSeries}".utf8)) + } + } } // A simulation of custom machine metrics data - for testing extractTimeBucket @@ -349,4 +373,25 @@ object MetricsTestData { final def getBlobNumBytes(columnNo: Int): Int = ??? } + // Takes the output of linearHistSeries and transforms them into Prometheus-schema histograms. + // Each bucket becomes its own time series with _bucket appended and an le value + def promHistSeries(startTs: Long = 100000L, numSeries: Int = 10, timeStep: Int = 1000, numBuckets: Int = 8): + Stream[Seq[Any]] = + MachineMetricsData.linearHistSeries(startTs, numSeries, timeStep, numBuckets) + .flatMap { record => + val timestamp = record(0) + val tags = record(4).asInstanceOf[Map[ZeroCopyUTF8String, ZeroCopyUTF8String]] + val metricName = tags("__name__".utf8).toString + val countRec = Seq(timestamp, record(1).asInstanceOf[Long].toDouble, + tags + ("__name__".utf8 -> (metricName + "_count").utf8)) + val sumRec = Seq(timestamp, record(2).asInstanceOf[Long].toDouble, + tags + ("__name__".utf8 -> (metricName + "_sum").utf8)) + val hist = record(3).asInstanceOf[bv.MutableHistogram] + val bucketTags = tags + ("__name__".utf8 -> (metricName + "_bucket").utf8) + val bucketRecs = (0 until hist.numBuckets).map { b => + Seq(timestamp, hist.bucketValue(b), + bucketTags + ("le".utf8 -> hist.bucketTop(b).toString.utf8)) + } + Seq(countRec, sumRec) ++ bucketRecs + } } diff --git a/core/src/test/scala/filodb.core/binaryrecord/BinaryRecordSpec.scala b/core/src/test/scala/filodb.core/binaryrecord/BinaryRecordSpec.scala index 995522486d..1eb3754ec6 100644 --- a/core/src/test/scala/filodb.core/binaryrecord/BinaryRecordSpec.scala +++ b/core/src/test/scala/filodb.core/binaryrecord/BinaryRecordSpec.scala @@ -18,13 +18,13 @@ class BinaryRecordSpec extends FunSpec with Matchers { val schema2_sl = new RecordSchema(Seq(StringColumn, LongColumn)) val schema2_is = new RecordSchema(Seq(IntColumn, StringColumn)) - val schema3_bdt = new RecordSchema(Seq(BitmapColumn, DoubleColumn, TimestampColumn)) + val schema3_bdt = new RecordSchema(Seq(IntColumn, DoubleColumn, TimestampColumn)) val schema4 = new RecordSchema(Seq(StringColumn, MapColumn)) val reader1 = TupleRowReader((Some("data"), Some(-15L))) val reader2 = TupleRowReader((Some(1234), Some("one-two-three"))) - val reader3 = TupleRowReader((Some(true), Some(5.7), Some(new Timestamp(1000000L)))) + val reader3 = TupleRowReader((Some(1), Some(5.7), Some(new Timestamp(1000000L)))) it("should create and extract individual fields and match when all fields present") { BinaryRecord(schema1_i, reader2).getInt(0) should equal (1234) @@ -47,7 +47,6 @@ class BinaryRecordSpec extends FunSpec with Matchers { binRec4.noneNull should equal (true) val binRec5 = BinaryRecord(schema3_bdt, reader3) - binRec5.getBoolean(0) should equal (true) binRec5.getDouble(1) should equal (5.7) binRec5.getLong(2) should equal (1000000L) binRec5.noneNull should equal (true) @@ -73,7 +72,6 @@ class BinaryRecordSpec extends FunSpec with Matchers { it("should get bytes out and get back same BinaryRecord") { val bytes = BinaryRecord(schema3_bdt, reader3).bytes val binRec = BinaryRecord(schema3_bdt, bytes) - binRec.getBoolean(0) should equal (true) binRec.getDouble(1) should equal (5.7) binRec.getLong(2) should equal (1000000L) } diff --git a/core/src/test/scala/filodb.core/binaryrecord2/BinaryRecordSpec.scala b/core/src/test/scala/filodb.core/binaryrecord2/BinaryRecordSpec.scala index ad2d68b3f6..021cadc36e 100644 --- a/core/src/test/scala/filodb.core/binaryrecord2/BinaryRecordSpec.scala +++ b/core/src/test/scala/filodb.core/binaryrecord2/BinaryRecordSpec.scala @@ -8,7 +8,7 @@ import filodb.core.metadata.Column.ColumnType import filodb.core.metadata.{Dataset, DatasetOptions} import filodb.core.query.ColumnInfo import filodb.memory.{BinaryRegion, BinaryRegionConsumer, MemFactory, NativeMemoryManager, UTF8StringMedium} -import filodb.memory.format.{SeqRowReader, UnsafeUtils} +import filodb.memory.format.{SeqRowReader, UnsafeUtils, ZeroCopyUTF8String => ZCUTF8} class BinaryRecordSpec extends FunSpec with Matchers with BeforeAndAfter with BeforeAndAfterAll { import MachineMetricsData._ @@ -370,8 +370,7 @@ class BinaryRecordSpec extends FunSpec with Matchers with BeforeAndAfter with Be schemaWithPredefKeys.consumeMapItems(recordBase, recordOff, 6, valuesCheckConsumer) } - it("should add map fields with addSortedPairsAsMap() and populate unique hashes") { - import collection.JavaConverters._ + it("should add map fields with addMap() and populate unique hashes") { val labels = Map("job" -> "prometheus", "dc" -> "AWS-USE", "instance" -> "0123892E342342A90", "__name__" -> "host_cpu_load") @@ -379,8 +378,7 @@ class BinaryRecordSpec extends FunSpec with Matchers with BeforeAndAfter with Be val builder = new RecordBuilder(MemFactory.onHeapFactory, schema2) def addRec(n: Int): Long = { - val pairs = new java.util.ArrayList((labels + ("n" -> n.toString)).toSeq.asJava) - val hashes = RecordBuilder.sortAndComputeHashes(pairs) + val pairs = (labels + ("n" -> n.toString)).map { case (k, v) => ZCUTF8(k) -> ZCUTF8(v) }.toMap builder.startNewRecord() val ts = System.currentTimeMillis @@ -390,7 +388,7 @@ class BinaryRecordSpec extends FunSpec with Matchers with BeforeAndAfter with Be builder.addDouble(10.1) // max builder.addLong(123456L) // count builder.addString(s"Series $n") // series (partition key) - builder.addSortedPairsAsMap(pairs, hashes) + builder.addMap(pairs) builder.endRecord() } @@ -524,6 +522,32 @@ class BinaryRecordSpec extends FunSpec with Matchers with BeforeAndAfter with Be partSchema2.consumeMapItems(recordBase, recordOff, 1, keyCheckConsumer) lastIndex shouldEqual 4 // 5 key-value pairs: 0, 1, 2, 3, 4 } + + it("should copy ingest BRs to partition key BRs correctly when data columns have a blob/histogram") { + val ingestBuilder = new RecordBuilder(MemFactory.onHeapFactory, histDataset.ingestionSchema) + val data = linearHistSeries().take(3) + data.foreach { row => ingestBuilder.addFromReader(SeqRowReader(row)) } + + records.clear() + ingestBuilder.allContainers.head.consumeRecords(consumer) + val histRecords = records.toSeq.toBuffer // make a copy + + // Now create partition keys + val partKeyBuilder = new RecordBuilder(MemFactory.onHeapFactory, histDataset.partKeySchema) + histRecords.foreach { case (base, off) => + histDataset.comparator.buildPartKeyFromIngest(base, off, partKeyBuilder) + } + + records.clear() + partKeyBuilder.allContainers.head.consumeRecords(consumer) + records should have length (3) + (0 to 2).foreach { n => + histDataset.comparator.partitionMatch(histRecords(n)._1, histRecords(n)._2, + records(n)._1, records(n)._2) shouldEqual true + histDataset.ingestionSchema.partitionHash(histRecords(n)._1, histRecords(n)._2) shouldEqual ( + histDataset.partKeySchema.partitionHash(records(n)._1, records(n)._2)) + } + } } describe("hashing functions") { diff --git a/core/src/test/scala/filodb.core/downsample/ShardDownsamplerSpec.scala b/core/src/test/scala/filodb.core/downsample/ShardDownsamplerSpec.scala index ebe503e375..20f8dedfae 100644 --- a/core/src/test/scala/filodb.core/downsample/ShardDownsamplerSpec.scala +++ b/core/src/test/scala/filodb.core/downsample/ShardDownsamplerSpec.scala @@ -1,7 +1,5 @@ package filodb.core.downsample -import java.util - import scala.collection.mutable import org.scalatest.{BeforeAndAfterAll, FunSpec, Matchers} @@ -46,14 +44,14 @@ class ShardDownsamplerSpec extends FunSpec with Matchers with BeforeAndAfterAll blockStore.releaseBlocks() } - val partKeyTags = new util.ArrayList[(String, String)]() - partKeyTags.add("dc"->"dc1") - partKeyTags.add("instance"->"instance1") + import ZeroCopyUTF8String._ + + val partKeyTags = Map("dc".utf8 -> "dc1".utf8, "instance".utf8 -> "instance1".utf8) val partKeyBuilder = new RecordBuilder(TestData.nativeMem, promDataset.partKeySchema, 4096) partKeyBuilder.startNewRecord() partKeyBuilder.addString("someStringValue") - partKeyBuilder.addSortedPairsAsMap(partKeyTags, RecordBuilder.sortAndComputeHashes(partKeyTags)) + partKeyBuilder.addMap(partKeyTags) partKeyBuilder.endRecord(true) val partKeyBase = partKeyBuilder.allContainers.head.base val partKeyOffset = partKeyBuilder.allContainers.head.allOffsets(0) diff --git a/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala b/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala index f8180c150f..48a2d1ab07 100644 --- a/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala +++ b/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala @@ -18,6 +18,7 @@ import filodb.core.query.{ColumnFilter, Filter} import filodb.core.store.{FilteredPartitionScan, InMemoryMetaStore, NullColumnStore, SinglePartitionScan} import filodb.memory.{BinaryRegionLarge, MemFactory} import filodb.memory.format.{ArrayStringRowReader, UnsafeUtils, ZeroCopyUTF8String} +import filodb.memory.format.vectors.MutableHistogram class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter with ScalaFutures { implicit val s = monix.execution.Scheduler.Implicits.global @@ -88,6 +89,33 @@ class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter w agg1 shouldEqual (3 + 8 + 13 + 18) } + it("should ingest histograms and read them back properly") { + memStore.setup(histDataset, 0, TestData.storeConf) + val data = linearHistSeries().take(40) + memStore.ingest(histDataset.ref, 0, records(histDataset, data)) + memStore.commitIndexForTesting(histDataset.ref) + + memStore.numRowsIngested(histDataset.ref, 0) shouldEqual 40L + // Below will catch any partition match errors. Should only be 10 tsParts. + memStore.numPartitions(histDataset.ref, 0) shouldEqual 10 + + val split = memStore.getScanSplits(histDataset.ref, 1).head + val filter = ColumnFilter("dc", Filter.Equals("1".utf8)) + // check sums + val sums = memStore.scanRows(histDataset, Seq(2), FilteredPartitionScan(split, Seq(filter))) + .map(_.getLong(0)).toList + sums shouldEqual Seq(data(1)(2).asInstanceOf[Long], + data(11)(2).asInstanceOf[Long], + data(21)(2).asInstanceOf[Long], + data(31)(2).asInstanceOf[Long]) + + 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] + } + } + it("should be able to handle nonexistent partition keys") { memStore.setup(dataset1, 0, TestData.storeConf) @@ -264,7 +292,8 @@ class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter w it("should recover index from time buckets") { import GdeltTestData._ - val readers = gdeltLines3.map { line => ArrayStringRowReader(line.split(",")) } + // NOTE: gdeltLines3 are actually data samples, not part keys. Only take lines which give unique part keys!! + val readers = gdeltLines3.take(8).map { line => ArrayStringRowReader(line.split(",")) } val partKeys = partKeyFromRecords(dataset1, records(dataset1, readers.take(10))) indexRecoveryTest(dataset1, partKeys) } diff --git a/doc/FAQ.md b/doc/FAQ.md index e827a6cd4c..f0142469c7 100644 --- a/doc/FAQ.md +++ b/doc/FAQ.md @@ -1,3 +1,12 @@ + + +**Table of Contents** *generated with [DocToc](https://github.com/thlorenz/doctoc)* + +- [FiloDB FAQ](#filodb-faq) + - [Q: FiloDB supports Prometheus style queries, but why shouldn't I just use Prometheus?](#q-filodb-supports-prometheus-style-queries-but-why-shouldnt-i-just-use-prometheus) + + + # FiloDB FAQ ## Q: FiloDB supports Prometheus style queries, but why shouldn't I just use Prometheus? diff --git a/doc/binaryrecord-spec.md b/doc/binaryrecord-spec.md index 6dfc2b7d28..d76260a3a5 100644 --- a/doc/binaryrecord-spec.md +++ b/doc/binaryrecord-spec.md @@ -5,10 +5,9 @@ - [BinaryRecord Specification v2](#binaryrecord-specification-v2) - [Motivation](#motivation) - [Differences from v1](#differences-from-v1) - - [[RecordSchema](../core/src/main/scala/filodb.core/binaryrecord2/RecordSchema.scala)](#recordschemacoresrcmainscalafilodbcorebinaryrecord2recordschemascala) + - [RecordSchema](#recordschema) - [Field Access](#field-access) - [PartitionKey extraction, hash, comparison](#partitionkey-extraction-hash-comparison) - - [Shard key calculation](#shard-key-calculation) - [BinaryRecord creation](#binaryrecord-creation) - [Get all optimal container bytes](#get-all-optimal-container-bytes) - [Get all full container bytes excluding current container](#get-all-full-container-bytes-excluding-current-container) @@ -18,6 +17,7 @@ - [Hash](#hash) - [Variable length fields - UTF8String](#variable-length-fields---utf8string) - [Variable length fields - Map field](#variable-length-fields---map-field) + - [Variable length fields - Histograms](#variable-length-fields---histograms) - [RecordContainer: Container format for multiple BinaryRecords](#recordcontainer-container-format-for-multiple-binaryrecords) @@ -55,6 +55,7 @@ The `RecordSchema` defines the schema for a `BinaryRecord`, and consists of defi - double (64 bits) - UTF8String (< 64KB total) - map** (< 64KB total, UTF8 string to UTF8 string) + - histograms, which are blobs with a special format * Data column fields must strictly precede partition key fields * map field is only allowed as the last field - Map field labels are sorted by key @@ -111,24 +112,6 @@ An `IngestionRecordSchema` should be able to do the following to help in identif def buildPartKeyFromIngest(ingestBase: Any, ingestOffset: Long, builder: RecordBuilder) -### Shard key calculation - -First of all, before we even create the ingestion BinaryRecord, the application gateway probably has to compute the shard from the partition key and incoming tags. To do that, first call a method to sort the incoming key-value pairs and compute hashes for each pair: - - def sortAndComputeHashes(pairs: List[(String, String)]): Array[Int] - -From here, one of two methods can be called to compute the necessary partition/shard hashes: - - def combineHashExcluding(pairs: List[(String, String)], - hashes: Array[Int], excludeKeys: Set[String]): Int - def combineHashIncluding(pairs: List[(String, String)], - hashes: Array[Int], includeKeys: Set[String]): Int - -Now, when creating the ingestion BinaryRecord, a convenient method can be called to add the map and put the hash in as well: - - def addSortedPairsAsMap(sortedPairs: List[(String, String)], - hashes: Array[Int]): Unit - ### BinaryRecord creation `BinaryRecord` creation uses an assistant class, [RecordBuilder](../core/src/main/scala/filodb.core/binaryrecord2/RecordBuilder.scala), which takes a [MemFactory](../memory/src/main/scala/filodb.memory/MemFactory.scala) so they can be created on or offheap. Since creation of `BinaryRecords` takes a variable amount of space per record, the builder allocates blocks ahead of time and carves out memory within the block as new `BinaryRecord`s are being built. The builder keeps track of block/memory usage as new `BinaryRecords` are being built. @@ -139,11 +122,7 @@ The builder has add methods that should be called in field order. The methods w builder.startNewRecord() builder.addLong(timestamp) builder.addDouble(value) -builder.startMap() -for { field <- fields } { - builder.addMapKeyValue(field.key, field.value) -} -builder.endMap() +builder.addMap(tags) val memory = builder.endRecord() ``` @@ -181,7 +160,8 @@ Returns all the full containers and removes the returned full containers from `R * Long - 8 bytes - little-endian Long * Double - 8 bytes * utf8 - 4 bytes - offset within BR to var-length UTF8 string area -* map - 4 bytes - offset within BR to map area +* map - 4 bytes - offset within BR to map area (with 4-byte length prefix) +* hist - 4 bytes - offset within BR to histogram blob, with 2 byte length prefix ### Hash @@ -203,6 +183,10 @@ Note that this is called a "Map" field but is actually just a list of key-value * +0006 to +0006+(keylen1 - 1) UTF8 bytes for key #1 * +n 2-byte length of value #1, followed by UTF8 bytes of value string #1 +### Variable length fields - Histograms + +Histograms are stored as blobs with a 2-byte length prefix and includes the historam bucket definition and compressed bucket values. Please see [BinaryHistogram](../memory/src/main/scala/filodb.memory/format/vectors/HistogramVector.scala) for more details. + ## RecordContainer: Container format for multiple BinaryRecords A [RecordContainer](../core/src/main/scala/filodb.core/binaryrecord2/RecordContainer.scala) is a container for multiple `BinaryRecords` for ingesting into Kafka, for example. diff --git a/doc/compression.md b/doc/compression.md index f56e7b1520..fe5f1f7bf1 100644 --- a/doc/compression.md +++ b/doc/compression.md @@ -1,3 +1,16 @@ + + +**Table of Contents** *generated with [DocToc](https://github.com/thlorenz/doctoc)* + +- [Compression](#compression) + - [Long/Integer Compression](#longinteger-compression) + - [Floating Point Compression](#floating-point-compression) + - [Predictive NibblePacking](#predictive-nibblepacking) + - [Example](#example) + - [Histograms](#histograms) + + + # Compression Here we will discuss some of the compression algorithms in FiloDB's underlying chunk format. @@ -30,7 +43,7 @@ The predictive nibblepacking scheme encodes 8 64-bit values at once, storing the | offset | description | | ------ | ----------- | | +0 | u8: bitmask, 1=nonzero value, 0=zero value | -| +1 | u8: bits 0-3 = number of trailing zero nibbles (0-15); 4-7 = number of leading zero nibbles (0-15); skipped if bitmask == 0 | +| +1 | u8: bits 0-3 = number of trailing zero nibbles (0-15); 4-7 = number of nonzero nibbles - 1 (0-15; 15=all 16 nibbles occupied); skipped if bitmask == 0 | | +2 | little-endian nibble storage for each nonzero value in the bitmask; each value has (16 - leading - trailing) nibbles. Skipped if bitmask = 0 | The total space required to encode the 8 values can be derived as follows: @@ -75,5 +88,10 @@ Now here is how they would be stored in memory: Or, if the above was viewed in a little-endian system as a 32-bit int, then the above would be 0x00456123. +## Histograms + +FiloDB supports first class histograms as HistogramColumns in schemas. This means histograms are ingested as single entities and kept together as a single time series. Compressed histograms (using NibblePacking) are stored in write buffers and may be compressed further. +Please see [BinaryHistogram](../memory/src/main/scala/filodb.memory/format/vectors/HistogramVector.scala) for more details about the on-the-wire / BinaryRecord format used for histograms. +TODO: document histogram compression techniques more when done. diff --git a/doc/downsampling.md b/doc/downsampling.md index 0fc7d50d33..b4ab7e5ae7 100644 --- a/doc/downsampling.md +++ b/doc/downsampling.md @@ -1,3 +1,15 @@ + + +**Table of Contents** *generated with [DocToc](https://github.com/thlorenz/doctoc)* + +- [Downsampling](#downsampling) + - [Configuration](#configuration) + - [Downsample Data Generation](#downsample-data-generation) + - [Best Practices](#best-practices) + - [Querying of Downsample Data](#querying-of-downsample-data) + + + # Downsampling Downsampling is a process of reducing the number of samples stored for a time series. diff --git a/doc/indexing.md b/doc/indexing.md index ccf06ccc01..6a9f773249 100644 --- a/doc/indexing.md +++ b/doc/indexing.md @@ -3,12 +3,8 @@ **Table of Contents** *generated with [DocToc](https://github.com/thlorenz/doctoc)* - [Lucene Indexing](#lucene-indexing) - - [Time Bucket Creation during NormalIngestion](#time-bucket-creation-during-normalingestion) + - [Time Bucket Creation during Normal Ingestion](#time-bucket-creation-during-normal-ingestion) - [Index Recovery](#index-recovery) - - [Recovery Behavior during Different Shard States](#recovery-behavior-during-different-shard-states) - - [Recovering State](#recovering-state) - - [onIndexBootstrapped State Transition](#onindexbootstrapped-state-transition) - - [NormalIngestion State](#normalingestion-state) diff --git a/doc/mermaid/chunked-iteration.mermaid b/doc/mermaid/chunked-iteration.mermaid new file mode 100644 index 0000000000..1567addc23 --- /dev/null +++ b/doc/mermaid/chunked-iteration.mermaid @@ -0,0 +1,15 @@ +sequenceDiagram + participant ChunkedRangeFunction + participant ChunkedWindowIterator + participant WindowedChunkIterator + loop for each time window + ChunkedWindowIterator->>ChunkedRangeFunction: reset() + ChunkedWindowIterator->>WindowedChunkIterator: nextWindow() + loop fetch and work on chunks + ChunkedWindowIterator->>WindowedChunkIterator: next() + WindowedChunkIterator->>ChunkedWindowIterator: ChunkQueryInfo + ChunkedWindowIterator->>ChunkedRangeFunction: addChunks() + end + ChunkedWindowIterator->>ChunkedRangeFunction: apply() + Note right of ChunkedWindowIterator: yield value for time window + end \ No newline at end of file diff --git a/doc/mermaid/chunked-iteration.png b/doc/mermaid/chunked-iteration.png new file mode 100644 index 0000000000000000000000000000000000000000..652279f74c9097acc4e162c2f68821b1e4d23a8a GIT binary patch literal 43199 zcmdSBbx_sO_cnSAKtd2yN69*Mq@<-$KuSPKkrbp$xo|AVqn}n^>e+2n6D$)N?Uq1mbEi z0&!*e20Hvj_|PL7fw+&55__upCShaB-6QPa^2g3d-QHO1)=frBdH<+9F1bb`{7(De z)CI*GGY^857j8XFWLW%|@a)%1`y261{RtVL7JthUPiz%-a~+ktA8qw+ha94()E6liBgDc_*-yY=jb9cDl=uE@XtPbSeQ-Fb%kud`Gt z1L{SpjR-NNf3JXi@FJozf|Gl}9RqBrazt?o-O~^$oI)C}p))t37 zn{1z){Lx#OmsZL9%`5O8?SW`uCsX)=sJfn494+;=h=?Ce*Afyt`AF^PnHUgh`0so> z75rmQsiZ>&@u*k}1%|`z2U0&rU4Q!2gxsG&AtEA0DXlmx)$uvFl=kHm1N#?dl~l{O9F>KeyYK(|EVhGd9a5>EV=-7+QFpM_(YeoE0mHHTKfdKZ z&nlh#e}13{2CRP%Tl4>RX8ymp(Lgr-xSL~R)yr4grQ+7XUwVP|+ znRq~Gw|soIowhO4a8y0Q9pAk%S(W(VgF`y;&_R_l30Iu#x6I5>Nl6*HOnD>9k?oEu z;j_+slBo$a!P1PWsf6>}h4XJQt`f5^+|7C6u#7wk=&N-VWoPf{D>0+N6ZW7qYQKoe)mz$y<=Ud8r z!S^`a4;^bBYh7L(8_lqUuO9Ch37l=HFzfIv$84WDr$ia<%(bZI=TJW>Js%2TYe>k8 zE>z2>Sl$@p==!`<#kKhpGoiU$^74b@PFY#}*dZ0!T+%(&C)!{W`8FKNzR~=Uz`8+&4Ml0CD>fDX23)CH!dl11L zfA87ux6J5EDt^QZz@s!=`1Mmvf~%o`Ev2YPJ6mTgE-*0T9C`GqU#OIzN{6JUy?rUS zFhj8{hbdf8Eiy@vU$Lr~Ba+@*q&JCtwB8f@MQ-(UsZn4^NZlN7#Hro-)J{;Ua_}c= zjNR6IQ)Ay}U2xD_=2{wFA6i{GSQC_0Q=_x7vFS>v`dpyvJ~uxfBA>}LM4XV2aD2X} znO=^(K+gP`4{sROJK7kFwVohA9G@(A6XOKWcnf!3Al=P}*6hk{XY57Km~MKW*2+6N z3Lw@;wwnzU69lzI?CrUxYK|TV3vXw%DJm-dpillVAauN4v+YI5cHvK?_hvBz<~}XT z@XwF8j@#3TSy?l@&UrBZRHDR)1Kq?GFxPT{#Slq;;L`d;|r_}>00qTB}Q%jdMl zZZoMG!!i+ASg0R#L{4#W=N{L2fprb@7*kVR90_Xih0p)8?CiAD(3=%`A?4*0eo^uB zV^dP`ov(o)M8<2>FV$Avqq)X6K`3xxkgI~DFjc6 zsHj4#w(IXB)&{d>oC6P8rk{j)Xs9MFUaue(<}NOJ7`0>7< z>C&8=Wi>VYMMaMVjt%bP2@(II^xkN);^J`ISA37_8LjW7iTUIFq=cB){>DIt=Z6-< zS(t6vIQE!N)B(5d)eM;;W#H-N=3O{Ejzv&ctPTi2W%TbeH=eBgG;aMtJfwa^NFZ8y zd~+h@K4rCwZ9qv$=iTs2?@JjruKX3}c|5HWkD2;jhovqHNTcqD=koAd$(tQKX8rmb z-rnT8HRV(+EHAW$X&3MKq963zOyz4G+~#){b2$D>W$m@xlfQE`vXgxIq@xUnh@Fc|TW4p$Q(w2ojw_aAyb*ydhDE%~Bc#bcKlc3jS!ljvJzk2Ne#f?s)2A;j zfZ*P}roG+@S}Lj}>j|zs)XiMOl~sDjC#6qUGtn5*H?|r~<;dCUN8b+)s&3H?yWx|O zb*U&sZ@bxT*9}=rlwV^~XYCW3oSK~6+21#uZ!JsUbs$BAv@|C0>hCQ?vLKH4dap}_ zku|<1S!v-r9)X5e>jae0N;+uuh(D4Q)1mV)%q~?EA}`m<=-nDG6W-hHEc`mK`do^) zg?q-MHMh{Y&9La>7q=KRY}4ELeNo7>BlD5gWG^j)l7^L)&~aX?-+!hJcXYPMyf2a5 zA4m`MQO^M($E?ej>@rQ#b+(OXv^Iz-(sw3ozi_Mm^c9wLHj`djq3U~uB!Qn_I6Gm2 zePizEca4q371n0>Y+)w@3T=DS?zX$*mTVDGj(HkZ%EC76P8j^I zyWg%3mh?KmkPN%?(D$kCj_vQ1Su8~=s+X^}CKn$vFlY!O9sQ(bm{i-slhqH_I2B_6P9SlD%28T&7PHuZ*8!6BU$e&KM}>=r{aaTg{|ab*mQ7*Sebq z>0s#n%?2jlr`e^Y&+bx8%X%J7g?AVzuEpCfF^PCyBH2wIF>o3-W8_GLjz45%R3R7m zociwFyJ$VilrVDsOdRy|xw$X~5r|`L5*+D#%@R2@G&ClWmKO59FuRjo{ktiLTXM<) zc$v8=DFz3_+G#^SeHnAKD`Lxx-t2WNY2m13dZF$jpV;xjX>IVuc0K=7xDktXxsvup z8|6q&!-cep&v^9#p3|Y#+qI$GhO2{HQ*Bm;=Nlz)ubSS^45f-Iakc%Ounn*ZgQVav z&f5@kj2aqL08Ya)PY30G;3TidZC`dFZ9eo9*1vVi zR9My>u+w3T*>?q`p{{;-jAakBhwylg$IfBl*&r0kp$Gwjmp$L7CZqeg_9KB#0l zFon?(5rr^aTvo@roHpm*_&oU(_-%v*#~(cuIaue` zK<>)qVconr5|(Y$W+PpQrzm{>y>aVc)_&shaKaz;2MpWyKot=I7;K8%IgoH6pcPr_j{9-AJ2GkdZkuz1-YxJr^=|x3UB42dw ztHG@NI*fsg?iX+)HP-sGEy?r|qQq-#Y_??vKOYW#6(vd^!N{6#>qP|}rMFHlDh7KJ zgVy#}1%}dP@Z#6|qkHV^?PFlb>QA5kT!Y8zc5>-WYKZ=sGT4`Pi5lVQUN47worHci z4x8S2h%>T&+$_6H;M!UUn`tgSwUDAuaL^cPnoN83Ua2@&Sj+qlbDeMHi7!4m7}*<~ zunm9V6C4&+G3F)P<+Vxr`1K#f%>-#=W!csWq4+wBZ*A?QWp%#E6TK9tb!Rbg>rBtH zu-d(z05X2(&*$eJwRN86gAB4YFVY8A&HKOc^74*B;opJaULCRhUVbs4Ao<8@)Et*A zu+6*{A@cGi)y0Kdxy_W=Oug3{pZi2D+gDIaV)H6>ivCuIuKfxkwe4z4aGK74~Gl&O*oLSj+t=daZ7!D z;mri~0-eR29TrV?JU~f&PHtBc<=Z%V#ZpHw{wyuM*Q4|)ETrVL9G*>b-=gKIq%&&s zGAy~gd2jgjR>E6Asi`n+)3uz))0;~nY;-TVi0({y2jH?1;P*vWR#x)oCq-*5>3xX+q}(`G!U&Ur|T)y`D}B%3=t>7!K29lnN|r#F9ZSR=8CSao*D++AHc44a7~A|rY19$*5H)$=Ig zM$jwJUHjWc>F&B&#(e`P&!fe1f2iYoS^XE;lMxTxk-V|i5Vk|pUmsln!7uHvdi$jI zhJ~PEjPw^E!-Y;O)wtj-5+enIBs{N)Vf&KtMH!S#S2>{Tc~w!fXbE%&PyU)Ba7}u z3W_A)djUO(+Iu@~Gm3Sm$ZJ$xDpK^6g(!J1w*k%nN^aB8W!U!$w%9HrxX#c<%htIm zJfEnp%V7Slv{SLSFMdqi&suX?T3V{Ss_&AkIdha-HLxx&B61(NkYUq%PKawILn9;e znR;qS)yqHLBFO-x=c=)!OGOfn^U)17?C5sd0e3$OE~25M5G%w>v0uy?FFie=ymFuS z#1=7Lu`MRu!Faem+GngNE&+jogdc@N-bXGs-@}Gq0`Vbt7vEd z$o)ArMb2H3Dl}*!D3V4)Mn<+WQNhN@mE-MLr2UYpP1A=7l!89-9?0a_gsIVPO#bpCTh~o=@mk z8@0v2>=^v{@p6PLGI}t)-$9Z-7#<}R4)*~*I)`Qg9Hnl&}dGt}}ij^{$gtR^dCRtK!L zyX6yj-o9unOYQ5mBFZvpP+cgXkXd$1g~TfP<^iV<3kmxfhVY5=t8kYPJ8TI`^; znj{u4)I-|YRoOQfyT2OF6D={#M`iS{UuOl5|A=z!MW0sG>vSIc?D+W;-^b6-`^=)c zs%nfz(4{(tB}Agrd6TrKCvgH3E9WWl6c4s=lKbT#pvIn_38QpGQBN)=@I*| zx;oF5@iO`lqRSspF~=%wi2&ys!A@vyZ3R8TU?B6a{m4pg^h?WUWlrmb>b1UxY?!rV zl9G~}mqhfx)E(?xW##4P)(eH}mYmpV59V8egQjL?(tDmAwirA&>i#0`9IIL9Mk0J! zd;F!s6)MK&B^nwXU=o=qn*Ei&fJFY=ut(DL8*Iia2!$JhzkN&SPwGVz4}U=XB#TH| z^EqswM5%Zo2Sf&$6;Q_Gy3x2N0t)Ay{)9d-<2LK^Iz^|d^p zU8GTTeEh4uy?!l)oCyz4!Fg-4=cli?`2d%dhRVjvzF7AL7<+qn_u~19to3-d3jN>R z-ExvqyUdc|kkZm8ge=wkYvRwJqf#?MOw93Wit{E$*+gd|U)}4b3Xg(2z-llM+1X~L zCz_gZsE~vrcRd*W+WwZ{M~eX=x(BI@zUVhI9hNCjK_08WmDS?O;>brnJ$)AueDms? zS^q<=&aU)=gVU6O)zWH9Yub0IG*m)f+`wJ9q<$o>0VepWTa&ssT{vTqkeIl$lPz*~ zC_cdFykX!zjHJrc>~+s=d0)pE&Hx31Zm(0tDozI z4e2Z{WtRKV9j`-4pEO9LLCW}v$EBr3dl}uUj z#lQb}$7kxO8v~FpD%U=oc%)`74^1p`>Hp z2|?|~{TJTp^WVz}bXa4)f2Yq@FT4VviH7rl84=HwPYt>k;KGq|3(slonR?$FH*PF^ z4WRf6c-3)ZG&JXhbwi%Ud=bTRx`{_;%mHdw0~Y;VzIi)OukORv)Wi;;;WzGQkWc!j z9dem;DT_-=tjZezC7vGK>r&U3>!Z-|`XeH+vs2@GIL7*;A-~1Na0G|p^vCvKHUT@?8Ut8?NZEIPXZPezFF$&XAV5J zU*I9-^5`=C_V_unGTo;8b6os-vNzUXylvQk#3Rki_jum=_mqxj1Cn12%(&a8ei3xf zKCBiEU|nktA}a$(Wj+4mt>};>rDryZt@&Xl=i&){PVIKTL`re3S6Gdy)$Mj5HV*)2 zX6t(e4e4=P9e*CL;g;H*jLQmDMIQus^8yGW2~LOU%&;(U!Gnvl$iX&BZzB%R=0Lo` zu@bZv2|AaJQMr7LB29NeK=VUIM%-vwBpie}!}${y6N&sTRiwQ3336SGj3+sOHCpBn zgBfz}Q9ue&3c!GR!DK2hK01XYHD-zRBHeRtlMcst{A}34iYS4^VVSkzl?Uh}@p}b8 zKv}d8LC)#P)hhckUXM)4QOPVQPjE6mPG7EaKhi@zH|Pydi;(&{8A7Kk)Fwq59o1Q} z&!0YhOUhkvK8(G-Vz54w^WyQ^kb1zV>_k1%$8EQrkjrFeW9;fdzYMZ;vpcb7a!EkP zjSQd~lQYFeOT@?5%1jQ!_LVNS5pn`Bk^@lWO8XK)wz-C0n+;f0)7^7plow=}E>lWS zA--ym2z&78l0$9Co%cbh1fWeTq2%k(R~FcE>)pq7FWqj0ReZEo?6XjP2jyILCwQ!5}BCjpO$u8Na$`U?l)S1 zZ*m|XdwJb`Od1MAG3ph@U}NL6p2RcCf`WqMW7Y3*X6$i1-a@ybPHQj|G81+>k^8nA zKVwXHD&Qgwee>pxj#C|^<@$4bc88^NF(Brq4p*cZV_3AG>*-MxHVDa1=*i0;Iwj9T z%ZrD<=Fj4yxT&c`koW8l^C$MR&v4=!&-q1!M~1hP-^A#09@TSYuOF}XmHYY zuIlLt4Hh5`iavFsddHRg&r=z!j!#dgemgXw8qP{ylRM#U`2YIK+yVc8=*KQTP;zv* z7MY!wc55-T{rc9H33*FsH6F&8l~Fq%_;5uNM#!v6Y}=Z_TGShet{cQ z&6^StEfd;RXC^Vbzoy8y_Vr~ux|mN}d~;rO@)1z6ZZ`+rMyM7-k*3C==e8Td?N@%kEyfj{l<4p8e=a9S%gGtt)z!ts8Xp(e4z1h3urNAq z?ju5R4SRZpBpO!Mt@q{lL_}>n^KCMWsr}JON%FF?I3Qe))P?%_T@RxW)CyJ_GkZeH z>v{SlJ3ISZdOB6zXFrByc$OyUpAQu1-hL)(3F2K*UK4&r5a=nu;Ef>diUT;-sP{NA ziItO;T>#b0s4a}V)V%NA>B0Jpb9?HT6&*W!#MxF&_-s=EaOF3;6IHKguYz18CLv*1 zq#@1QagXTkUGcr8?y&}M@1PMuhb0`&SA&0M11L=o)`qyL__VbvY!-iFP=2#t>=4-h zEvk-Wf8bZiA(i?jz`@DMZ@vW*Vf_9R!ipuIhX86=1w6~C_ z2lHv+LWa;t6qA&^O+a8Dl2*qD3J@Wio@{TD;LgE;c;H#&iuXOaoE5GwJIbxVK-__Kd<=-e0o1C2d zi@a3Q9r?l_AI}vEoq$p1%Y{e<Rn?u# z*y;Q#OQcdY@ZB z#~HG5@8mO^+S&p@x)@(lD}k=(-Q21PFGq`vtPMX4d)E_Q zLc_^=E#`%l6*G`slhN-lyW%(kVElDbhL4{j_2`SC@Q9$9&HyQoHj-u9{>w|C_+n1o4vp9c$bVU4+vj=lyZk* zQC2229uHOrB;&eG0Srl_WtnuwFs4V53VBpsRn^e=tX1z(1p_Ou@lKDDmXmt{`j;Y% zM6JEO{UJ5AXeb%4@|9Q6mxrE@h+G`IIMX`F!YYtxU_39!OOw4XH0$={l{`lqyc?9I zvmpd?p8BFQmW<^cZcXJPIN$vJV|Z4f!Tbv74wj;17@3kZBd6`O09sbE*Qu0thGvQB ztBG>!SCF0_GBeAfCIH+l*_D+%sON5BhqrP%Z)l)p9Us3*kAm(HrMiz+om&}v7Nil& zHZYXIUQzZLulKA0NU3<6PFB%p90DL{z9xZMGR*vO$Sw;nuO=-$eeC$*;RfUB?Fw$G zJGKrZkqleF3J`a1m*{Jo_9P_VNrBDpGIU~L;g7=gYb`|XxzPPW0ec__`G|utGY(p% z+G{#8sph%`Akk^f0TN6@Ei=SxnF(a;90c%*ZKm`n)U2W!R*Z~{(v0a?SXjZ0v1*d+Y{jmA#72QOp0*(c7y|7$SbKF{UCeCdL3$2pVg2MMVXl z!_o)k9EMpO2%vawTQ!(L*fT=~y8PD3*o4gg)=MI-Z)@i!g}^A?CuU}5Rd7@MCSY7= zAGW5Wr_UwfFhxZYg!Lr@EVF`wy3cr>dj+5OMJ*~QDZP%rj#t{HX_c6UV%@k=X_e}@ zyC8*@1%H3am_AtLu&gQ~;&Zx^79OQ6o&N1xR<+amLl|C^GE|xuToc7~?}o*?PcJko zTvm59=i|AozZ7dQ=cpIzaqT%cIJ|&bqzX|J-EREQ&g1ZT>YJmFVmn(j6-k|6o11+> zQ&259-yX}- z#Af@uQPLan@#7Dd8P5P;-)msGBC;4yTvSP}ar<4!84@y3-#4j90^3ew_8ll|{nbvEU$|}M z-@M^TN=gFQoDh7MDIz-BA1rcz|NI0&C^f1dJ#k@q!i$O*?t7DLD75AYe!fa3E@)+` z-~;jt2-y2AiZ`aK0L5l5gbi#Yoq#Ul9vh)Ul>PxUkf&P{EbVx|<;pMxl zw=FEp3i&l_Z>~UnIxJ$8!-kjoYKb+kCIvl%vLFn~s@zYdCuKnT z1@GHyC#RN{7BmF~1$YQ6j$6xre)<|*)V6hkw&++4BGzBq0yAMyIomrsjkYf@ zS6-F98X^{W^GGE__9diKL~~nPiTjaFwd3m5^NH$=g1<6k?{Qh(r=ofbicCa8LRXej8jtUnO>w35JxNllr|bd$4b~t)_d8|N7OJCX6zwM1vlJ!U6GmD8&Fo zk@D&2rDX6BKqEIL)Dbd$BDf(2!5ctJ(QJQSsT>q!;z_tIG{XKFEK)yBuY--XA>ZEK zUR6&oX>DYCh8+!LopQ*uUpOsexL|j@0>{c~@kDrVaB)p2Bq8&G)^N%~$f6mDvNt<( zhIYCF$N>f6p$xeM9wiptYE}2CbQz-P$Ve4t^mTT~u_|Ug2}QgGi=D9(R(tTy7a>G! z!+OGpqec(u=p@RlC)DE0y^wBB7w4`lrXo(8<3n5ZY|tbY8Ujpfttcfe9gIr!bdgfp zOLKb!goGIvr|WDeq|nk}(n%PPln@sm^DIORm0N2=K4OJR%VE}28h3QEoXix`qC9uZ zDv4e)DQNttb4n^R>Re=b(`sAq?*Yqc4FwQ;9zH&0unSryCBtfaUkn6}=(+nsgFOOt zlZJ-+=8El}#A`%ES$fD*L!uE74Z__H){2bVBi1j{qakEoa>ZwB6s1mGVH!b&QevW< zKh|9=#3YV(pR zK>%|oM=C-iCeL$>l*=kmKGSG8Pt$}vg27C!P%l(Iv&PyRSLucl-#2lk(MR#!Z`{=l zz>IW7dVjKcQs)tIikD~Q+xN^$SOS8ALE+&e=*E-JJ5z5RmI7~riMQxAFAcjv^el>P zchwH#SEujZvhC1xLY8`Ac6XHJScR{sQbM?ZePkm6t=g9gF8|7?kdRKfntV3DN>8Zc z7{9vx``4(G;^Ht3j?`&v{L-keqeZXLp+p#^Rv@X@Rvgoha2nseOI`Yz^FoIEpU+?H zgxXj0pUvM-_kWu|eYcbO+2q?OW#}?cg6IV8V7xyIG%qx7P>HQZ5{gr8J(5WK1TDre zUkp6^r7jY!a;u=91`O-pVIWbW{Ww(3*3yhE6*FEk6*g0-z_zw28SBDks}8* z?8<8wm*p|DWD{V&$7g4;Cci)}>L9UDx--Xh7dU*a>)vZSyS@y$WFc}&O3Lvv%S<_J zJiJUOYab|uDPR*3eg#uMLnc`IQ?GxJpk5y-g!WYrG&mw4#Q>ec+uhxLL>da*Fzn;U zThByAH7j3J5r&|p-%(Oh8sGo*>p3okz>Pa)q^yvnQSW&6jF{Vo44#WtCgzd(@B5$x zLUJ))?1(ltHD&j{@X&iRi-y7#pMeea3Lb%NXyXqQN#vKb zP$po=*SBE`HpWUc%U<0D|799HHL6NL6{M5C=qjW27Rv+T7*ibya|{^fa^sftGz``tijAm zL_`EFKR_{eq_edh*ij+i&i+|EkN_A2;zwH>20~q39hZ#9CpMNS`r(U4!y?6W>AS#j z-zO(iqW%CDX<)89i?RBzDl)^#c%RA3P6Kv%^j9oj}xn(zajM3|#*P{m#p? z)66{1p-zSzhn%@2fSJE zL`#kS*yGN5cmoAmWevhY(KqNr+`NhSSrS5nVxE_LA079P)}3mDF5X>BE-r0k8it=9~ed;q}#E7wfk^7H3g+XiF|xVD8Z&mCaNn%uYR z+IAN^%LpZ4fm2I=&D;4^HbDY??7C|Vjw~hVBG)}rR~yhd1LL0&h(85d0>F%xj_y6c znw@dW!f2=GbEE)Pqn&P`5+bkEoo!_z`T!7<#`jUo_ND4S#XS+qMQl)5*b?}z?(_3c zY@KIiF&sjpG9-;)?(iev26#bZ_$8>Z)}jD4OrT)dIXXrOS^vBIp@mXyrvL~as2`AG zb!#@s?%qxBww@m8O>$NQU$ROJrUiGXy6CF!se?R*dN& zq6lN)Q8Zg|%?x^CDyjWD{iT!xZ@GbqNkv2B_xraZY=I`YMS&bPCg$wfc7r4%Ftnvm z?&;q#NLcze+jU28Hp@ofSQT@hSF|!JJrlBMJ$g0B0D#D*AOBsS`2u)t=7EWTDIaW| z+J=vaZN@MB7?}PIzVam;-Gb(@3F3+6@CVpXWIlRzZh+LEX=o77c%88z_Tm%8w=Q$*ku^Jb) z<=*6x7?7zL0H`j3vVHB(TMUZ(G&Bp<1cJcf>yD<~eI*@DfL*tNya_DBVgPKHVC%m* z-xdzm{K-KiwjC31v=ag%%Z3%Ql!u1^pxtl4i%@}Y(N6=iMhl>@ry?Q<#leR>R-=!A zBuT(;lpcStgbVE6fPLBwR!$fhpUc*Vp9c52xCnvdfgq=Fv^CZK>z4skFerxsAk5A} zNrl8%V%q)S@#9aRR)c(*1==wzv@vMhrB-7EKJY00006*c!2?P^h=fbw!WRa>hEP2G zE|VI_pigU@H;Eq`UPk~}bddIS<3Y~(ijWRFIm`y@ghJd)uQCR5c; z$bI!E-qL!s>A z$y+E1;rMnTfNjUI)Ko;r$Dl#zSSJ{)|<(#(A+%l9Kj6!W952xa6n7 zqwmlq$&cQN)%W@eMbl%ag$N;VH0khxlrtC&n=oZ$9bD;i)9%~AWa8k-lHtadEY^_v zQy|}>Bo@&3QH2sT094dN+&tjmP^sD6SYL-jd1;Wo1r7#P5GWTeL^fR0at%-52dF)8 z4)G?C|A!#5(sOgig1+W;x=If=Ruw=qA^*(tKr*n=%V;cfBm6d-s zHo8DME^u!yzg2M$b*^`g)xi4Y-tKOH#)}+P)YV_39&mBR0PKaUub59(J_b;k1}lmf z%-2@GN7G;9Vhf-`&5DvPvVExxL!T6hYuyZS<@fxYky0OI2~*^TkCkc0J+0XT}9 zFFdzMz>L!NNMQHE7DD+?g@h;o9|CIp%Cf$?nvE2G{1|$4L{Q38Gi)r9{m^FqQBo$J#qO(l9bgoiYa7i=M)%-e%o} ze&a1eXlVKQ`;Xx~s5-{yz{M4}#)cDLRP5{o-rkqFTie;0NrnLTaL9N*V%>ds>&_i9 z1B3g}zpsVl264z?@WW@Fk1$alPLFXY4&wE>b)G51B*S!A?4YBg&n+*5Eczms5cYv7 ztOW4${wyVGuoDFZ2cy7iPa^;P+}u?WMa5ea6}AlpZ~QB7ONBs@pIij0@bu}^jp@4h ze?o=J=6Fkgy37?G$CX+03bM|2mQPk-CtyN^PqEvYRw=V*N1Nc?yaq>vlz}J zQ5+24nIrtM{*pbeG^Rde9;ngJrlxB?C1$;^_Ii_r6#x@K(nJ3NF$~8y-@*#L$02Ei z0wxds{$uzT1@9pVj90^=qOs2GTIepB3Vx9c)RsdX1h16Qt$h9T%o~>U3Y_2JK3R&l zg?5wBH4NNuD6!7ew7;l9uVU0^t)me@qJ~mK=@r(XTyM5+uBLQ4{|II0)-iCVH$trDflQHj2P1mfki=aKHe z4jfQ26dwWgu@Q7O(8b`)R2#rs#AoQsDL(lDW^6i?WCyTY?4FXM1CaRZ$a80R}RZ>fD?%Dh=WxTmA6^kWL_z z1EsZ?tSss_dJVdL01O=t@Qly9<^kFUr)WA1fz~VBuTpn43a7g|AJwb3O_NA35nQkM z0~f^hp!+jOe)GD6e4-0-d>5jB@aLUJyh0Da`G#QjI@pi1HI)5AQ{aDuex%)^gH0{ z%T!L6{y@fCtQxlJ3@!h(QsJ5AZYt|}01Z!oG{J%4C?2WO0(ah6x*(Bk+91V2%D&S| zNFA|;J7BAYm4~`rYjyb)9z5O^u4su6W2lzXhjdwgacXI;1r5=iot;{U+;m5=XOk5F zIqEO{{CEv2pT<*1)R%lUKUjvC^Ab}S@{&3 zVAli_MSeu^@E{5b{B<#-#ag)){&q8qjV&JR*=h$QF+D!+^JT3Bj zE+Zp)b|}}ErNjxs!1wQ+Up-EtZ}Vqv?j6A9m!SBAtDL(^$I$S>+DKtIY>Ii%p8cT< zW;0p2(|D#|YK{-!BcR^%4500fe?=@MjP3`}YJX_mqw-~;;hUfzd?Zp1?SIvxG-z!Abz)p#XXy4FpX_(!&^d#=0V^<;k2$US#QT!kbwX&y#&|{bm=WN9^V)Z~0s&c@nwaRFC?zw>#OE0CwxYlDo4Uprdh4rMF6 zI&lF8GrOHRanydn^1m8Eb5|@#Y@YxY3^0E>R?T&{c8D;=vR5yjM_dOo4-<*=FuG%* z7^Do4#U$Z-8j{L&4`)6pWBI54SnFm2u?Hf!ZWn2)cF0M=s8;XcieRH>Fr5~{j~2iQ zrqzMYU<~sKepna*898lNJon94Wqb;Iwo}zX+x78&EZSZH$uqb-eA0l{s91IR^fdxh z48_I8zrs9^LT?aCdjJ#xLnvgswE$(GMbHcu>Qh2Z4-5~Ngktd;MDGv8kFUa!W{GFd zt|AQJogZm1BWPYO{Q9K1B~wyb+6<+6VKZO9;UbQ6s^0S+V0m8}DF)r@Cm=fCq`E%% ziW7R6A+X4Q!07Q=z?Gdwsu5%xs3406O^uC;neyb%Bm4kT&H;9yJ93z7#ss7F4NwM~ zK!11-IOQua_hV>AoLi) z08y(AANJGta;o5~BEzkF_d+1zjleTQ3^{UXaTF`Yx%Dz7$wPaE&TybJ-vpY@o}-{xMnF6k)gy-` zSrie0qAU(j0+_2G{D?%*pqMUTlr9e2@cXF`Bz{}6@bJv*>&0MW-+sV#IChzusw@sI zy_U(U5OJ%#Tj3PxAzn!I;GRVHz#12o<~R;+|Hw$?^O;M|+kJi2WXm&Ek<1dIUl%D3@}r3 z6r+lmvGF%pWg-w?uKHBI*-3@(3V{t>e+$tJh_?k+)lWK|0BA4~?9*H5zE7bKC!3}E z5K;s^Y-uV)AU?GO6o}Ds>v+WdDCzXkDu>LK>AD)lhtYEYo9Napey6;JJnE z@@^d-9?t#r#h8O#0=QD%8 zR-PIWc#qo#J`eS67H9~RJIs{?%H9L3erL7VUvydowp9t5VlK%lDF z>B@{?`f33|*AFg7_|?_b0e^D|TUNR8X};|71&#<{MQM+-%?g;t*R$_H`~s&J7F-6* z51vD(+EEH|%d9u4d*I-a%_Rt?jRucyz{np%Bfn}GAX_lR5H^(`57>TCLk797_uLb< zQ7)XmBt-X}jqCiqP7g-c3h;yxj(&d(GLAA9Uz3FvL~6i-5P8Z|3m ziy^?kCA$R^aU0PLdIAGj@5DhH12LQl9zwuv|2g6oIC8-D`tlL6Sc4a`85ALS1y>ps z9`GGao8z2UM4+v2mP^ga`2rkK+HeR<3NQv(LKX}jG%e=Hy@VOK(v8aKm-C`gnkNu zHsRs~1*11L9w+wYn~ToYAD;HS5mHf6@t01I@O1`;8k1sPd+1t6(Yi?~dy1HrzQYcPv%Pn8S_U3|Rf{nGIPS|>lEsG7A+U{=#(cp;EWQS9| zTCTrdVL;~cW&4qY=eZpl4=>}_p^UmlYO1tM$xjrJ`a#+CjgG#HfP;BUJYMHKP>ZP9 zB4c9g@(nLN*WpOz&?{_% zWw~|xwlB1U?z6Dm1o`ZH9S_oQHcn2? ziNjqt>krEVm382`xGECE>Pan^p?-hSQSd_>E&}C_6uvz71z_L4PHJlbY2{u=EL%7T zZ7;zN=5No&gnw56y@g<_FO67C9C_5%H&%5Od_!@|Cx~LdXO@=t7@}F(iVQp0^3_5B ztbPR_qIq8`8uTs*FC6ZWSCCe6XoHIc99IyEIjXJOPD=rw=xJ!~KY4NobeHxbx@c*CPzch%)3-8N#bGm3Z*9+O z)R|Bu{y&I&6L2ouzFqW2hLCwqM3Y1mDIsMHMI;ItlVpkpLo$<$MW#wZLNZl^%wtlC zGL#CD$UKv2o%i#8-&)7_t@Ryiuf6xNcgK4i@AEwJAFlhle#7}Y&+}q_bD>S{`kWZ_ zm=DI=3;l{@kab*|9#@(BsHHZ}iwprRumYbZUfPxoI$!3^TU1b*aDxp|je|G&-L+dt zUYt-1^c+R#-%6sx`AP#uC35psTB%7866@S=| zRweFRL4i&4HMoeOTr6D7(MqMz(_47>`}ekhN+k7s?=`Pt;};|{+KXK)Ua?_n=Ers zSS6!$Bz%x*$!Xh{k~iU1)Nd{fp(qvvv^+WbsvGq$D}rrhnpWH;bY(W>iD_4cS|OR! zAqZ@W035XPg`W(>VX(;if04syoF^-koxu?w7Rr9SauU=Ryh02!^1Ziy>b+fvhSw&8qmlvD~UZG)BPe{|(W2fWdriaGH9`+S9 z4ter<)9iIrrDI@F4&@SO!C3}+zX6grcT+QzFQB4b`1pw8(DH;bFakwXfI>n-2hh!< z)}8?Eqp8C`IK)@!wOrKH<3}_vRMdxS)00+p1imc1-+#F_RecDh-BF+e$K6Y^>{Ok{ zOo~cPwWm{2_23czy7K`?m0Md|(|M}D&6C-(_=a(CmZ90|qS#ZHjqfm;lE7KoVwMif zpwa&r`t<%Gq87P6kcHQu1?Z9l5llfvLzk$%0~zC_5JiHbXJwemF!IbyUWg>5A@Qq{@(qiA$_pT-#}w z_~U<_Zo623?PMKmi>f)g1LRC@JoYyRp_x{a50^kkwk|d(PSg>qmC8MS?%cVY-zU>Q z-O^4xXq{=8rgsZT_%u}UiVd?EVn{@OG(cH}2I?n2|j9 z>B0?-8VTcz^-J|zHT((+W-phSj`k`)@^r8KBHNv%cj&I@sXqxq?>}Ej+xxrwX6ai` z9SbjI;oEzrIz@7m2ZnqP%3k&5`s_Tgb$Sm+0C%SBbO|*loA25Wx@pl5J6Ow%md#{6 z9MvPo8e(?EO?=Y1JOK{)R6!2Z$!e6GyY_A!XhU(lS%b}EZlv**>4`hiV3(oCK{HTk zF4=$mHV12&$B2j5`?9Awzn5EwXAfN7E}|8xyYe@D>ZRjFQI?GiZf=CcP}|G82&3!| zQ#^OMN7=T2GF9u4T2aq7P|j+5n=91s6PFTrws{--+A;1ed>d{LHHt_i$X;x__|RI@ zbk5;?tnTlM=KW9PJsvt5r*s_Ges!3oW%$;Sd>)PC)Gn__-zzpNbR^@Gwe(VP370krd!(fJ`=23EkpB`D`QLyh|Ni6hguvk7knsNg zbKlCYbEB3*k;=7ILv$o6I7GTogpE}Oy5ZNDqH9uyct)tU` z$I{c+=MU}2n`7lI5+hSu8qp9CNDL}r#3FJrgskFfBp$xiC(s4+uFg;8TxQRyw zS;*g>0K7M%zsF!t1u+D6Ec7VND_dLHfZSz(2!Tq0C@>1g=Ix33%89Ht7*{A-SXudFDzJI;W&)tu z5uBPmfQ6;)J6Ujo$*eE|FuWQoeeK${OV_S_FwjBcap&H>5Ik+A0~=LT1xZjXqw@5b zGZx>9oZXfe_Mq6yKmVRD{{Hs13hGd_Zu zNKSZ(Jo73_KzX!8&){G%;zZ^9_ZvX8mXwlG*3_iO<5;Jqr4>C(@XkxJ>8|Fh0&_E^ zNAZ|f+UFaU05B#WtI=j010MO79D7S58A62%MYTWD2Wpy z9As|AI5+QNo9MB4NeWows)C5v0xcDS#vZGBb5sf7F_Ik@h6f>_D>ODMA3J_r6_5;Q zUcux6K;T67mg3zW9UVzA_zpNjylT>u8=tOy;bhrib=uea%gb)6Kk^qz#hTdM8jsj;CIZzPO zOeGA3VGf{mp}+_dr1M!ju-CA;j-Y2S`>wHBRBbC}6(hofUp^!@Ef`gJRf)O&{;-Mg zfpm(&Kuv#nVtVr8(5wd-$Tis8WMb(kG0Nt`AJiF2be=1-9fZ%97dA~)7jSyuxe$yY z3QVrrsZG5#qC1+^kF0%9sjlN9PcO&7z`!LJm$$A**AduJvMicW0r=aSB_&w^YN17j z_!kWUa$Xi?Rzk+(-V7W3*3{craR!_l&nrk`QnRHAa7+ zF!cgK!menV`e7sNacIN?<`R}Q-@ZK%nxqE#2Vz^V?do?vX_hDI3gP4tY8 zZorCxHI#fiE9HM++m~-_;@8z+1+onV4>_F&MnXbcN%SYG^M>KV8tih;lYxnrCPtpz zes9l#Dqz1SneF|8se_!HT=DQ1FrvI7gk!|WZKbkx_+zp^Khv#$;_Xox7~ECw_BGv7 zs{S7_?0*l({{_VTfBxf)cvdBn)_|)1;{|B~!EZyEDKmk_NeM%p02T@BHBcRDu(hUN z-aupoC6GiEa&kpJA^D+>BsbpR;9&Lcn}dM~+}9{@MDS|}%{$2H6_H_Db}+}RA_7Qr zFQCKDm$vLA=>VNN{$d+d;y92LQw6bm?enR-@?8!)X%x1ZHIrYI)_T z$CO#S6!9{Fn3%m)d}aHI2J?O2>}-de;qBYDCF3uXq?kO{aZ_w(a@d8=#yw#hdzn7D z{-QKE>XU7TW)_&4Pe5>pZ%4L;Bu0sr6LBFBrMKse8!b>)0=eaM+kg!TgAR+Ft_s(2 zfgj#$H%uzLHUV`asq&Bm*W3WPw!7P&;4uIkkgr1R9oRW|;I;K5d%Wm!~V3VHj zazzA}fpIPNQj8`w&JS?8WU~q{sK?(FdIBJ(ARwAKf`mZAfmm;gpS8D+YHQac^_U~^ z`BKrQfILo+&&L9N_Q#6_ZfIpD-|z2!1WX@j9c6pY*?*vE;F4szB%j;eU4~3B&m=iC zWLQJ-K!1I1#229wMCj*j8~}gOI+CAw0l8BWi5M4( zVNUG_hcP;ETMnaIAs-M}I{YXe@adH?#W6o$e~nL+Z{fbCihgZfZ7q3>B`j;@S`8*PEG{ z$$KuaV2q3SQ-uW~p&?W+RK%b5>`=_MXJBt_99PUjiI9A{E>FjKA{9WA~6Ip0p$w6vo$<I4cw zvEfX8(c*1-biSY=B%@9M_)Jg!BE~l4sRw>o??=r52OGJUb{$NpZI57ngBF^FoBJkk zE+JuIN`#v#KnDZ?&QDGwAF0P-U|>M!Nh{|%761kZ-ga^*J?SSFoTSf?TV!yM4uBoz zNkk=DquWtYen8Nx-@RiLH+p#($vF?KOc-5n!Z_>Qr&(dqWB-J*?Wm2WUp@fX0M&W_ zgte{hEl@o%UZH$b6e)SRBO|ln+d0$j_I5zf6nK~fd7>~5=tFy54KY~%?F-)w2XbXu zmEt|mMK~*76pq@03j_XjM?e7|pnNr;TxfzAX{a#R%0tyrx-!@0!eB!daOYk=raX+F zo8xUUZlEGi0N(^4_3HrTH=YOKU6Nk=9!e2sE^;buK5&2st8ye$FO4`C@Bvhc8gt1W zwEx8diY>!F$juqSmz|w`!QugQ83xLj&F(lHk-lrDS1yG*ue@7AsAnSU(>0T9qkROp z;~D(rd$qK+)iA3BH~d7&614N{U(8df!4*_u^%O@6;&h{Fk{wbwXw4c1%h-qtBnMKw#Si- zcGzAe{C4>R|}P*Ck;wRp}xKerwbRjd^`Iq*lY0#5Z2{Q{-C z$?f7|_t+B`>}`)9J$gG`RZUGP`2-pNBuW)i41LK398e8@?z6@U)_-{rCcjW;p+~X z29Xe31>?7w(JpRmV(^KAOmo%IHj$eRL|4GC_sVapr;0X1(Cu5o^Awsk%wjy{!4r)JQ|RLVYV59ZUN;VKiZ%goVtm@mMy4K zp+|r*Rx$8z4Az7J#ccqfgI=x|ppFmfY)~nEk>-!2>GcAQ>jY7ioE5@DQEWH~=?0`) zHE_7d|Ar%}57T47^K{J2Bn<}D>qPxVCs-?KFZ@pOmV`PBrdu zA$Fa761Kc>53%GYh}~5v+R@nhB1wQlzY)p|GWq%W`CVF?a{vvHoNK^{o|}!0?Gi?D zgR>r(1DpNG!64T#hd)A4F#=_8)BHKdFVCrQET{-fwMYywuduKlSU$F1E`oC&Sx!R--jnG1qY5AAA3zgnjs#m_A^H!>j0))XxU-n%)Zays z!3tyjqOFt9P3JVIe|{?t%bgwOe&sTmnS z19&L3C_c5cn&PHn@8BKHKR!A_>@CT{nT^j5?dmi##A!z}3m<|*T~a*k?Cf{YW`orJ zdyzK~%}h0HZ9`#$kY~;`!28lj91HSq&{pZ^8)e7bS3Egpy2V9J$iiRrJgO?}0nGXq zO@vyCHuSw<(uxtCZ8cRjeC2o+;GH1o&sEsjWSbh@Y<0!OCCa$>m_o*Dd}gvu*|!zp zfB`b?dcfTz80~ibI`_9Kd8GescQvi=Fx}F3nUp0P#*BaHNX?3U=*7tFLqgPuX;zRT zG;g|+^a~k+q_h#{fUxuO^LKt6W#`;a_5?FVWSRggD+Iwnc-ywLN&sDxjCK2exKWYp z>1A_!VGkK}0xA4Rxs$+F&dl9+ zy^>I4qq6J3SLIQ?rzkGcbY z5`qWK@UrN9Wy`1If-|;~jd@E2saZok2(O^CW7p=Ur=x$nLS->8qIb*M^WR&X?N}jl z2V)a9UCTwN)uS`jlgurZbKGI;dA zE_I@6!1=2A^oboDW=aCnct+3#MLvGjNjw;cQGDsgxweb@NKq`v(`4}kCSxUv?cTw` z(~k;(0<=`4CnXTf$i@ZhXJKO#_j@sJ_gN)?4J1y4lGXxZ1L%aR5g?s0F1Rah%mp|F zzYz247H?+e*{hxAdB6 zdz+_W)gtNAe-xreRxszu`MG>9uy%5;0=vi!{#8D}E-2h&kFffbuKb45lkM*AvwT31 z;?m)}((BKBs1myy_qUj`Y{S=J#iOgc$3lry69iLCNI&+tO_-B-{KF&!1r!G48UZ4% zLp(Nja7avF#3)|{+AuKgnZX^hU-Ie;-#;!+B6*Sg9kN^oaasta zJfL&(bv@?C!qF7~{Dlaa0)QSVOnQE$Kr1Q$8G#2JY9Psm^sCh@KrAcq{frR)NthGf z)|<#t5ugQ}DongkFyh5FfEe`P&<#G+$7f@&UcQIjg<%pwprqJS?4%LZGc>ebBdXsP zEf`=HIY`eV6t2&a#7L414caX{xpe`lfjngnNCJ~f5p-vmuG6D<$73W-c1RTzsk3%p zc^4)gizOuIQYhnTG36tv<*0G#n3(EOp0M)a)~EyMk)$F?4_)-B5GGtH{~*iC$|=Yg zqW>)I$&>2(dKv(J=-hlz-H`8u=Z=D(gb>k9-Be0GjBWk#u?9QRC-LU07bNCHzg@rt zG-7_(XBKd-LD^n|Lq`x$f>9B&Xi7!F(jf?vZ$^5Hm{wqP!y{wQgO>Dp!5m~wmSCbH zZ0&raleF9QCQd28tjju_QV=WD@W}gzP$&L_dS)TuVz|x1$vOK@w+CCebdQK${u~U9 zETMrQI~y`MVP8~X_^w1p61Px^oEZ~;Sx`XjWAfW3wzy7mKLEq2`Fa+bc&7WPin`|O zXF2Jq@v^s}eX&l&Hz8vU58PBt;%1|$1 z{#L4k@+`pe#AC}*qzKZn2RqUkv@>K~HAmK$`T0le?MI}Hn_TWmXt4cLOyQ|`4Xj>0 zcs6$SvRy)xXVRn0C4uSI=dia7!?gcg!W<@cR%i+_yR1cbMfM}8n&GM?!*jYr)b7v%Sr^`x934)X>Ha5J$Gb5=IOizlz zBP20mpD?aNiP~}Fx~&j6)Wd@@07ZTu`pqeKT;-EA3H%%P9$1m$2J-vN{y7DO!P_SJ z(5x)Ga--uwoEOaTo!?{Epll%16ljT<)>+ja01g&q0$=D^y* zWM9$MVJ=K;ZectDB_$petGlTnf%b46DE-hq>XzS}M1cXjVGfkuA0Z+je9FocpbkxL zyXcG{YX4h1t4%4j1($%SNlNv3i()Z_S38nI#KACwtu2atl1%41_RSYT0nC(J@Jt|) zOc+dT9qX=$B%uQE4+8(;jx>OwLn14Eo&?EbFvu~vh1^17l!*I2WK_ZI11ydd%-ccs z5`Z%zBSFr3;g=MSH4&1P2icnHRf@S3IFe|j@J+w3; zMO5AvQoe*D9D_~bFfsuoHVi;LAts?G#Y|+3pBe68M&YK6xouBR&uxbpX~Lw;gA;yZ z3l(ci6^72m!NG(3wytNg@yt$?i<>O)TJbycq0^;*Y;4sphI12uT_rlyUq|SCQQBe- zr%6Wbb>V8HE{U9Ax3|k^F&H^c`-bIok(5>ig-u{nG|v+DLhE30qx;{>H&cPdBaxI*llEC#NLwbptZpMQ5=Aut5*a8|z`9 z2!!3l?nt1@=Dx)cGJv3vV`K9TyVL7bHp>;e`>d#r93V~Tk0OEHbDZ9Qa59687<1|T z`SS}%JChT=%CM}WLu*q9A|8zEz0bL8ZD0kbQ{s2s3l;sxC~L|70r0O63nf@rjZ+x{ zz^M~>IM!=uXpsJ1K>J_=MAG~@u4(8WT`&RM{FdbDzU614qNMB}LU^gE(_4J||7Xea(^y*%BinUAtkUFxSZl^U1YnjY}__Tohf`UM$F5W)*q6Agg&>S)ox!Ys?XPr_@t2-02}i_uh?dESw6adAU|QDW{@u!4F= zN5kQC4X*$zq+T!pB{KE!z-{N_9mJLDq^d~+i8!MmKG9d1TGo|Y|{c5AjT0j zP+Xu1Gc_Q~FF$NH&b`>b*48n(S8+b}Ktl)D6kO;aIyHj=iA=hHlf(x1BG9heD=|We zb{$q?mL!W?TRTWEvlhcT5(_0;eIOKMKVa|Rum@?KEEzz6gN?6t&UDf>6p3Vwf=~;P zg5>C0I66ikC<#Lu1^CZ5HFe9s&pXd_{4eI6e`Zkf?5X8aO_4A1UONP!ND{Mh47zAU zW9adb)G!{z93ucYV;{~*EhvN`%!d{TtO7W^B4~l)`j$a`vtNQf#t--yiS46*V?of6 z+`nHP`<9iRoeJ17*0$!u2PWvvzqIGVfe8~hMEFTjZ5%vKD?Au?NKpSNp~yUpS~m;< zs<^JXrG=GLi34jO>AfIbFd~MLXVQ5UMMr`I_%Rm1(#hfkp9WuIbU^)ef4C-%P>%AM z3aLLxDcJ|qIiE0tW6*n{N7~$Q@2rW*lfvH0W%%dL3iGl$CNQqQY z^`NqJ@%i?`h8d(pkf3(vCUtodUj$;mljc8%0l1L3D#3ds|6ij|W#BV~2mkx@v)2lf zxu|e)JlBtqvysCA%xp;Ot-o`lp$KWf=E&gynots;fSVN)3R}>3fa=kpgvA_7_JiWp zKX<>ytiUG33IGe@3oSv8LJ(V30f4=b4|@%VJd$OOnfMC@%KtF=To2xaCDbD#%n5#H?KY-Mdx0yk=W3^k%I5{Pr#t7^lG!j{-8t zM6pjgM-{nLysMP5&Y17B&X8U6K{Ct6YK z?Cgf^Q}0~9za}lTj;x_Dd?)p7xbZ>4oCG|1HKItN7zs!Nj)(C+C9#ZSjDJvqCX4>( zZG?ybwDukpD0@*o!}nu>DAYLrE3|@>7wxUftMTb~`&Fa1;`8-q(ZGhsbJRaLSVV${ zLo4Gyp%y?2c0fn*?q8^dQFL=R6#-RrKtMq0&!@qr%orH}ml7G-i@&(hv*+4+E0Rh& zvm24)&Q5z>F^m5iv$%?a5YyM-!2;x8KllF#Tj2Nog{z=s0$`zn$HDNL17Xr5e1hbl z2>;KjqyIdTO?`MP*c3!VKUVh3IeATwERSYBW8Sf&xq7#M2rmid7>aBk;1^JkC(>PF z=wEY)stMG6rzTL?Um}xlZv4q&&5Tpo?iO)J>W&V+uMnbtO(z=b7D?)({$Bt`$Ta^S zpq09Jsp>kHwl8c)HoAOvuC~!G{?#SJh^Cw_o?tV>O#v*_^D@0Jr8m_dH}b!4G0$Ls zUx!`EykYDl&wkqt?aveq9?jlo<7W$clhGMh61l9G)kcO*OUs{^vjsUg0?5{iFZ_wm zuGADi63>pG?DA#`|8YM!AmFY^T-nA&I8L7@++GTc+Tj>+V0%RCw)PN!niO=>-ahOSeQ%DU<6K*_V>VoLhJGZ8{> z+uQl9n>x8pf21S=g;Nt#i2Vr|u{FFcP|ve$Ji++)*;QW$!_*ebj}vJ;!=G5QjoV|> zyS^Ir=~wcX>bB%p&YfZb<2eFBKNTQPb3u7b%qa>(71a%7p%cPV_QOZ$Q2FV?pNC_I zwBp7a-bff$d9TSb@(zW5U$wb&?CM<1m2zUbazWe7>rUF$XUr({RDRYM6Wf)#2t z;&hqEK2l@ostavr*d{qHk22Go>rlXBTz;d_Dy3uNKQkgr$DG6A7aikQY6OpJz8z%x zeBtHk=bnn9r>>|_bJMCQ9;*xz_r}61=k9-WqvK4xr1hrQ-F#tn3^cu63C16jyX*&N zgY3I>G+G|n-ru=k{Uzyifj!UP*^5ks%iz0rE&0+va>jz(8C)B?=$VQGmDMGFelHO< zDehA8 z0miDJZnaczK_}0>gEnFd?oNdc8c_jjcKREL69t;opFNf~V3G2SnNPe}75V1Pa#PV& zTg`#l)y-Ry(|Z#$^=;OoziZ>%3!@D6QGsj99ope93R22XX0Z{Eu7-Z)`CBnRsa7{7 zx!_WEZw(Sxi-0wz#KQ4mF!f0H+(QMkIgMt3Bcn*B2@usn3Jo3PrEQ}yMt#f702 zrqyB>?qtI)le$YZ83t{feGOOY*VM$Q2>yc}tQ6j7tBTnbd+@vnANBW|6RYN;IVR!n z3P&B}bjU-KvrawDY|Z$y2lY1;7iEoF3rYu&);4oUpC&+h4a3G-j~wwx#%3fr>!34OBvym5!B zrxJZ^w!F`i;};JnOZxvgZZq;tUO-0X$!iliSNq_v0%I5ES6?r@jC>`3V`?F%vwZqo ztaa9@t91hfRu(u5J2&oP6nss&BjTN3=nE>o&$)h23)FkG`?su1s)*^^x3}GC#-huE zN6vYJQm$&UmjY*O_5q*4n~ev&SGoA4S;+AhsU-UR^b`YmdNC;ZbrlxK6JUFg4l4wK zau+jJepFkiW-B2!%(wltedbTwaQAd>W&9p1f;101f)a$R*2q~FUJ;s1c_;&d;HH=g zh7XC9f_d)`#TFIewYK6)UTsYNN0N~b+4*lfom^Tn{|c+kK9M?j^+IsC@tT^}xNF|H zIo;D!R|Bys{%hNmfo zH|3Z%mC-d!&7Cfoql;M3A!Rt9|41({d~)=ZyBu2T8JTnZ;jn3gdj758l6jtAqI1oA z&t{5I`3qg2e_@byQ!O=6d%nB)sHV1R_N#86Y=`>E_=JRHwSD^oOhW@$C~1@r8o5_c z*$t0;A5p`<%miI|oMt_$)Nyh4qs>zg{W|EGB&C`>o*Be)O?jp+n`|L6rVOa1Gt8*f zsn&dpi+OjSEvF(n%a*pnE2CP_d)XK1safEw?!R*Zl%uu|UitF+Z(?HNukpkO5BkT! zmB7sG7AdCrBqqKl$Bl_E>B(*HAT@RaH##VTm`U%)uL32b;c^ zv!~}Y+?#F4yFBeCZKub6zi?AS)fVpO&(@yZ3!mlTHC`%QK4icY6dOD6arhGkD~ahw z;m5)!Vz;ihwLK_(@kGI4+R^?~+8u+o0|WGoBi;07VauG+=?=Aqsd5l1OW^AIM9N}3@wt4qyfz8L(L&q`GGX6kmf zx5pN73WS#wbW48`jhC%`+QyshSfKFDV$mtf#A_$V`J--8lvW>Dm3qF0Uq2Ju_>FH# z^w*W}V~w6=OLG^QE@!=lAX$KE6TNR_Bxz(c^)fMC*$l2I)z#Jgj}nSib(2qQ-m^#7 z6!bHoR*ln8qc}iPxDnH;YD2Rdfrz$Nm^lnupdb)-q&3PxK4dprC62|d!D+mH{-&-lv`T4*1IqC-_y#VOOARRo2j3ZTO7k@ z5LNur=-&C=cQ=l)IkFV9TmCfS^V__e?wyFJT+O@bB`dGX=lb(9M3lOluR5Ro;PRok zIpMe0z{ATAe=lsXwCu~56J~lnS5I+UF?t=b;g6ifh+J-UU7Ad(2XFl&co?P}xv_&^ z(V11Faj28)8b^(qxS!Ew^%8qK3&$X8v+>IA-@9uHj+`ypmvi$=)kf!K6=5$E-dl}= zK}TO?HYvM{I@#S*J|}$RJoUq7YeTlP!j<7xdKldw zjlTNBEme*n?R}WJVEj}G17i#fN%k4M*O<{q9nHmPwvSYP@3Z3q);|ebOd&BbM!1`R z{q0{+un*%hdf=;!Jdy-K3jc{p(6B%@nm+8T+h&=JgMFOWW-If?*1E?Z^YbK?#kCmV zMGvWyK*jjqb>jE;D|L4a{H%+6BIMcZeBdBY`ODW=9Rr2@&FUL2@BjHBQU1z>d9gx= z_=w<)lWv&}$7ZIN__3NN4$8&4Yh0_%8=DyF5YV(9U5l*ZXYjkZ<=m*3w7a8N{V}76 z@pAsa)P~(XGhaAP8+TNf$|_IVmi%gjCwC3Xb>>9QG^_ybXXT4gR?ps|Sydt)Ip7xtjc-=_PiDzf_ zu>QEwMRhn~crRM__ydI!^aZMezgBjtmA*dBXLZ5Ck%so>>$u5q8)b^h_I7D5Ev*`A z1u3Z)E{a=Z0Vq#w{K(4QFcZDI8*D-zQbh?GqZ6BixySiDY4{J3;zhu)7SAxJ)$eP zc;>F`hbga^S>DDv$QV24iUVp`xZY2Wyl^ti5E0XX4Gczc$Le?SP1m%waROw+>OJog z6&3Xh3>>nt4|`GccGW9Y-QNmP^l3s?hogDx%&E?jK9NIz%})e|s5~0^(o`3!LS&q3 znko+!DfVjw@8{Yz*CYwkt$bz@A6i>eYOlXF>h_nk?@KnI0PU&stAVNS1^srLt=u(b zqNzvMZGLYb3m1^}O?#QlCxuyWM@Bzo=0ZQ-b|UDnj7)A~NC(Z8j6WxWGia>Heg9ti z;({Q5$nvwNHJeksDYVa{2@w`PBlZufeUyvgAFLp%PU#gKMBPh&2*~$yU3|MimPSu zg1@qEEhfVW;3-qe-gD-xK>h8Li4#sV|`b_v6GXMMyJ+IHfdqNTtg3SC>VjA0V6W~+10a^w6 zcg!iRZFiT6i${~=lbBKalD~g4i|clrVVkbJ{D1l@|EyR{RCOge{Z*^!=<`_uwGsd}R}EEE3v2Ah}n{$-q*d z`N=aG+*?RjaFFWa)u7LYJWH)8Mbt}}bz1kN@ae&eE$_08$b&@A3qQV5v+pPmR|~g? zfSd!gTAGA}{d+u%Bp!ON+JbubQgggBG$cgw{JTpp9alK~{1rV1XC;j1#Bl{*`23HJ zA^5zfU!14LYi(Oyov+Y6;6Bx?`8D+B3zuJPFdO#<5bidz2Un1QypVJHm1CP#_gwU0 zI$w;Eib_O#dUdC#e!d>(Iywq?o$097>j>+n@|cHqEhz%HRD_bie__&THn?=npr_4H zPH4-PO&HqL-go+o4p;GtSjloWtqDd4x7Nokc)7!AE-IeOvKUWMl$U=O9HhN)W#-7|T&ML;`@e` zxM+sem9(lqee50w;zR1k3+t4WUAKrI4>ibs*nrz;!Qlx{@}du{Si5xKV7sqUa>i#K zMDmHzSGzS{~5Q1i5BSO7zy*Gx$fi^I(`K{5P2d}u}YAIcPl&O?!_RR zX=y%b9WnBKBUQNCX1s@1+;bKdNlI$!#YlM?7zapHFvgZIBzcwW!G{1%=!aK-mscyD zkR;4HVB=-}`$Ou{W6#{(_c;HgrK7(`OT_OVm;0RCUD>aI%Nms$A|ya_?{M+~h5nIV zv5@>Si;2k}^8avyJ*Cu(rE!|^@U*>cnSBAAyJUnno3Fd8$sG{5=`b!Ir?_k17c;7x zyQV?XZb{^yk&z;8!iC_*XswB`Gp!Y&wLKeGzRFuHn`5m(lM>_ zfwG*Uy(xX^M`5~+p^6HBO#9iG{;w0+#t9&7ZSh)NRDU4vzCLdEcc?t}9Mx!4GicvL zu4uZ=E9`XKQ*ND;v^cqNYDo!%jJAbean=}ZFn=waz5Vc2US1`9_SQ>Cm~Gu&Ujq^h za1aU~%{8TU1SaRUD@wsPO<`Bo{OyGov||4LTZl)tpU#6;JWdeYyB~NDj-H}1HkKn? zqq{H#LYZdj;Zwz?w_ydyAi!{G$AMM8p^_=t-*-P{+=yUXdKUj(jSzxL?tYi+l`wwy%e z_p)tY3x*deyBu*XdJVPNt&mq1LhF8bE3)i&x|ny>j^m-npD{ZOiC}0OSmd#f8jOyh z=;&m{{rkP8>vEh|(k#5>P1{ONe1-rVzE=<3>o-6F5_6;ZbLJ6<{qD?kg4oj1LJR*6 zg_jp}nQ`V1>*})l`E9qF>UV`?iHWGZw{I6tg^HNzGBsEZZ0voI3Z(<35Kc*Dz9;4_ z5vb%Fy<@HRrL7h0m*q}B@7JOex_$qCXipDst997yu%VXOLnXSQscg+(u!r%=d-?3t zz-o6<^6tq8y)J#tt+tCTsb&qJ<($yh4*;*Hv*++2mc>JcOA2m(X1NBli%zMSzjG_~C&H}|f(TJP_Cp+hqIMc=ZrEzoB9 zHb!^X;jTrvJROg4SMB?B;d8uXufxvrj0^=9CG$U@_h18r4GvPdtz4DBxR>oF(@F$h z&2EY}Ej>NKMDuH@yLbT0^Dd{a`o6dwSrx49r|edym;oi*b=*tj)=`*cEG_lSvuaQq zYWn{5D`s|c!C~J5;gnB7V8INs|57B%fh>mm=XnvdSkXtW^PwMAchBeG;4p8Dl1Tez zTr|OMD>kSuSQd#@gZ~NtZj%F%?$drO9MZgq9aRq%ELfC!LL~;s4~$no^|Q0FU=qx2 zNT0KK;!!SgpMe4}A1>B9AITwc(~<9>-@I>DG6VrJ=X^`Opb^-oswM2q@(-*?xW!aH~7tDtOx zRMpBb$Ap!r^`Th(Sxuez@q$W%cVB}JJE8LW+9uVbZw!+4ifShH7ShS9(F@d;57XF!mDMEVVAy~r8M&CGktaRt5>5$sEc(;?V zzp-qLO2YSyJJgI%^K*Ie3g*X>xVcFR&KldCM^5`q%n4k0Ue@2=RsFpz4I;Aq6IrvC z(*txAhau50ytZWKW!t$5a3!lGvzyz>6r2eL*lOsV3`EjPa7R|nzu0!3;xOoO>i7yH zU9L(Yw&Z)NZK=(-TAy+%t6g}D@{XLdP2ADj-@{4Fk2(}iVvC$#BcEI9;;lOKbU}7B zGbw)nvl=Z}ZG|Fczc{M(NwXNbKj~zHVO1<#(d$Ueo62j~q6d#$?Pw_iK) z^{@A`bK(inv$bDd7^=EVIkdcXRU{v%lub@BhKH)SEQyY&uBti$`6NAhV7MYaJxDFu z4-+FwAK2v-9z7;KBT?VC&%RRxAF{43%RNOS`h2TRo||(pUia}e^wKoR`g{?HfBvUm z%N$!+VC2HrJE~nSj{O9Y=(ds3O)AIUxjT;x4i?`RrLZMs;Kocua%^>yjEp4D}l&5*8u(zr0Y>owxn0f^<{WB~a}8Dg>1j>9w{A@a zN63VQeyLE@3$b0CQayTfftG>WD@4%SgjT)bE67|FPmZmR`VJ$)E~FF+l#>%PBS8TH z8@I{`pD=!lTOflC>i@nI3{?%z>5h#9_GP>U2x?a(VT$Gnc`jKLPDhU+t~a zpjnV@!sO)dk6DI_L?wg7OMlH@L3N?Z_;)k@11#&)TB-St(=ZACpyiJHOKeVb*ozw% z_k)KKRCc}Q*hBBXL-ly+oeDW88#E3M%nt7t*0u8F*WeYlZO{QU@=IB5O+25R2E)ttLr@+zF0(-w>xOPq+F ziQXHq{lqu*+axEGa^?ls>5|Wv4D3)}le-e8pcoeOJC0@(?m7I8r9k7e2Bu zF-0r@c}X^~F@JwAsb^S@e|kVX5L^yuB1k_T`>w)K-+E7jKnzu?z}HVs&N_kC4HsP> zOPFH)aovec7%Mnzu$_%)J3885=TJh>qet#;b6*BXL1)1FPiem{gP*Ct(VnnqHa>}u*g#l^%jCiXBgD+VMht6O62k$-x>7Bmrz-J9x) zixqawu1J)39=UiQ37}_Ok>9F8+ZWpmwL!d?E1tYl*{_=d3V)%6%YB?|=n^j&sXZHd zI;f!Fcwjvv_?^nhlF^62t9y3o_Xly?6nMo+EwcP7KNl+DWMjjeU3lW)Sw}lNa0?79 zK0f*Zx=xzgKtg@13#=KCZ)j|6wh_3m$#CTB)pce!X173cM}d6}y;1ZJynw`OsB@E4 zj)hqqOEF3`6?Qhgah1Tu=>BG2XPDAj8DVRksE`a(P`sDccw0YOT3APVOI$+^#llVnQe%Vd%%gxc6dbTL@X{{&VFN!Nd zxTNoe>Agz2b4Pb)->|^`zTdbP?IBDwX34&cm9-b!n}RHrD=Gv*CyGRCBqY=;7!c@C z-6S!8J9ej7=|(fvxxUYticMj^mKQ4?>6efTGlY99u2T?~hFVogE;AKsC{8$ylFoO3 zXS1*`!7u9uH9Y_F??N`SJZlXtt-3TVjSDS5Wr@QmE?C%}Zy0V5OuaUGP;bfgigXxj z!Sm;$FT9HkFaF^qH_^a*V~ytG5*d;v%ZOfe;ly4+db-OEcr@8kx>@^HoT zAr(yeBRmxgmb9oOI$tzQ)Lj4l!GU1xa|vX@%qJI4Hq4Jk4waP3*q<{g6h<^THaGm| zlyb62oN?WfmsOYaZYM`ard_-2P?S*=$(ep_&fP&in|!Nu{R7*#45{(Y@dG4b}KPzqvs7Hw!a^TLs>EP%F=! zklM^ooV{Oc?pw8_#E=#EHmzgL_tGA3O||`0^5$)c|ICaqf}e=;}6L{#* z>WM6=p57M5@2}^5;e_yaW>O8i&eUsBR@#0+|JF4(MNzg;qeR~%Thk z;yrHXx~`%+3VDSzh#r3XyBTXEHf?0(EoO(->ZVb_TY)&Sbla1z@UmyXO9$P&xx?t? zLT=-*(zo|4OG!iE>tD=P{hZSjbnV#h@d(%DWSIen`T8n9 zHWSy@hAEK8sS-tQ^CHfwg~=_O0|YK!rLnYR(#YklkQ`gNY9l|)i35vnfsW~+Z+v`B z>u|TH6!(q4ZdR?2dG(_1B_y00yz&aq|Mhh(tl7pL+Wh{Bi3(BCPFglTpxcII8+Y`* zxwF!WD@AzykKb>Pf0n**F7{`|0SaYsmprI_lBM2!_#o=q8gVB(dlTNe4;KJvHLI!w z#1);nVw)6KRz?L9FchugL&bQQ=1$}0d-GFEqz2H)(zNHUx%R7~mp~k+AWF+@8Eh}ZkUn%*vONU!5og~Nl9XhMG5Jm2B zP_&%_9l0epskexhM%xD+75^r?RB!K^%U)g-&tKS$8=mgmKGa=*W}b^Hmh({k$`W%e z(29@wR+o=GlxHX}&sEO;;FIgbj9cGMd6>E1xN^p!2vsOqE+Ik3czFLn*PT64#@_aa z@moa}*mDKO5;OF!r0SrJN-_wcd2Y{#L+bMhbo zZwqh6Kt$=nVDrLI`Z(B6l9?hqckkG-(Pt?4VYgRTU_gNP#Bi=L46JG{0VkXrSr4tb zs;Q}CLeuYGzm&2+9RBoVjBdmC@XpQ)>!^>Ddh5Q9F>8XL)6`HoqTf}xsdwxL+mR#D z;F2GJpnC(TtTk9)FgLh~%7FDw;HGl{QDtR&&WHvYRMWGdmA0PQ-Nz_{XzIE6YcEPJ zw8*!mshXfS+@W0y-49<^Vc^g7q;mBmM-Fj`E%3<7K3Vb}-=lH7ECdAsZa9CaRuiO? zj0)}2-zyJhDhvVq5QOKA>*hnPPI>3w)4REs1$eD26FlWS6g|s}FPb=gkH)%m&1*CGs?87xQ+5xG`UdGRa%vwJho|>MERiCACvb?C$E>b}yc0w!2(f4H1bgPfyvw zLI%7BTfztM3PK97En3^e`lT{>oCzS6PIRx86F>=Kb9dP?1}2;57*K0t}2rZVaed&oZAm<9e*!8)ZFOtR8^7n)`PvUZ2~l z{cxF`iiVNq(!9O%RsMSvJmnM6D$)=T%fFiO>@F`Se9%v5idl^`ge}?2f2f~x^kgKO zyTRJ3zSS7s9OGgpg^s-DACeVrR2&m-YSp~${LSZ`7BPJR(L2Y71{o)$q1y1s#eDB zecdl@U!?H{ULNq=rm)tco)j9hYrjV14NscTyLavL?K)V8iu0{@gt||cv!k-K9Q)=} z*B~fy*Yv)^_439kh3g)-0QPKAV4}abaK=Xse1Ux)pBG|gA{nn4N zC5_%WMee@7>COLkwu;g=WNN>um>75&f7WRvc-q7Cu%Naqf;p2!*|!fJixoNLBK zmu@wNeXjDqNm6g%#FG81KHUKL+-r2obE9%X*%zntvPg(7 zjE{Z7=~T`uk}a|ITvnrVkt<0FH~e|vO&#vlVTj+3o z;nL4IvqG&%c6KLvoo2`FU7e+}DJmNyBMn0VUFbznv6PpuLRn!eW4USVyB*hazwZ|1VdFYqBzYu7Xn_uA6(w5sLm^7!$w zOOpfleYgEEL=tZC{QEnJF0$EQC}0Mq*RNaIr%QKK}$&Gs>D^_;SS$#CY9B~3^;%o zX;YwGZY5fm?j$BsXB#l=NH0OxO**KGUL~Gtr`O=YK=MP4zqLJ=lZ)%;+^F#-m#1*f zWG*vZJG_J;I_%S>iw^GJ&y5!G*KY2O8^dvwd`@M_;L_XVjH3)tuGpWsi7TOyVN{LG zu3c-V66|~T?moa{YS1{omBhGruhkrZNwAB{^WJ{_SG6`L(MgRR$i-#q`00A64cL;7 zIKOhFkdg7pvdeVJQwlrKiuoECFYAG zq5K+8|WDD(m?c&i6aQ)iz`ovLK&gf_3QQ0S8nx5fKFz z*4DF!Z>;#6d+C?+of2u!JlS?f-%N>a9Ssv~@Or)V_Iyw(ySj^=Y~mKDE8=M6){H;j zRk88uOX8LI%9oDNMvUY9bp+l?snzCjcayoEq*{a1A6=(RXL}tjlEM2Ynlkv&~ zr;8^!Pn=!*N@K;clV?r0tbNt@)3KlFbMuN4Q&r#)%%WAyD|GVm9D;-rvbiH7LrYCU z=8B3)OuBP+?JDJ!HfQ)(>44|4kNgP$-kcfA9yrs;xN*-OppC1HrDxCj?EN7QXkP?W a%@6y(QX=QRORYS{00f?{elF{r5}E+;2_Z@V literal 0 HcmV?d00001 diff --git a/doc/query-engine.md b/doc/query-engine.md index b12b5f0ab4..d7464109f2 100644 --- a/doc/query-engine.md +++ b/doc/query-engine.md @@ -12,6 +12,7 @@ - [Example](#example) - [Execution Detail](#execution-detail) - [Periodic Samples and Range Functions](#periodic-samples-and-range-functions) + - [ChunkedWindowIterator Details](#chunkedwindowiterator-details) - [Alternate Design Approaches to Consider](#alternate-design-approaches-to-consider) - [Aggregation across Range Vectors](#aggregation-across-range-vectors) @@ -84,7 +85,7 @@ the result data structure. The query result `QueryResult` is like a mini dataset time series, each represented by `RangeVector`. All the `RangeVector`s share the same schema which is included in the result object. Each `RangeVector` has an associated key called the `RangeVectorKey` and could be seen as the time series key for the result. The `RangeVector` exposes an iterator of rows. Each row within -the `RangeVector` starts wih the timestamp, followed by the data value, usually a double. +the `RangeVector` starts wih the timestamp, followed by the data value(s), usually a double. ## Execution Plan Constructs @@ -245,6 +246,19 @@ There are two categories of `RangeFunction`s. The chunked range functions have Look here for examples of [RangeFunctions](../query/exec/rangefn/RangeFunction.scala) +### ChunkedWindowIterator Details + +The `ChunkedWindowIterator` applies `ChunkedRangeFunctions` to each time window, outputting a computed value for each window. It works as follows: + +![](mermaid/chunked-iteration.png) + +1. `ChunkedWindowIterator` advances to the next window +2. it first calls `reset` on the `ChunkedRangeFunction` +2. It then loops through each relevant chunk within the window, calling the `addChunk` method of the range function to update its result for each chunk. The range function is free to use any method from the raw chunk data, usually this is pretty fast compared to row-wise iteration. +3. Finally, the `apply` method is called on the range function to present result for that time window from all the intermediate results from the chunks. + +Note that `ChunkedWindowIterator` support different types of `RowReader` outputs for different types of output columns. One version is for `TransientRow` which has Double values, and another is for `TransientHistRow` which supports Histogram values as output. + #### Alternate Design Approaches to Consider 1. With sliding window periodic sample generation can happen as a O(n) time, and O(w) memory where n is number of raw samples, and w is samples per window. The alternate approach to compare performance with later is to diff --git a/doc/sharding.md b/doc/sharding.md index dc35c68d8e..8751580d66 100644 --- a/doc/sharding.md +++ b/doc/sharding.md @@ -11,6 +11,7 @@ - [Shard Event Subscriptions](#shard-event-subscriptions) - [Subscribe to Shard Status Events](#subscribe-to-shard-status-events) - [Shard Status and Shard Status Events](#shard-status-and-shard-status-events) + - [Automatic Reassignment of Shards](#automatic-reassignment-of-shards) - [Unsubscribe to Shard Status Events](#unsubscribe-to-shard-status-events) - [Auto Unsubscribe](#auto-unsubscribe) - [Cluster/Shard State Recovery](#clustershard-state-recovery) diff --git a/doc/spark-oldold.md b/doc/spark-oldold.md index 904cf6f380..4ff038b280 100644 --- a/doc/spark-oldold.md +++ b/doc/spark-oldold.md @@ -1,3 +1,16 @@ + + +**Table of Contents** *generated with [DocToc](https://github.com/thlorenz/doctoc)* + +- [FiloDB and Spark](#filodb-and-spark) + - [Using FiloDB Data Source with Spark](#using-filodb-data-source-with-spark) + - [Configuring FiloDB](#configuring-filodb) + - [Passing Cassandra Authentication Settings](#passing-cassandra-authentication-settings) + - [Spark Data Source API Example (spark-shell)](#spark-data-source-api-example-spark-shell) + - [Querying Datasets](#querying-datasets) + + + # FiloDB and Spark NOTE: These docs are DEPRECATED. Currently the Spark module is not working. If you are interested in helping to revive this module and these docs, please raise an issue and/or use the mailing lists. Thank you. diff --git a/jmh/src/main/scala/filodb.jmh/HistogramIngestBenchmark.scala b/jmh/src/main/scala/filodb.jmh/HistogramIngestBenchmark.scala new file mode 100644 index 0000000000..df297b0a82 --- /dev/null +++ b/jmh/src/main/scala/filodb.jmh/HistogramIngestBenchmark.scala @@ -0,0 +1,120 @@ +package filodb.jmh + +import java.util.concurrent.TimeUnit + +import ch.qos.logback.classic.{Level, Logger} +import com.typesafe.config.ConfigFactory +import org.openjdk.jmh.annotations.{Level => JMHLevel, _} + +import filodb.core.{MachineMetricsData, MetricsTestData, TestData} +import filodb.core.binaryrecord2.RecordBuilder +import filodb.core.memstore._ +import filodb.core.store._ +import filodb.memory.MemFactory +import filodb.memory.format.SeqRowReader + +//scalastyle:off regex +/** + * Benchmark measuring ingestion/encoding performance of various HistogramColumn schemes + * (plus versus traditional Prom schema). + * All benchmarks measure histograms per second ingested. + * Encoding/compression is included - multiple chunks are built and compressed. + * All samples include tags with five tags. + */ +@State(Scope.Thread) +class HistogramIngestBenchmark { + import MachineMetricsData._ + + org.slf4j.LoggerFactory.getLogger("filodb").asInstanceOf[Logger].setLevel(Level.WARN) + + // HistogramColumn schema, test data, builder + println("Be patient, generating lots of containers of histogram schema data....") + val histSchemaData = linearHistSeries(numBuckets = 64).map(SeqRowReader) + // sized just big enough for ~300 entries per container 700 * 300 + val histSchemaBuilder = new RecordBuilder(MemFactory.onHeapFactory, histDataset.ingestionSchema, 230000) + histSchemaData.take(300*100).grouped(300).foreach { rows => + rows.foreach(histSchemaBuilder.addFromReader) + println(s"We have ${histSchemaBuilder.allContainers.length} containers, " + + s"remaining = ${histSchemaBuilder.containerRemaining}") + histSchemaBuilder.newContainer() // Force switching to new container + } + val histContainers = histSchemaBuilder.allContainers.toArray + + // Prometheus schema, test data, builder + println("Be patient, generating lots of containers of prometheus schema data....") + val promDataset = MetricsTestData.timeseriesDataset + val promData = MetricsTestData.promHistSeries(numBuckets = 64).map(SeqRowReader) + val promBuilder = new RecordBuilder(MemFactory.onHeapFactory, promDataset.ingestionSchema, 4200000) + promData.take(300*66*100).grouped(300*66).foreach { rows => + rows.foreach(promBuilder.addFromReader) + println(s"We have ${promBuilder.allContainers.length} containers, " + + s"remaining = ${promBuilder.containerRemaining}") + promBuilder.newContainer() // Force switching to new container + } + val promContainers = promBuilder.allContainers.toArray + + println(s"DONE generating.\nHistSchema container size=${histContainers.head.numBytes} " + + s"# records=${histContainers.head.countRecords}") + println(s"Prom schema container size=${promContainers.head.numBytes} " + + s"# records=${promContainers.head.countRecords}") + + import monix.execution.Scheduler.Implicits.global + + val config = ConfigFactory.load("application_test.conf").getConfig("filodb") + val policy = new FixedMaxPartitionsEvictionPolicy(1000) + val memStore = new TimeSeriesMemStore(config, new NullColumnStore, new InMemoryMetaStore(), Some(policy)) + val ingestConf = TestData.storeConf.copy(shardMemSize = 512 * 1024 * 1024, maxChunksSize = 100) + memStore.setup(histDataset, 0, ingestConf) + memStore.setup(promDataset, 0, ingestConf) + + val hShard = memStore.getShardE(histDataset.ref, 0) + val pShard = memStore.getShardE(promDataset.ref, 0) + + var containerNo = 0 + + // Setup per iteration to clean shard state and make sure ingestion is repeatable. + // NOTE: need to use per-iteration, not invocation, or else the setup costs affect the benchmark results + @Setup(JMHLevel.Iteration) + def cleanIngest(): Unit = { + println(s"hShard #partitions=${hShard.numActivePartitions} pShard #partitions=${pShard.numActivePartitions}") + hShard.reset() + pShard.reset() + containerNo = 0 + } + + /** + * Ingest 300 histograms every invocation; 30 per partition. Every roughly 3 invocations there is encoding. + * Each iteration = 100 invocations, thus there is plenty of encoding cycles. + * Note that adding partitions is only done at the start of each iteration, not invocation, since the setup + * to clean the shard state is only done at the beginning of each iteration. + * + * Time reported is time to ingest 30k histograms. To get throughput, divide 30k by the time in seconds + */ + @Benchmark + @BenchmarkMode(Array(Mode.SingleShotTime)) + @OutputTimeUnit(TimeUnit.SECONDS) + @Warmup(batchSize=50) + @Measurement(batchSize=100) + def ingestHistColumn1(): Unit = { + hShard.ingest(histContainers(containerNo), 0) + containerNo += 1 + } + + /** + * Ingest 300 histograms every invocation; 30 per partition. Every roughly 3 invocations there is encoding. + * Each iteration = 100 invocations, thus there is plenty of encoding cycles. + * Note that adding partitions is only done at the start of each iteration, not invocation, since the setup + * to clean the shard state is only done at the beginning of each iteration. + * + * Time reported is time to ingest 30k histograms. To get throughput, divide 30k by the time in seconds + */ + @Benchmark + @BenchmarkMode(Array(Mode.SingleShotTime)) + @OutputTimeUnit(TimeUnit.SECONDS) + @Warmup(batchSize=50) + @Measurement(batchSize=100) + def ingestPromHistograms(): Unit = { + pShard.ingest(promContainers(containerNo), 0) + containerNo += 1 + } +} \ No newline at end of file diff --git a/jmh/src/main/scala/filodb.jmh/HistogramQueryBenchmark.scala b/jmh/src/main/scala/filodb.jmh/HistogramQueryBenchmark.scala new file mode 100644 index 0000000000..72a252b20e --- /dev/null +++ b/jmh/src/main/scala/filodb.jmh/HistogramQueryBenchmark.scala @@ -0,0 +1,126 @@ +package filodb.jmh + +import java.util.concurrent.TimeUnit + +import scala.concurrent.Await +import scala.concurrent.duration._ + +import akka.actor.ActorSystem +import ch.qos.logback.classic.{Level, Logger} +import com.typesafe.config.ConfigFactory +import monix.execution.Scheduler +import monix.reactive.Observable +import org.openjdk.jmh.annotations._ + +import filodb.coordinator.{FilodbCluster, IngestionStarted, ShardMapper} +import filodb.coordinator.client.QueryCommands._ +import filodb.coordinator.queryengine2.QueryEngine +import filodb.core.{MachineMetricsData, MetricsTestData, TestData} +import filodb.core.binaryrecord2.RecordBuilder +import filodb.core.memstore._ +import filodb.core.store._ +import filodb.memory.format.SeqRowReader +import filodb.memory.MemFactory +import filodb.prometheus.parse.Parser +import filodb.query.QueryConfig + +//scalastyle:off regex +/** + * Benchmark measuring query performance of various HistogramColumn schemes + * (plus versus traditional Prom schema). + * All samples include tags with five tags. + * Write buffer queries only - since most queries will be to write buffers only. + */ +@State(Scope.Thread) +class HistogramQueryBenchmark { + import MachineMetricsData._ + + org.slf4j.LoggerFactory.getLogger("filodb").asInstanceOf[Logger].setLevel(Level.WARN) + + import monix.execution.Scheduler.Implicits.global + + val config = ConfigFactory.load("application_test.conf").getConfig("filodb") + val policy = new FixedMaxPartitionsEvictionPolicy(1000) + val memStore = new TimeSeriesMemStore(config, new NullColumnStore, new InMemoryMetaStore(), Some(policy)) + val ingestConf = TestData.storeConf.copy(shardMemSize = 512 * 1024 * 1024, maxChunksSize = 200) + + // HistogramColumn data: 10 series, 180 samples per series = 1800 total + println("Ingesting containers of histogram schema data....") + val histSchemaData = linearHistSeries(numBuckets = 64).map(SeqRowReader) + val histSchemaBuilder = new RecordBuilder(MemFactory.onHeapFactory, histDataset.ingestionSchema, 230000) + histSchemaData.take(10 * 180).foreach(histSchemaBuilder.addFromReader) + + memStore.setup(histDataset, 0, ingestConf) + val hShard = memStore.getShardE(histDataset.ref, 0) + histSchemaBuilder.allContainers.foreach { c => hShard.ingest(c, 0) } + memStore.commitIndexForTesting(histDataset.ref) // commit lucene index + + // Prometheus hist data: 10 series * 66 = 660 series * 180 samples + println("Ingesting containers of prometheus schema data....") + val promDataset = MetricsTestData.timeseriesDataset + val promData = MetricsTestData.promHistSeries(numBuckets = 64).map(SeqRowReader) + val promBuilder = new RecordBuilder(MemFactory.onHeapFactory, promDataset.ingestionSchema, 4200000) + promData.take(10*66*180).foreach(promBuilder.addFromReader) + + memStore.setup(promDataset, 0, ingestConf) + val pShard = memStore.getShardE(promDataset.ref, 0) + promBuilder.allContainers.foreach { c => pShard.ingest(c, 0) } + memStore.commitIndexForTesting(promDataset.ref) // commit lucene index + + val system = ActorSystem("test", ConfigFactory.load("filodb-defaults.conf")) + private val cluster = FilodbCluster(system) + cluster.join() + + private val coordinator = cluster.coordinatorActor + private val shardMapper = new ShardMapper(1) + shardMapper.updateFromEvent(IngestionStarted(histDataset.ref, 0, coordinator)) + + // Query configuration + val hEngine = new QueryEngine(histDataset, shardMapper) + val pEngine = new QueryEngine(promDataset, shardMapper) + val startTime = 100000L + 100*1000 // 100 samples in. Look back 30 samples, which normally would be 5min + + val histQuery = """histogram_quantile(0.9, sum_over_time(http_requests_total{job="prometheus",__col__="h"}[30s]))""" + val promQuery = """histogram_quantile(0.9, sum_over_time(http_requests_total_bucket{job="prometheus"}[30s]))""" + + // Single-threaded query test + val numQueries = 500 + val qOptions = QueryOptions(1, 100).copy(shardOverrides = Some(Seq(0))) + val hLogicalPlan = Parser.queryToLogicalPlan(histQuery, startTime/1000) + val hExecPlan = hEngine.materialize(hLogicalPlan, qOptions) + val querySched = Scheduler.singleThread(s"benchmark-query") + val queryConfig = new QueryConfig(config.getConfig("query")) + + val pLogicalPlan = Parser.queryToLogicalPlan(promQuery, startTime/1000) + val pExecPlan = pEngine.materialize(pLogicalPlan, qOptions) + + @TearDown + def shutdownFiloActors(): Unit = { + cluster.shutdown() + } + + // These are boh single threaded queries + @Benchmark + @BenchmarkMode(Array(Mode.Throughput)) + @OutputTimeUnit(TimeUnit.SECONDS) + @OperationsPerInvocation(500) + def histSchemaQuantileQuery(): Long = { + val f = Observable.fromIterable(0 until numQueries).mapAsync(1) { n => + hExecPlan.execute(memStore, histDataset, queryConfig)(querySched, 60.seconds) + }.executeOn(querySched) + .countL.runAsync + Await.result(f, 60.seconds) + } + + @Benchmark + @BenchmarkMode(Array(Mode.Throughput)) + @OutputTimeUnit(TimeUnit.SECONDS) + @OperationsPerInvocation(500) + def promSchemaQuantileQuery(): Long = { + val f = Observable.fromIterable(0 until numQueries).mapAsync(1) { n => + pExecPlan.execute(memStore, promDataset, queryConfig)(querySched, 60.seconds) + }.executeOn(querySched) + .countL.runAsync + Await.result(f, 60.seconds) + } +} \ No newline at end of file diff --git a/jmh/src/main/scala/filodb.jmh/SparkReadBenchmark.scala b/jmh/src/main/scala/filodb.jmh/SparkReadBenchmark.scala deleted file mode 100644 index 5b6ce1ef69..0000000000 --- a/jmh/src/main/scala/filodb.jmh/SparkReadBenchmark.scala +++ /dev/null @@ -1,120 +0,0 @@ -package filodb.jmh - -import java.util.concurrent.TimeUnit - -import ch.qos.logback.classic.{Level, Logger} -import com.typesafe.config.ConfigRenderOptions -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.functions.sum -import org.openjdk.jmh.annotations._ - -import filodb.core.binaryrecord2.RecordBuilder -import filodb.core.memstore.SomeData -import filodb.core.store._ -import filodb.core.TestData -import filodb.memory.format.TupleRowReader -import filodb.memory.MemFactory -import filodb.spark.{FiloDriver, FiloExecutor, FiloRelation} - -/** - * A benchmark to compare performance of FiloRelation against different scenarios, - * for an analytical query summing 5 million random integers from a single column of a - * FiloDB dataset. Description: - * - sparkSum(): Sum 5 million integers stored using MemStore. - * - sparkBaseline(): Get the first 2 records. Just to see what the baseline latency is of a - * DataFrame query. - * - memStoreOnly(): No Spark, just reading rows from the MemStore - * - sparkCassSum(): Sum 5 million integers using CassandraColumnStore. Must have run CreateCassTestData - * first to populate into Cassandra. - * NOTE: This has been moved to SparkCassBenchmark.scala - * - * To get the scan speed, one needs to subtract the baseline from the total time of sparkSum/sparkCassSum. - * For example, on my laptop, here is the JMH output: - * {{{ - * Benchmark Mode Cnt Score Error Units - * SparkReadBenchmark.memStoreOnly ss 15 ≈ 10⁻³ s/op - * SparkReadBenchmark.sparkBaseline ss 15 0.013 ± 0.001 s/op - * SparkReadBenchmark.sparkSum ss 15 0.045 ± 0.002 s/op - * SparkCassBenchmark.sparkCassSum ss 15 0.226 ± 0.035 s/op - * }}} - * - * (The above run against Cassandra 2.1.6, 5GB heap, with jmh:run -i 3 -wi 3 -f3 filodb.jmh.SparkReadBenchmark) - * - * Thus: - * - Cassandra scan speed = 5000000 / (0.226 - 0.013) = 23.47 million ops/sec - * - InMemory scan speed = 5000000 / (0.045 - 0.013) = 156.25 million ops/sec - */ -@State(Scope.Benchmark) -class SparkReadBenchmark { - org.slf4j.LoggerFactory.getLogger("filodb").asInstanceOf[Logger].setLevel(Level.ERROR) - - val NumRows = 5000000 - // Source of rows - val sess = SparkSession.builder.master("local[4]") - .appName("test") - .config("spark.ui.enabled", "false") - .config("spark.filodb.memstore.chunks-to-keep", "20") - .getOrCreate - val sc = sess.sparkContext - // Below is to make sure that Filo actor system stuff is run before test code - // so test code is not hit with unnecessary slowdown - val filoConfig = FiloDriver.initAndGetConfig(sc) - - import IntSumReadBenchmark._ - - // Initialize metastore - import filodb.coordinator.client.Client._ - parse(FiloDriver.metaStore.newDataset(dataset)) { x => x } - FiloDriver.memStore.setup(dataset, 0, TestData.storeConf) - val split = FiloDriver.memStore.getScanSplits(dataset.ref).head - - // Write raw data into MemStore - val builder = new RecordBuilder(MemFactory.onHeapFactory, dataset.ingestionSchema) - rowIt.take(NumRows).map(TupleRowReader).foreach { row => builder.addFromReader(row) } - val data = builder.allContainers.zipWithIndex.map { case (container, i) => SomeData(container, i) }.head - FiloDriver.memStore.ingest(dataset.ref, 0, data) - - @TearDown - def shutdownFiloActors(): Unit = { - FiloDriver.shutdown() - FiloExecutor.shutdown() - sc.stop() - } - - val df = sess.read.format("filodb.spark").option("dataset", dataset.name).load - - // How long does it take to iterate through all the rows - @Benchmark - @BenchmarkMode(Array(Mode.SingleShotTime)) - @OutputTimeUnit(TimeUnit.SECONDS) - def sparkSum(): Any = { - df.agg(sum(df("int"))).collect().head - } - - val configStr = filoConfig.root.render(ConfigRenderOptions.concise) - - // Measure the speed of MemStore's ScanChunks etc. over many chunks - // Including null check - @Benchmark - @BenchmarkMode(Array(Mode.SingleShotTime)) - @OutputTimeUnit(TimeUnit.SECONDS) - def memStoreOnly(): Any = { - val it = FiloRelation.perPartitionRowScanner(configStr, dataset.asCompactString, dataset.colIDs("int").get, - FilteredPartitionScan(split), AllChunkScan).asInstanceOf[Iterator[InternalRow]] - var sum = 0 - while (it.hasNext) { - val row = it.next - if (!row.isNullAt(0)) sum += row.getInt(0) - } - sum - } - - // Baseline comparison ... see what the minimal time for a Spark task is. - @Benchmark - @BenchmarkMode(Array(Mode.SingleShotTime)) - @OutputTimeUnit(TimeUnit.SECONDS) - def sparkBaseline(): Any = { - df.select("int").limit(2).collect() - } -} \ No newline at end of file diff --git a/memory/src/main/scala/filodb.memory/BinaryRegion.scala b/memory/src/main/scala/filodb.memory/BinaryRegion.scala index dc0f039c87..ee70c74c33 100644 --- a/memory/src/main/scala/filodb.memory/BinaryRegion.scala +++ b/memory/src/main/scala/filodb.memory/BinaryRegion.scala @@ -33,7 +33,7 @@ object BinaryRegion { // TODO: Can we PLEASE implement our own Unsafe XXHash which does not require creating a DirectBuffer? def hash32(base: Any, offset: Long, len: Int): Int = base match { - case a: Array[Byte] => hash32(a) + case a: Array[Byte] => hasher32.hash(a, offset.toInt - UnsafeUtils.arayOffset, len, Seed) case UnsafeUtils.ZeroPointer => hasher32.hash(UnsafeUtils.asDirectBuffer(offset, len), Seed) } diff --git a/memory/src/main/scala/filodb.memory/Block.scala b/memory/src/main/scala/filodb.memory/Block.scala index cfe085abd0..ca59337e5b 100644 --- a/memory/src/main/scala/filodb.memory/Block.scala +++ b/memory/src/main/scala/filodb.memory/Block.scala @@ -184,6 +184,9 @@ class Block(val address: Long, val capacity: Long, val reclaimListener: ReclaimL stringBuf.toString } + // debug method to set memory to specific value for testing + private[memory] def set(value: Byte): Unit = + UnsafeUtils.unsafe.setMemory(address, capacity, value) } diff --git a/memory/src/main/scala/filodb.memory/format/BinaryVector.scala b/memory/src/main/scala/filodb.memory/format/BinaryVector.scala index a183972ba5..bde32c90be 100644 --- a/memory/src/main/scala/filodb.memory/format/BinaryVector.scala +++ b/memory/src/main/scala/filodb.memory/format/BinaryVector.scala @@ -54,6 +54,7 @@ object BinaryVector { case Classes.Long => (p => vectors.LongBinaryVector(p)) case Classes.Double => (p => vectors.DoubleVector(p)) case Classes.UTF8 => (p => vectors.UTF8Vector(p)) + case Classes.Histogram => (p => vectors.HistogramVector(p)) } /** @@ -64,6 +65,7 @@ object BinaryVector { //scalastyle:off import BinaryVector.BinaryVectorPtr +import vectors.HistogramVector.HistIterator //scalastyle:on trait TypedIterator { @@ -71,6 +73,7 @@ trait TypedIterator { final def asLongIt: vectors.LongIterator = this.asInstanceOf[vectors.LongIterator] final def asDoubleIt: vectors.DoubleIterator = this.asInstanceOf[vectors.DoubleIterator] final def asUTF8It: vectors.UTF8Iterator = this.asInstanceOf[vectors.UTF8Iterator] + final def asHistIt: HistIterator = this.asInstanceOf[HistIterator] } trait BooleanIterator extends TypedIterator { @@ -121,6 +124,7 @@ trait VectorDataReader extends AvailableReader { def asLongReader: vectors.LongVectorDataReader = this.asInstanceOf[vectors.LongVectorDataReader] def asDoubleReader: vectors.DoubleVectorDataReader = this.asInstanceOf[vectors.DoubleVectorDataReader] def asUTF8Reader: vectors.UTF8VectorDataReader = this.asInstanceOf[vectors.UTF8VectorDataReader] + def asHistReader: vectors.HistogramReader = this.asInstanceOf[vectors.HistogramReader] } // An efficient iterator for the bitmap mask, rotating a mask as we go @@ -166,6 +170,7 @@ sealed trait AddResponse case object Ack extends AddResponse final case class VectorTooSmall(bytesNeeded: Int, bytesHave: Int) extends AddResponse case object ItemTooLarge extends AddResponse +case object BucketSchemaMismatch extends AddResponse /** * A BinaryVector that you can append to. Has some notion of a maximum size (max # of items or bytes) diff --git a/memory/src/main/scala/filodb.memory/format/RowReader.scala b/memory/src/main/scala/filodb.memory/format/RowReader.scala index d98bbca640..06d7be8bcf 100644 --- a/memory/src/main/scala/filodb.memory/format/RowReader.scala +++ b/memory/src/main/scala/filodb.memory/format/RowReader.scala @@ -5,9 +5,12 @@ import java.sql.Timestamp import scala.reflect.ClassTag +import org.agrona.concurrent.UnsafeBuffer import org.joda.time.DateTime import scalaxy.loops._ +import filodb.memory.format.vectors.Histogram + /** * A generic trait for reading typed values out of a row of data. * Used for both reading out of Filo vectors as well as for RowToVectorBuilder, @@ -28,6 +31,20 @@ trait RowReader { def getBlobOffset(columnNo: Int): Long def getBlobNumBytes(columnNo: Int): Int // Total number of bytes for the blob + // By default this is not implemented as histograms can be parsed from multiple serialized forms or actual objects + def getHistogram(columnNo: Int): Histogram = ??? + + /** + * Retrieves a view into the blob at column columnNo without duplicating contents. + * Smart implementations could reuse the same UnsafeBuffer to avoid allocations. + * This default implementation simply allocates a new one. + */ + def blobAsBuffer(columnNo: Int): UnsafeBuffer = { + val buf = new UnsafeBuffer(Array.empty[Byte]) + UnsafeUtils.wrapUnsafeBuf(getBlobBase(columnNo), getBlobOffset(columnNo), getBlobNumBytes(columnNo), buf) + buf + } + final def getBuffer(columnNo: Int): ByteBuffer = { val length = getBlobNumBytes(columnNo) getBlobBase(columnNo) match { @@ -212,6 +229,7 @@ final case class SingleValueRowReader(value: Any) extends RowReader { def getDouble(columnNo: Int): Double = value.asInstanceOf[Double] def getFloat(columnNo: Int): Float = value.asInstanceOf[Float] def getString(columnNo: Int): String = value.asInstanceOf[String] + override def getHistogram(columnNo: Int): Histogram = value.asInstanceOf[Histogram] def getAny(columnNo: Int): Any = value def getBlobBase(columnNo: Int): Any = value def getBlobOffset(columnNo: Int): Long = 0 @@ -226,6 +244,7 @@ final case class SeqRowReader(sequence: Seq[Any]) extends RowReader { def getDouble(columnNo: Int): Double = sequence(columnNo).asInstanceOf[Double] def getFloat(columnNo: Int): Float = sequence(columnNo).asInstanceOf[Float] def getString(columnNo: Int): String = sequence(columnNo).asInstanceOf[String] + override def getHistogram(columnNo: Int): Histogram = sequence(columnNo).asInstanceOf[Histogram] def getAny(columnNo: Int): Any = sequence(columnNo) def getBlobBase(columnNo: Int): Any = ??? def getBlobOffset(columnNo: Int): Long = ??? @@ -267,6 +286,7 @@ final case class SchemaSeqRowReader(sequence: Seq[Any], def getDouble(columnNo: Int): Double = sequence(columnNo).asInstanceOf[Double] def getFloat(columnNo: Int): Float = sequence(columnNo).asInstanceOf[Float] def getString(columnNo: Int): String = sequence(columnNo).asInstanceOf[String] + override def getHistogram(columnNo: Int): Histogram = sequence(columnNo).asInstanceOf[Histogram] def getAny(columnNo: Int): Any = sequence(columnNo) def getBlobBase(columnNo: Int): Any = sequence(columnNo).asInstanceOf[Array[Byte]] def getBlobOffset(columnNo: Int): Long = 0 @@ -367,4 +387,10 @@ object RowReader { final def compare(reader: RowReader, other: RowReader, columnNo: Int): Int = getFieldOrDefault(reader, columnNo).compareTo(getFieldOrDefault(other, columnNo)) } + + implicit object HistogramExtractor extends TypedFieldExtractor[Histogram] { + final def getField(reader: RowReader, columnNo: Int): Histogram = reader.getHistogram(columnNo) + final def compare(reader: RowReader, other: RowReader, columnNo: Int): Int = + getFieldOrDefault(reader, columnNo).compare(getFieldOrDefault(other, columnNo)) + } } diff --git a/memory/src/main/scala/filodb.memory/format/RowToVectorBuilder.scala b/memory/src/main/scala/filodb.memory/format/RowToVectorBuilder.scala index 880bdffc4b..6637eeb9be 100644 --- a/memory/src/main/scala/filodb.memory/format/RowToVectorBuilder.scala +++ b/memory/src/main/scala/filodb.memory/format/RowToVectorBuilder.scala @@ -24,6 +24,7 @@ object Classes { val Double = java.lang.Double.TYPE val String = classOf[String] val UTF8 = classOf[ZeroCopyUTF8String] + val Histogram = classOf[vectors.Histogram] } object RowToVectorBuilder { val MaxElements = 1000 diff --git a/memory/src/main/scala/filodb.memory/format/UnsafeUtils.scala b/memory/src/main/scala/filodb.memory/format/UnsafeUtils.scala index 84131a870d..3cc6d58c44 100644 --- a/memory/src/main/scala/filodb.memory/format/UnsafeUtils.scala +++ b/memory/src/main/scala/filodb.memory/format/UnsafeUtils.scala @@ -3,6 +3,7 @@ package filodb.memory.format import java.nio.ByteBuffer import com.kenai.jffi.MemoryIO +import org.agrona.concurrent.UnsafeBuffer import scalaxy.loops._ // scalastyle:off number.of.methods @@ -11,6 +12,7 @@ object UnsafeUtils { // scalastyle:off val ZeroPointer: Any = null + val ZeroArray = null.asInstanceOf[Array[Byte]] // scalastyle:on val arayOffset = unsafe.arrayBaseOffset(classOf[Array[Byte]]) @@ -45,6 +47,14 @@ object UnsafeUtils { MemoryIO.getCheckedInstance.newDirectByteBuffer(address, size) } + def wrapUnsafeBuf(base: Any, offset: Long, numBytes: Int, buf: UnsafeBuffer): Unit = { + if (base != UnsafeUtils.ZeroPointer) { + buf.wrap(base.asInstanceOf[Array[Byte]], offset.toInt - arayOffset, numBytes) + } else { + buf.wrap(offset, numBytes) + } + } + /** * Generic methods to read and write data to any offset from a base object location. Be careful, this * can easily crash the system! diff --git a/memory/src/main/scala/filodb.memory/format/vectors/Histogram.scala b/memory/src/main/scala/filodb.memory/format/vectors/Histogram.scala new file mode 100644 index 0000000000..c6f7df9535 --- /dev/null +++ b/memory/src/main/scala/filodb.memory/format/vectors/Histogram.scala @@ -0,0 +1,273 @@ +package filodb.memory.format.vectors + +import org.agrona.concurrent.UnsafeBuffer +import scalaxy.loops._ + +import filodb.memory.BinaryRegion +import filodb.memory.format._ + +/** + * A trait to represent bucket-based histograms as well as derived statistics such as sums or rates of + * increasing histograms. + * The schema is based on Prometheus histograms. Each bucket is designed to contain all observations less than + * or equal to the bucketTop ("upper bound"), thus it is cumulative with increasing bucket number. + * Furthermore the LAST bucket should contain the total number of observations overall. + */ +trait Histogram extends Ordered[Histogram] { + def numBuckets: Int + + /** + * Gets the bucket definition for number no. Observations for values <= this bucket, so it represents + * an upper limit. + */ + def bucketTop(no: Int): Double + + /** + * Gets the counter or number of observations for a particular bucket, or derived value of a bucket. + */ + def bucketValue(no: Int): Double + + /** + * Returns an UnsafeBuffer pointing to a serialized BinaryHistogram representation of this histogram. + * @param buf if Some(buf) supplied, then that buf is either written into or re-used to wrap where the serialized + * representation is. The supplied buffer must be large enough to hold serialized histogram. + * if None is passed, then the thread-local buffer may be re-used, in which case be careful as that + * buffer will be mutated with the next call to serialize() within the same thread. + */ + def serialize(intoBuf: Option[UnsafeBuffer] = None): UnsafeBuffer + + /** + * Finds the first bucket number with a value greater than or equal to the given "rank" + * @return the bucket number, or numBuckets if not found + */ + final def firstBucketGTE(rank: Double): Int = { + var bucketNo = 0 + while (bucketValue(bucketNo) < rank) bucketNo += 1 + bucketNo + } + + final def topBucketValue: Double = bucketValue(numBuckets - 1) + + /** + * Calculates histogram quantile based on bucket values using Prometheus scheme (increasing/LE) + * TODO: monotonicity check, which will be in a separate function. + */ + final def quantile(q: Double): Double = { + val result = if (q < 0) Double.NegativeInfinity + else if (q > 1) Double.PositiveInfinity + else if (numBuckets < 2) Double.NaN + else { + // find rank for the quantile using total number of occurrences (which is the last bucket value) + var rank = q * topBucketValue + // using rank, find the le bucket which would have the identified rank + val b = firstBucketGTE(rank) + + // now calculate quantile + // If the rank is at the top return the last bucket (though shouldn't we interpolate here too?) + if (b == numBuckets-1) return bucketTop(numBuckets-2) + else if (b == 0 && bucketTop(0) <= 0) return bucketTop(0) + else { + // interpolate quantile within le bucket + var (bucketStart, bucketEnd, count) = (0d, bucketTop(b), bucketValue(b)) + if (b > 0) { + bucketStart = bucketTop(b-1) + count -= bucketValue(b-1) + rank -= bucketValue(b-1) + } + bucketStart + (bucketEnd-bucketStart)*(rank/count) + } + } + result + } + + /** + * Compares two Histograms for equality. + * If the # buckets or bucket bounds are not equal, just compare the top bucket value. + * If they are equal, compare counts from top on down. + */ + def compare(other: Histogram): Int = { + if (numBuckets != other.numBuckets) return topBucketValue compare other.topBucketValue + for { b <- 0 until numBuckets optimized } { + if (bucketTop(b) != other.bucketTop(b)) return topBucketValue compare other.topBucketValue + } + for { b <- (numBuckets - 1) to 0 by -1 optimized } { + val countComp = bucketValue(b) compare other.bucketValue(b) + if (countComp != 0) return countComp + } + return 0 + } + + override def equals(other: Any): Boolean = other match { + case h: Histogram => compare(h) == 0 + case other: Any => false + } + + override def toString: String = + (0 until numBuckets).map { b => s"${bucketTop(b)}=${bucketValue(b)}" }.mkString("{", ", ", "}") + + override def hashCode: Int = { + var hash = 7.0 + for { b <- 0 until numBuckets optimized } { + hash = (31 * bucketTop(b) + hash) * 31 + bucketValue(b) + } + java.lang.Double.doubleToLongBits(hash).toInt + } +} + +object Histogram { + val empty = new Histogram { + final def numBuckets: Int = 0 + final def bucketTop(no: Int): Double = ??? + final def bucketValue(no: Int): Double = ??? + final def serialize(intoBuf: Option[UnsafeBuffer] = None): UnsafeBuffer = { + val buf = intoBuf.getOrElse(BinaryHistogram.histBuf) + BinaryHistogram.writeNonIncreasing(HistogramBuckets.emptyBuckets, Array[Long](), buf) + buf + } + } +} + +/** + * A histogram class that can be used for aggregation and to represent intermediate values + */ +final case class MutableHistogram(buckets: HistogramBuckets, values: Array[Double]) extends Histogram { + final def numBuckets: Int = buckets.numBuckets + final def bucketTop(no: Int): Double = buckets.bucketTop(no) + final def bucketValue(no: Int): Double = values(no) + final def serialize(intoBuf: Option[UnsafeBuffer] = None): UnsafeBuffer = { + val buf = intoBuf.getOrElse(BinaryHistogram.histBuf) + BinaryHistogram.writeNonIncreasing(buckets, values.map(_.toLong), buf) + buf + } + + /** + * Copies this histogram as a new copy so it can be used for aggregation or mutation. Allocates new storage. + */ + final def copy: Histogram = MutableHistogram(buckets, values.clone) + + /** + * Adds the values from another MutableHistogram having the same bucket schema. If it does not, then + * an exception is thrown -- for now. Modifies itself. + */ + final def add(other: MutableHistogram): Unit = + if (buckets == other.buckets) { + for { b <- 0 until numBuckets optimized } { + values(b) += other.values(b) + } + } else { + throw new UnsupportedOperationException(s"Cannot add other with buckets ${other.buckets} to myself $buckets") + } +} + +object MutableHistogram { + def empty(buckets: HistogramBuckets): MutableHistogram = + MutableHistogram(buckets, new Array[Double](buckets.numBuckets)) +} + +/** + * A scheme for buckets in a histogram. Since these are Prometheus-style histograms, + * each bucket definition consists of occurrences of numbers which are less than or equal to the bucketTop + * or definition of each bucket. + */ +sealed trait HistogramBuckets { + def numBuckets: Int + + /** + * Gets the bucket definition for number no. Observations for values <= this bucket, so it represents + * an upper limit. + */ + def bucketTop(no: Int): Double + + /** + * Serializes this bucket scheme to a byte array + */ + def toByteArray: Array[Byte] + + /** + * Materializes all bucket tops into an array. WARNING: Allocation. + */ + final def allBucketTops: Array[Double] = { + val tops = new Array[Double](numBuckets) + for { b <- 0 until numBuckets optimized } { + tops(b) = bucketTop(b) + } + tops + } + + override def toString: String = allBucketTops.mkString("buckets[", ", ", "]") +} + +object HistogramBuckets { + val OffsetNumBuckets = 0 + val OffsetBucketDetails = 2 + + import BinaryHistogram._ + + /** + * Creates the right HistogramBuckets from a binary definition. NOTE: length field not included here + * The first two bytes of any binary bucket schema definition are the number of buckets. + */ + def apply(binaryBucketsDef: Array[Byte], formatCode: Byte): HistogramBuckets = formatCode match { + case HistFormat_Geometric_Delta => geometric(binaryBucketsDef, UnsafeUtils.arayOffset) + case HistFormat_Geometric1_Delta => geometric1(binaryBucketsDef, UnsafeUtils.arayOffset) + case _ => emptyBuckets + } + + def apply(bucketsDef: BinaryRegion.NativePointer, formatCode: Byte): HistogramBuckets = formatCode match { + case HistFormat_Geometric_Delta => geometric(UnsafeUtils.ZeroArray, bucketsDef) + case HistFormat_Geometric1_Delta => geometric1(UnsafeUtils.ZeroArray, bucketsDef) + case _ => emptyBuckets + } + + // Create geometric buckets definition + def geometric(bucketsDefBase: Array[Byte], bucketsDefOffset: Long): HistogramBuckets = + GeometricBuckets(UnsafeUtils.getDouble(bucketsDefBase, bucketsDefOffset + OffsetBucketDetails), + UnsafeUtils.getDouble(bucketsDefBase, bucketsDefOffset + OffsetBucketDetails + 8), + UnsafeUtils.getShort(bucketsDefBase, bucketsDefOffset + OffsetNumBuckets).toInt) + + def geometric1(bucketsDefBase: Array[Byte], bucketsDefOffset: Long): HistogramBuckets = + GeometricBuckets_1(UnsafeUtils.getDouble(bucketsDefBase, bucketsDefOffset + OffsetBucketDetails), + UnsafeUtils.getDouble(bucketsDefBase, bucketsDefOffset + OffsetBucketDetails + 8), + UnsafeUtils.getShort(bucketsDefBase, bucketsDefOffset + OffsetNumBuckets).toInt) + + /** + * Creates a binary bucket definition for a geometric series of histogram buckets. + * That means each bucket contains values <= (firstBucket) * (multipler) ^ (bucketNo) + * where bucketNo starts at 0 and goes till (numBuckets - 1) + * @param firstBucket initial bucket value + * @param multiplier the geometric multiplier between buckets + * @param numBuckets the total number of buckets + */ + final def geometricBucketDef(firstBucket: Double, + multiplier: Double, + numBuckets: Int): Array[Byte] = { + require(numBuckets < 65536, s"Too many buckets: $numBuckets") + val bytes = new Array[Byte](18) + UnsafeUtils.setShort(bytes, UnsafeUtils.arayOffset + OffsetNumBuckets, numBuckets.toShort) + UnsafeUtils.setDouble(bytes, UnsafeUtils.arayOffset + OffsetBucketDetails, firstBucket) + UnsafeUtils.setDouble(bytes, UnsafeUtils.arayOffset + OffsetBucketDetails + 8, multiplier) + bytes + } + + // A bucket definition for the bits of a long, ie from 2^0 to 2^63 + // le's = [1, 3, 7, 15, 31, ....] + val binaryBuckets64 = GeometricBuckets_1(2.0d, 2.0d, 64) + + val emptyBuckets = GeometricBuckets(2.0d, 2.0d, 0) +} + +/** + * A geometric series bucketing scheme, where each successive bucket is a multiple of a previous one. + */ +final case class GeometricBuckets(firstBucket: Double, multiplier: Double, numBuckets: Int) extends HistogramBuckets { + final def bucketTop(no: Int): Double = firstBucket * Math.pow(multiplier, no) + def toByteArray: Array[Byte] = HistogramBuckets.geometricBucketDef(firstBucket, multiplier, numBuckets) +} + +/** + * A variation of a geometric series scheme where the bucket values are 1 less than the geoemtric series. + */ +final case class GeometricBuckets_1(firstBucket: Double, multiplier: Double, numBuckets: Int) extends HistogramBuckets { + final def bucketTop(no: Int): Double = (firstBucket * Math.pow(multiplier, no)) - 1 + def toByteArray: Array[Byte] = HistogramBuckets.geometricBucketDef(firstBucket, multiplier, numBuckets) +} diff --git a/memory/src/main/scala/filodb.memory/format/vectors/HistogramCompressor.scala b/memory/src/main/scala/filodb.memory/format/vectors/HistogramCompressor.scala new file mode 100644 index 0000000000..9c9bd7570b --- /dev/null +++ b/memory/src/main/scala/filodb.memory/format/vectors/HistogramCompressor.scala @@ -0,0 +1,66 @@ +package filodb.memory.format.vectors + +import scala.io.Source + +import org.agrona.concurrent.UnsafeBuffer + +import filodb.memory.NativeMemoryManager + +//scalastyle:off +// Input is a file with one line per histogram, bucket values are comma separated +// This app is designed to measure histogram compression ratios based on real world histogram data +object HistogramCompressor extends App { + if (args.length < 1) { + println("Usage: sbt memory/run ") + println("Tests chunk compression, not runtime or performance, against real-world increasing bucket histogram files") + sys.exit(0) + } + val inputFile = args(0) + val chunkLength = if (args.length > 1) args(1).toInt else 300 + val numChunks = 20 + + val memFactory = new NativeMemoryManager(500 * 1024 * 1024) + val inputBuffer = new UnsafeBuffer(new Array[Byte](8192)) + + val appender = HistogramVector.appendingColumnar(memFactory, 64, chunkLength) + val bucketDef = HistogramBuckets.binaryBuckets64 + + var binHistBytesSum = 0 + var binHistBytesMax = 0 + var numRecords = 0 + + var encodedTotal = 0 + var writeBufferTotal = 0 + + Source.fromFile(inputFile).getLines + .take(numChunks * chunkLength) + .grouped(chunkLength).foreach { chunkLines => + // Ingest each histogram, parse and create a BinaryHistogram, then ingest into our histogram column + chunkLines.foreach { line => + val buckets = line.split(",").map(_.trim.toLong) + val histSize = BinaryHistogram.writeNonIncreasing(bucketDef, buckets, inputBuffer) + numRecords += 1 + binHistBytesMax = Math.max(binHistBytesMax, histSize) + binHistBytesSum += histSize + + appender.addData(inputBuffer) + } + + // Optimize and get optimized size, dump out, aggregate + val writeBufSize = HistogramVector.columnarTotalSize(appender.addr) + val optimized = appender.optimize(memFactory) + val encodedSize = HistogramVector.columnarTotalSize(optimized) + println(s" WriteBuffer size: ${writeBufSize}\t\tEncoded size: $encodedSize") + encodedTotal += encodedSize + writeBufferTotal += writeBufSize + } + + // Dump out overall aggregates + val avgEncoded = encodedTotal.toDouble / numChunks + val avgWriteBuf = writeBufferTotal.toDouble / numChunks + println(s"Average encoded chunk size: $avgEncoded") + println(s"Average write buffer size: $avgWriteBuf") + println(s"Compression ratio: ${avgWriteBuf / avgEncoded}") + println(s"Average binHistogram size: ${binHistBytesSum / numRecords.toDouble}") + println(s"Max binHistogram size: $binHistBytesMax bytes") +} \ No newline at end of file diff --git a/memory/src/main/scala/filodb.memory/format/vectors/HistogramVector.scala b/memory/src/main/scala/filodb.memory/format/vectors/HistogramVector.scala new file mode 100644 index 0000000000..b8db4de191 --- /dev/null +++ b/memory/src/main/scala/filodb.memory/format/vectors/HistogramVector.scala @@ -0,0 +1,370 @@ +package filodb.memory.format.vectors + +import debox.Buffer +import org.agrona.concurrent.UnsafeBuffer +import scalaxy.loops._ + +import filodb.memory.{BinaryRegion, MemFactory} +import filodb.memory.format._ +import filodb.memory.format.BinaryVector.BinaryVectorPtr +import filodb.memory.format.Encodings._ + +/** + * BinaryHistogram is the binary format for a histogram binary blob included in BinaryRecords and sent over the wire. + * It fits the BinaryRegionMedium protocol. + * Format: + * +0000 u16 2-byte total length of this BinaryHistogram (excluding this length) + * +0002 u8 1-byte combined histogram buckets and values format code + * 0x00 Empty/null histogram + * 0x01 geometric + NibblePacked delta Long values + * 0x02 geometric_1 + NibblePacked delta Long values (see [[HistogramBuckets]]) + * + * +0003 u16 2-byte length of Histogram bucket definition + * +0005 [u8] Histogram bucket definition, see [[HistogramBuckets]] + * First two bytes of definition is always the number of buckets, a u16 + * +(5+n) remaining values according to format above + * + * NOTE: most of the methods below actually expect a pointer to the +2 hist bucket definition, not the length field + */ +object BinaryHistogram { + // Pass in a buffer which includes the length bytes. Value class - no allocations. + case class BinHistogram(buf: UnsafeBuffer) extends AnyVal { + def totalLength: Int = buf.getShort(0).toInt + 2 + def numBuckets: Int = buf.getShort(5).toInt + def formatCode: Byte = buf.getByte(2) + def bucketDefNumBytes: Int = buf.getShort(3).toInt + def bucketDefOffset: Long = buf.addressOffset + 5 + def valuesIndex: Int = 2 + 3 + bucketDefNumBytes // pointer to values bytes + def valuesNumBytes: Int = totalLength - valuesIndex + def intoValuesBuf(destBuf: UnsafeBuffer): Unit = + UnsafeUtils.wrapUnsafeBuf(buf.byteArray, buf.addressOffset + valuesIndex, valuesNumBytes, destBuf) + override def toString: String = s"" + + /** + * Converts this BinHistogram to a Histogram object. May not be the most efficient. + * Intended for slower paths such as high level (lower # samples) aggregation and HTTP/CLI materialization + * by clients. Materializes/deserializes everything. + * Ingestion ingests BinHistograms directly without conversion to Histogram first. + */ + def toHistogram: Histogram = formatCode match { + case HistFormat_Geometric_Delta => + val bucketDef = HistogramBuckets.geometric(buf.byteArray, bucketDefOffset) + // TODO: flat buckets won't be supported anymore. Fix this + FlatBucketHistogram(bucketDef, this) + case HistFormat_Geometric1_Delta => + val bucketDef = HistogramBuckets.geometric1(buf.byteArray, bucketDefOffset) + // TODO: flat buckets won't be supported anymore. Fix this + FlatBucketHistogram(bucketDef, this) + } + } + + private val tlValuesBuf = new ThreadLocal[UnsafeBuffer]() + def valuesBuf: UnsafeBuffer = tlValuesBuf.get match { + case UnsafeUtils.ZeroPointer => val buf = new UnsafeBuffer(new Array[Byte](4096)) + tlValuesBuf.set(buf) + buf + case b: UnsafeBuffer => b + } + + // Thread local buffer used as temp buffer for writing binary histograms + private val tlHistBuf = new ThreadLocal[UnsafeBuffer]() + def histBuf: UnsafeBuffer = tlHistBuf.get match { + case UnsafeUtils.ZeroPointer => val buf = new UnsafeBuffer(new Array[Byte](8192)) + tlHistBuf.set(buf) + buf + case b: UnsafeBuffer => b + } + + val HistFormat_Null = 0x00.toByte + val HistFormat_Geometric_Delta = 0x01.toByte + val HistFormat_Geometric1_Delta = 0x02.toByte + + case class FlatBucketValues(buf: UnsafeBuffer) extends AnyVal { + def bucket(no: Int): Long = buf.getLong(no * 8) + } + + private case class FlatBucketHistogram(buckets: HistogramBuckets, binHist: BinHistogram) extends Histogram { + binHist.intoValuesBuf(valuesBuf) + final def numBuckets: Int = buckets.numBuckets + final def bucketTop(no: Int): Double = buckets.bucketTop(no) + final def bucketValue(no: Int): Double = FlatBucketValues(valuesBuf).bucket(no) + final def serialize(intoBuf: Option[UnsafeBuffer] = None): UnsafeBuffer = + intoBuf.map { x => x.wrap(binHist.buf); x }.getOrElse(binHist.buf) + } + + /** + * Writes binary histogram with geometric bucket definition and data which is non-increasing, but will be + * decoded as increasing. Intended only for specific use cases when the source histogram are non increasing + * buckets, ie each bucket has a count that is independent. + * Buckets are written as-is for now. + * @return the number of bytes written, including the length prefix + */ + def writeNonIncreasing(buckets: HistogramBuckets, values: Array[Long], buf: UnsafeBuffer): Int = { + val formatCode = buckets match { + case g: GeometricBuckets => HistFormat_Geometric_Delta + case g: GeometricBuckets_1 => HistFormat_Geometric1_Delta + case _ => ??? + } + val bucketDef = buckets.toByteArray + + val bytesNeeded = 2 + 1 + 2 + bucketDef.size + 8 * values.size + require(bytesNeeded < 65535, s"Histogram data is too large: $bytesNeeded bytes needed") + require(buf.capacity >= bytesNeeded, s"Buffer only has ${buf.capacity} bytes but we need $bytesNeeded") + + buf.putShort(0, (bytesNeeded - 2).toShort) + buf.putByte(2, formatCode) + buf.putShort(3, bucketDef.size.toShort) + buf.putBytes(5, bucketDef) + val valuesIndex = 5 + bucketDef.size + for { b <- 0 until values.size optimized } { + buf.putLong(valuesIndex + b * 8, values(b)) + } + bytesNeeded + } + + def writeNonIncreasing(buckets: HistogramBuckets, values: Array[Long]): Int = + writeNonIncreasing(buckets, values, histBuf) +} + +object HistogramVector { + type HistIterator = Iterator[Histogram] with TypedIterator + + val OffsetNumHistograms = 6 + val OffsetFormatCode = 8 // u8: BinHistogram format code/bucket type + val OffsetBucketDefSize = 9 // # of bytes of bucket definition, including bucket def type + val OffsetBucketDef = 11 // Start of bucket definition + val OffsetNumBuckets = 11 + // After the bucket area are regions for storing the counter values or pointers to them + + final def getNumBuckets(addr: BinaryVectorPtr): Int = UnsafeUtils.getShort(addr + OffsetNumBuckets).toInt + + final def getNumHistograms(addr: BinaryVectorPtr): Int = UnsafeUtils.getShort(addr + OffsetNumHistograms).toInt + final def resetNumHistograms(addr: BinaryVectorPtr): Unit = + UnsafeUtils.setShort(addr + OffsetNumHistograms, 0) + final def incrNumHistograms(addr: BinaryVectorPtr): Unit = + UnsafeUtils.setShort(addr + OffsetNumHistograms, (getNumHistograms(addr) + 1).toShort) + + final def formatCode(addr: BinaryVectorPtr): Byte = UnsafeUtils.getByte(addr + OffsetFormatCode) + final def afterBucketDefAddr(addr: BinaryVectorPtr): BinaryRegion.NativePointer = + addr + OffsetBucketDef + bucketDefNumBytes(addr) + final def bucketDefNumBytes(addr: BinaryVectorPtr): Int = UnsafeUtils.getShort(addr + OffsetBucketDefSize).toInt + final def bucketDefAddr(addr: BinaryVectorPtr): BinaryRegion.NativePointer = addr + OffsetBucketDef + + // Matches the bucket definition whose # bytes is at (base, offset) + final def matchBucketDef(hist: BinaryHistogram.BinHistogram, addr: BinaryVectorPtr): Boolean = + (hist.formatCode == formatCode(addr)) && + (hist.bucketDefNumBytes == bucketDefNumBytes(addr)) && { + UnsafeUtils.equate(UnsafeUtils.ZeroPointer, addr + OffsetBucketDef, hist.buf.byteArray, hist.bucketDefOffset, + hist.bucketDefNumBytes) + } + + // Columnar HistogramVectors composed of multiple vectors, this calculates total used size + def columnarTotalSize(addr: BinaryVectorPtr): Int = { + val bucketAddrPtr = afterBucketDefAddr(addr) + val headerBytes = UnsafeUtils.getInt(addr) + headerBytes + (0 until getNumBuckets(addr)).map { b => + val bucketVectorAddr = UnsafeUtils.getLong(bucketAddrPtr + 8*b) + UnsafeUtils.getInt(bucketVectorAddr) + 4 + }.sum + } + + val ReservedBucketDefSize = 256 + def appendingColumnar(factory: MemFactory, numBuckets: Int, maxItems: Int): ColumnarAppendableHistogramVector = { + // Really just an estimate. TODO: if we really go with columnar, make it more accurate + val neededBytes = OffsetBucketDef + ReservedBucketDefSize + 8 * numBuckets + val addr = factory.allocateOffheap(neededBytes) + new ColumnarAppendableHistogramVector(factory, addr, maxItems) + } + + def apply(p: BinaryVectorPtr): HistogramReader = + new ColumnarHistogramReader(p) +} + +/** + * A HistogramVector appender composed of individual primitive columns. + * Just a POC to get started quickly and as a reference. + * Note that the bucket schema is not set until getting the first item. + * After the bucket definition: + * An array [u64] of native pointers to the individual columns + * + * TODO: initialize num bytes and vector type stuff + * + * Read/Write/Lock semantics: everything is gated by the number of elements. + * When it is 0, nothing is initialized so the reader guards against that. + * When it is > 0, then all structures are initialized. + */ +class ColumnarAppendableHistogramVector(factory: MemFactory, + val addr: BinaryVectorPtr, + maxItems: Int) extends BinaryAppendableVector[UnsafeBuffer] { + import HistogramVector._ + import BinaryHistogram._ + resetNumHistograms(addr) + + private var bucketAppenders: Option[Array[BinaryAppendableVector[Long]]] = None + + val dispose = () => { + // first, free memory from each appender + bucketAppenders.foreach(_.foreach(_.dispose())) + // free our own memory + factory.freeMemory(addr) + } + + final def numBytes: Int = UnsafeUtils.getInt(addr) + 4 + final def maxBytes: Int = numBytes + final def length: Int = getNumHistograms(addr) + final def isAvailable(index: Int): Boolean = true + final def isAllNA: Boolean = (length == 0) + final def noNAs: Boolean = (length > 0) + + private val valueBuf = new UnsafeBuffer(Array.empty[Byte]) + + // NOTE: to eliminate allocations, re-use the UnsafeBuffer and keep passing the same instance to addData + final def addData(buf: UnsafeBuffer): AddResponse = { + val numItems = getNumHistograms(addr) + val h = BinHistogram(buf) + val numBuckets = h.numBuckets + if (numItems == 0) { + // Copy the bucket definition and set the bucket def size + UnsafeUtils.unsafe.copyMemory(buf.byteArray, h.bucketDefOffset, + UnsafeUtils.ZeroPointer, bucketDefAddr(addr), h.bucketDefNumBytes) + UnsafeUtils.setShort(addr + OffsetBucketDefSize, h.bucketDefNumBytes.toShort) + UnsafeUtils.setByte(addr + OffsetFormatCode, h.formatCode) + + // initialize the buckets + initBuckets(numBuckets) + } else if (numItems >= maxItems) { + return VectorTooSmall(0, 0) + } else { + // check the bucket schema is identical. If not, return BucketSchemaMismatch + if (!matchBucketDef(h, addr)) return BucketSchemaMismatch + } + + // Now, iterate through the counters and add them to each individual vector + h.intoValuesBuf(valueBuf) + val values = FlatBucketValues(valueBuf) + bucketAppenders.foreach { appenders => + for { b <- 0 until numBuckets optimized } { + val resp = appenders(b).addData(values.bucket(b)) + require(resp == Ack) + } + } + + incrNumHistograms(addr) + Ack + } + + final def addNA(): AddResponse = Ack // TODO: Add a 0 to every appender + + def addFromReaderNoNA(reader: RowReader, col: Int): AddResponse = addData(reader.blobAsBuffer(col)) + def copyToBuffer: Buffer[UnsafeBuffer] = ??? + def apply(index: Int): UnsafeBuffer = ??? + + def finishCompaction(newAddress: BinaryRegion.NativePointer): BinaryVectorPtr = newAddress + + // NOTE: do not access reader below unless this vect is nonempty. TODO: fix this, or don't if we don't use this class + lazy val reader: VectorDataReader = new ColumnarHistogramReader(addr) + + def reset(): Unit = { + bucketAppenders.foreach(_.foreach(_.dispose())) + bucketAppenders = None + resetNumHistograms(addr) + } + + // Optimize each bucket's appenders, then create a new region with the same headers but pointing at the + // optimized vectors. + // TODO: this is NOT safe for persistence and recovery, as pointers cannot be persisted or recovered. + // For us to really make persistence of this work, we would need to pursue one of these strategies: + // 1) Change code of each LongAppendingVector to tell us how much optimized bytes take up for each bucket, + // then do a giant allocation including every bucket, and use relative pointers, not absolute, to point + // to each one; (or possibly a different kind of allocator) + // 2) Use BlockIDs and offsets instead of absolute pointers, and persist entire blocks. + override def optimize(memFactory: MemFactory, hint: EncodingHint = AutoDetect): BinaryVectorPtr = { + val optimizedBuckets = bucketAppenders.map { appenders => + appenders.map(_.optimize(memFactory, hint)) + }.getOrElse(Array.empty[BinaryVectorPtr]) + + val newHeaderAddr = memFactory.allocateOffheap(numBytes) + // Copy headers including bucket def + val bucketPtrOffset = (afterBucketDefAddr(addr) - addr).toInt + UnsafeUtils.copy(addr, newHeaderAddr, bucketPtrOffset) + + for { b <- 0 until optimizedBuckets.size optimized } { + UnsafeUtils.setLong(newHeaderAddr + bucketPtrOffset + 8*b, optimizedBuckets(b)) + } + + newHeaderAddr + } + + // NOTE: allocating vectors during ingestion is a REALLY BAD idea. For one if one runs out of memory then + // it will fail but ingestion into other vectors might succeed, resulting in undefined switchBuffers behaviors. + private def initBuckets(numBuckets: Int): Unit = { + val bucketPointersAddr = afterBucketDefAddr(addr) + val appenders = (0 until numBuckets).map { b => + val appender = LongBinaryVector.appendingVectorNoNA(factory, maxItems) + UnsafeUtils.setLong(bucketPointersAddr + 8*b, appender.addr) + appender + } + bucketAppenders = Some(appenders.toArray) + + // Initialize number of bytes in this histogram header + UnsafeUtils.setInt(addr, (bucketPointersAddr - addr).toInt + 8 * numBuckets) + } +} + +trait HistogramReader extends VectorDataReader { + def buckets: HistogramBuckets + def apply(index: Int): Histogram + def sum(start: Int, end: Int): MutableHistogram +} + +class ColumnarHistogramReader(histVect: BinaryVectorPtr) extends HistogramReader { + import HistogramVector._ + + final def length: Int = getNumHistograms(histVect) + val numBuckets = if (length > 0) getNumBuckets(histVect) else 0 + val bucketAddrs = if (length > 0) { + val bucketAddrBase = afterBucketDefAddr(histVect) + (0 until numBuckets).map(b => UnsafeUtils.getLong(bucketAddrBase + 8 * b)).toArray + } else { + Array.empty[BinaryVectorPtr] + } + val readers = if (length > 0) bucketAddrs.map(LongBinaryVector.apply) else Array.empty[LongVectorDataReader] + + val buckets = HistogramBuckets(bucketDefAddr(histVect), formatCode(histVect)) + val returnHist = MutableHistogram.empty(buckets) + + /** + * Iterates through each histogram. Note this is expensive due to materializing the Histogram object + * every time. Using higher level functions such as sum is going to be a much better bet usually. + */ + def iterate(vector: BinaryVectorPtr, startElement: Int): TypedIterator = + new Iterator[Histogram] with TypedIterator { + var elem = startElement + def hasNext: Boolean = elem < getNumHistograms(histVect) + def next: Histogram = { + val h = apply(elem) + elem += 1 + h + } + } + + def length(addr: BinaryVectorPtr): Int = length + + // WARNING: histogram returned is shared between calls, do not reuse! + final def apply(index: Int): Histogram = { + require(length > 0) + for { b <- 0 until numBuckets optimized } { + returnHist.values(b) = readers(b).apply(bucketAddrs(b), index) + } + returnHist + } + + // sum_over_time returning a Histogram with sums for each bucket. Start and end are inclusive row numbers + final def sum(start: Int, end: Int): MutableHistogram = { + require(length > 0 && start >= 0 && end < length) + for { b <- 0 until numBuckets optimized } { + returnHist.values(b) = readers(b).sum(bucketAddrs(b), start, end) + } + returnHist + } +} \ No newline at end of file diff --git a/memory/src/main/scala/filodb.memory/format/vectors/IntBinaryVector.scala b/memory/src/main/scala/filodb.memory/format/vectors/IntBinaryVector.scala index 85205eca55..5c901fec46 100644 --- a/memory/src/main/scala/filodb.memory/format/vectors/IntBinaryVector.scala +++ b/memory/src/main/scala/filodb.memory/format/vectors/IntBinaryVector.scala @@ -84,7 +84,7 @@ object IntBinaryVector { case 4 => new IntAppendingVector(addr, maxBytes, nbits, signed, dispose) { final def addData(v: Int): AddResponse = checkOffset() match { case Ack => - val origByte = UnsafeUtils.getByte(writeOffset) + val origByte = if (bitShift == 0) 0 else UnsafeUtils.getByte(writeOffset) val newByte = (origByte | (v << bitShift)).toByte UnsafeUtils.setByte(writeOffset, newByte) bumpBitShift() @@ -95,7 +95,7 @@ object IntBinaryVector { case 2 => new IntAppendingVector(addr, maxBytes, nbits, signed, dispose) { final def addData(v: Int): AddResponse = checkOffset() match { case Ack => - val origByte = UnsafeUtils.getByte(writeOffset) + val origByte = if (bitShift == 0) 0 else UnsafeUtils.getByte(writeOffset) val newByte = (origByte | (v << bitShift)).toByte UnsafeUtils.setByte(writeOffset, newByte) bumpBitShift() diff --git a/memory/src/test/scala/filodb.memory/format/vectors/HistogramTest.scala b/memory/src/test/scala/filodb.memory/format/vectors/HistogramTest.scala new file mode 100644 index 0000000000..ceebdec65d --- /dev/null +++ b/memory/src/test/scala/filodb.memory/format/vectors/HistogramTest.scala @@ -0,0 +1,91 @@ +package filodb.memory.format.vectors + +import org.agrona.concurrent.UnsafeBuffer + +object HistogramTest { + val bucketScheme = GeometricBuckets(1.0, 2.0, 8) + val rawHistBuckets = Seq( + Array[Double](10, 15, 17, 20, 25, 34, 76, 82), + Array[Double](6, 16, 26, 26, 36, 38, 56, 59), + Array[Double](11, 16, 26, 27, 33, 42, 46, 55), + Array[Double](4, 4, 5, 33, 35, 67, 91, 121) + ) + val rawLongBuckets = rawHistBuckets.map(_.map(_.toLong)) + val mutableHistograms = rawHistBuckets.map { buckets => + MutableHistogram(bucketScheme, buckets) + } + + val quantile50Result = Seq(37.333333333333336, 10.8, 8.666666666666666, 28.75) +} + +import BinaryHistogram._ + +class HistogramTest extends NativeVectorTest { + describe("HistogramBuckets") { + it("can list out bucket definition LE values properly for Geometric and Geometric_1") { + val buckets1 = GeometricBuckets(5.0, 3.0, 4) + buckets1.allBucketTops shouldEqual Array(5.0, 15.0, 45.0, 135.0) + + val buckets2 = GeometricBuckets_1(2.0, 2.0, 8) + buckets2.allBucketTops shouldEqual Array(1.0, 3.0, 7.0, 15.0, 31.0, 63.0, 127.0, 255.0) + } + + it("can serialize and deserialize properly") { + val buckets1 = GeometricBuckets(5.0, 2.0, 4) + HistogramBuckets(buckets1.toByteArray, HistFormat_Geometric_Delta) shouldEqual buckets1 + + val buckets2 = GeometricBuckets_1(2.0, 2.0, 8) + HistogramBuckets(buckets2.toByteArray, HistFormat_Geometric1_Delta) shouldEqual buckets2 + } + } + + import HistogramTest._ + + describe("Histogram") { + it("should calculate quantile correctly") { + mutableHistograms.zip(quantile50Result).foreach { case (h, res) => + val quantile = h.quantile(0.50) + info(s"For histogram ${h.values.toList} -> quantile = $quantile") + quantile shouldEqual res + } + } + + it("should serialize to and from BinaryHistograms and compare correctly") { + val binHistograms = mutableHistograms.map { h => + val buf = new UnsafeBuffer(new Array[Byte](2048)) + h.serialize(Some(buf)) + } + + binHistograms.zip(mutableHistograms).foreach { case (binHistBuf, mutHist) => + val binHist = BinaryHistogram.BinHistogram(binHistBuf).toHistogram + binHist shouldEqual mutHist + binHist.hashCode shouldEqual mutHist.hashCode + println(binHist) + } + } + + it("should serialize to and from an empty Histogram") { + val binEmptyHist = BinaryHistogram.BinHistogram(Histogram.empty.serialize()) + binEmptyHist.numBuckets shouldEqual 0 + binEmptyHist.toHistogram shouldEqual Histogram.empty + Histogram.empty.toString shouldEqual "{}" + } + + it("should compare different histograms correctly") { + mutableHistograms(0) shouldEqual mutableHistograms(0) + mutableHistograms(0) should not equal ("boofoo") + + mutableHistograms(0).compare(mutableHistograms(1)) should be > 0 + } + + it("should copy and not be affected by mutation to original") { + val addedBuckets = rawHistBuckets(0).zip(rawHistBuckets(1)).map { case(a,b) => a + b }.toArray + val hist = mutableHistograms(0) + val hist2 = hist.copy + hist shouldEqual hist2 + hist.add(mutableHistograms(1)) + hist should not equal (hist2) + hist.values shouldEqual addedBuckets + } + } +} \ No newline at end of file diff --git a/memory/src/test/scala/filodb.memory/format/vectors/HistogramVectorTest.scala b/memory/src/test/scala/filodb.memory/format/vectors/HistogramVectorTest.scala new file mode 100644 index 0000000000..3ca12d372b --- /dev/null +++ b/memory/src/test/scala/filodb.memory/format/vectors/HistogramVectorTest.scala @@ -0,0 +1,111 @@ +package filodb.memory.format.vectors + +import org.agrona.concurrent.UnsafeBuffer + +import filodb.memory.format._ + +class HistogramVectorTest extends NativeVectorTest { + import HistogramTest._ + + it("should throw exceptions trying to query empty HistogramVector") { + val appender = HistogramVector.appendingColumnar(memFactory, 8, 100) + + appender.length shouldEqual 0 + appender.isAllNA shouldEqual true + val reader = appender.reader.asInstanceOf[ColumnarHistogramReader] + + reader.length(appender.addr) shouldEqual 0 + reader.numBuckets shouldEqual 0 + intercept[IllegalArgumentException] { reader(0) } + } + + val buffer = new UnsafeBuffer(new Array[Byte](4096)) + + def verifyHistogram(h: Histogram, itemNo: Int): Unit = { + h.numBuckets shouldEqual bucketScheme.numBuckets + for { i <- 0 until bucketScheme.numBuckets } { + h.bucketTop(i) shouldEqual bucketScheme.bucketTop(i) + h.bucketValue(i) shouldEqual rawHistBuckets(itemNo)(i) + } + } + + it("should accept BinaryHistograms of the same schema and be able to query them") { + val appender = HistogramVector.appendingColumnar(memFactory, 8, 50) + rawLongBuckets.foreach { rawBuckets => + BinaryHistogram.writeNonIncreasing(bucketScheme, rawBuckets, buffer) + appender.addData(buffer) shouldEqual Ack + } + + appender.length shouldEqual rawHistBuckets.length + + val reader = appender.reader.asInstanceOf[ColumnarHistogramReader] + reader.length shouldEqual rawHistBuckets.length + + (0 until rawHistBuckets.length).foreach { i => + val h = reader(i) + verifyHistogram(h, i) + } + + reader.iterate(0, 0).asInstanceOf[Iterator[Histogram]] + .zipWithIndex.foreach { case (h, i) => verifyHistogram(h, i) } + } + + it("should optimize histograms and be able to query optimized vectors") { + val appender = HistogramVector.appendingColumnar(memFactory, 8, 50) + rawLongBuckets.foreach { rawBuckets => + BinaryHistogram.writeNonIncreasing(bucketScheme, rawBuckets, buffer) + appender.addData(buffer) shouldEqual Ack + } + + appender.length shouldEqual rawHistBuckets.length + + val reader = appender.reader.asInstanceOf[ColumnarHistogramReader] + reader.length shouldEqual rawHistBuckets.length + + (0 until rawHistBuckets.length).foreach { i => + val h = reader(i) + verifyHistogram(h, i) + } + + val optimized = appender.optimize(memFactory) + val optReader = new ColumnarHistogramReader(optimized) + optReader.length shouldEqual rawHistBuckets.length + (0 until rawHistBuckets.length).foreach { i => + val h = optReader(i) + verifyHistogram(h, i) + } + + val sum = optReader.sum(0, rawHistBuckets.length - 1) // should not crash + val expected = (0 until 8).map { b => rawHistBuckets.map(_(b)).sum }.toArray + sum.values shouldEqual expected + + appender.reset() + appender.length shouldEqual 0 + } + + it("should reject BinaryHistograms of schema different from first schema ingested") { + val appender = HistogramVector.appendingColumnar(memFactory, 8, 50) + rawLongBuckets.foreach { rawBuckets => + BinaryHistogram.writeNonIncreasing(bucketScheme, rawBuckets, buffer) + appender.addData(buffer) shouldEqual Ack + } + + appender.length shouldEqual rawHistBuckets.length + + // A record using a different schema + BinaryHistogram.writeNonIncreasing(HistogramBuckets.binaryBuckets64, Array[Long](0, 1, 2, 0), buffer) + appender.addData(buffer) shouldEqual BucketSchemaMismatch + } + + it("should reject new adds when vector is full") { + val appender = HistogramVector.appendingColumnar(memFactory, 8, 4) + rawLongBuckets.foreach { rawBuckets => + BinaryHistogram.writeNonIncreasing(bucketScheme, rawBuckets, buffer) + appender.addData(buffer) shouldEqual Ack + } + + appender.length shouldEqual rawHistBuckets.length + + appender.addData(buffer) shouldEqual VectorTooSmall(0, 0) + } +} \ No newline at end of file diff --git a/memory/src/test/scala/filodb.memory/format/vectors/IntBinaryVectorTest.scala b/memory/src/test/scala/filodb.memory/format/vectors/IntBinaryVectorTest.scala index aec26e643e..bfb07f70cf 100644 --- a/memory/src/test/scala/filodb.memory/format/vectors/IntBinaryVectorTest.scala +++ b/memory/src/test/scala/filodb.memory/format/vectors/IntBinaryVectorTest.scala @@ -2,6 +2,7 @@ package filodb.memory.format.vectors import debox.Buffer +import filodb.memory.{BlockMemFactory, MemoryStats, PageAlignedBlockManager} import filodb.memory.format._ class IntBinaryVectorTest extends NativeVectorTest { @@ -94,6 +95,28 @@ class IntBinaryVectorTest extends NativeVectorTest { IntBinaryVector(frozen).toBuffer(frozen).toList shouldEqual orig } + it("should append correctly when memory has previous values / was not zeroed") { + import collection.JavaConverters._ + val blockStore = new PageAlignedBlockManager(10 * 1024 * 1024, new MemoryStats(Map("test"-> "test")), null, 16) { + freeBlocks.asScala.foreach(_.set(0x55)) // initialize blocks to nonzero value + } + val blockFactory = new BlockMemFactory(blockStore, None, 24, true) + + // original values will get mixed with nonzero contents if append does not overwrite original memory + val builder = IntBinaryVector.appendingVectorNoNA(blockFactory, 10, nbits=4, signed=false) + val orig = Seq(0, 1, 1, 3, 4) + orig.foreach(x => builder.addData(x) shouldEqual Ack) + builder.reader.toBuffer(builder.addr).toList shouldEqual orig + + // original values will get mixed with nonzero contents if append does not overwrite original memory + val builder2 = IntBinaryVector.appendingVectorNoNA(blockFactory, 10, nbits=2, signed=false) + val orig2 = Seq(0, 1, 1, 0, 2) + orig2.foreach(x => builder2.addData(x) shouldEqual Ack) + builder2.reader.toBuffer(builder2.addr).toList shouldEqual orig2 + + blockStore.releaseBlocks() + } + it("should optimize even with NoNA vectors to less nbits") { val orig = Seq(0, 2, 1, 3, 2) val builder1 = IntBinaryVector(memFactory, orig) diff --git a/query/src/main/scala/filodb/query/exec/HistogramQuantileMapper.scala b/query/src/main/scala/filodb/query/exec/HistogramQuantileMapper.scala index 7f354fef82..69e9a6f6a3 100644 --- a/query/src/main/scala/filodb/query/exec/HistogramQuantileMapper.scala +++ b/query/src/main/scala/filodb/query/exec/HistogramQuantileMapper.scala @@ -1,10 +1,12 @@ package filodb.query.exec import monix.reactive.Observable +import org.agrona.concurrent.UnsafeBuffer import filodb.core.query._ import filodb.memory.format.{RowReader, ZeroCopyUTF8String} -import filodb.query.{Query, QueryConfig} +import filodb.memory.format.vectors.Histogram +import filodb.query.QueryConfig object HistogramQuantileMapper { import ZeroCopyUTF8String._ @@ -101,40 +103,23 @@ case class HistogramQuantileMapper(funcParams: Seq[Any]) extends RangeVectorTran } } + private case class PromRateHistogram(buckets: Array[Bucket]) extends Histogram { + final def numBuckets: Int = buckets.size + final def bucketTop(no: Int): Double = buckets(no).le + final def bucketValue(no: Int): Double = buckets(no).rate + final def serialize(intoBuf: Option[UnsafeBuffer] = None): UnsafeBuffer = ??? + } + /** * Calculates histogram quantile using the bucket values. * Similar to prometheus implementation for consistent results. */ private def histogramQuantile(q: Double, buckets: Array[Bucket]): Double = { - val result = if (q < 0) Double.NegativeInfinity - else if (q > 1) Double.PositiveInfinity - else if (buckets.length < 2) Double.NaN + if (!buckets.last.le.isPosInfinity) return Double.NaN else { - if (!buckets.last.le.isPosInfinity) return Double.NaN - else { - makeMonotonic(buckets) - // find rank for the quantile using total number of occurrences - var rank = q * buckets.last.rate - // using rank, find the le bucket which would have the identified rank - val b = buckets.indexWhere(_.rate >= rank) - - // now calculate quantile - if (b == buckets.length-1) return buckets(buckets.length-2).le - else if (b == 0 && buckets.head.le <= 0) return buckets.head.le - else { - // interpolate quantile within le bucket - var (bucketStart, bucketEnd, count) = (0d, buckets(b).le, buckets(b).rate) - if (b > 0) { - bucketStart = buckets(b-1).le - count -= buckets(b-1).rate - rank -= buckets(b-1).rate - } - bucketStart + (bucketEnd-bucketStart)*(rank/count) - } - } + makeMonotonic(buckets) + PromRateHistogram(buckets).quantile(q) } - Query.qLogger.debug(s"Quantile $q for buckets $buckets was $result") - result } /** diff --git a/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala b/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala index eb90478c85..b70072950d 100644 --- a/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala +++ b/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala @@ -7,7 +7,7 @@ import org.jctools.queues.SpscUnboundedArrayQueue import filodb.core.metadata.Column.ColumnType import filodb.core.metadata.Dataset import filodb.core.query._ -import filodb.core.store.WindowedChunkIterator +import filodb.core.store.{ChunkSetInfo, WindowedChunkIterator} import filodb.memory.format.{vectors => bv, RowReader} import filodb.query.{BadQueryException, Query, QueryConfig, RangeFunctionId} import filodb.query.exec.rangefn.{ChunkedRangeFunction, RangeFunction, Window} @@ -48,30 +48,35 @@ final case class PeriodicSamplesMapper(start: Long, // Generate one range function to check if it is chunked val sampleRangeFunc = rangeFuncGen() + // Really, use the stale lookback window size, not 0 which doesn't make sense + val windowLength = window.getOrElse(if (functionId == None) queryConfig.staleSampleAfterMs else 0L) + sampleRangeFunc match { - // Chunked: use it and trust it has the right type - case c: ChunkedRangeFunction => - // Really, use the stale lookback window size, not 0 which doesn't make sense - val windowLength = window.getOrElse(if (functionId == None) queryConfig.staleSampleAfterMs + 1 else 0L) + case c: ChunkedRangeFunction[_] if valColType == ColumnType.HistogramColumn => + source.map { rv => + IteratorBackedRangeVector(rv.key, + new ChunkedWindowIteratorH(rv.asInstanceOf[RawDataRangeVector], start, step, end, + windowLength, rangeFuncGen().asChunkedH, queryConfig)) + } + case c: ChunkedRangeFunction[_] => source.map { rv => IteratorBackedRangeVector(rv.key, - new ChunkedWindowIterator(rv.asInstanceOf[RawDataRangeVector], start, step, end, - windowLength, rangeFuncGen().asInstanceOf[ChunkedRangeFunction], - queryConfig)) + new ChunkedWindowIteratorD(rv.asInstanceOf[RawDataRangeVector], start, step, end, + windowLength, rangeFuncGen().asChunkedD, queryConfig)) } // Iterator-based: Wrap long columns to yield a double value case f: RangeFunction if valColType == ColumnType.LongColumn => source.map { rv => IteratorBackedRangeVector(rv.key, new SlidingWindowIterator(new LongToDoubleIterator(rv.rows), start, step, end, window.getOrElse(0L), - rangeFuncGen(), queryConfig)) + rangeFuncGen().asSliding, queryConfig)) } // Otherwise just feed in the double column case f: RangeFunction => source.map { rv => IteratorBackedRangeVector(rv.key, new SlidingWindowIterator(rv.rows, start, step, end, window.getOrElse(0L), - rangeFuncGen(), queryConfig)) + rangeFuncGen().asSliding, queryConfig)) } } } @@ -91,25 +96,20 @@ final case class PeriodicSamplesMapper(start: Long, /** * A low-overhead iterator which works on one window at a time, optimally applying columnar techniques * to compute each window as fast as possible on multiple rows at a time. - * - * TODO: we can add a sliding-window version of this as well. Assuming start2 < end1: - * - Calculate initial (first) window - (start1, end1) - * - Subtract non-overlapping portion of initial window start2 - start1 - * - Add next portion of window beyond overlap: end2 - end1 - * However, note that sliding window iterators have to do twice as much work, adding and removing, so it would - * only be worth it if the overlap is more than 50%. */ -class ChunkedWindowIterator(rv: RawDataRangeVector, - start: Long, - step: Long, - end: Long, - window: Long, - rangeFunction: ChunkedRangeFunction, - queryConfig: QueryConfig) extends Iterator[TransientRow] with StrictLogging { +abstract class ChunkedWindowIterator[R <: MutableRowReader]( + rv: RawDataRangeVector, + start: Long, + step: Long, + end: Long, + window: Long, + rangeFunction: ChunkedRangeFunction[R], + queryConfig: QueryConfig) +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.chunkInfos(start - window, end), start, step, end, window) + val it = new WindowedChunkIterator(rv, start, step, end, window) // 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. @@ -117,11 +117,10 @@ class ChunkedWindowIterator(rv: RawDataRangeVector, it } - private val sampleToEmit = new TransientRow() + def sampleToEmit: R override def hasNext: Boolean = windowIt.hasMoreWindows - - override def next: TransientRow = { + override def next: R = { rangeFunction.reset() // TODO: detect if rangeFunction needs items completely sorted. For example, it is possible // to do rate if only each chunk is sorted. Also check for counter correction here @@ -132,10 +131,11 @@ class ChunkedWindowIterator(rv: RawDataRangeVector, wit.nextWindow() while (wit.hasNext) { - val nextInfo = wit.nextInfo + val queryInfo = wit.next + val nextInfo = ChunkSetInfo(queryInfo.infoPtr) try { - rangeFunction.addChunks(rv.timestampColID, rv.valueColID, nextInfo, - wit.curWindowStart, wit.curWindowEnd, queryConfig) + rangeFunction.addChunks(queryInfo.tsVector, queryInfo.tsReader, queryInfo.valueVector, queryInfo.valueReader, + wit.curWindowStart, wit.curWindowEnd, nextInfo, queryConfig) } catch { case e: Exception => val timestampVector = nextInfo.vectorPtr(rv.timestampColID) @@ -158,6 +158,22 @@ class ChunkedWindowIterator(rv: RawDataRangeVector, } } +class ChunkedWindowIteratorD(rv: RawDataRangeVector, + start: Long, step: Long, end: Long, window: Long, + rangeFunction: ChunkedRangeFunction[TransientRow], + queryConfig: QueryConfig, + // put emitter here in constructor for faster access + var sampleToEmit: TransientRow = new TransientRow()) extends +ChunkedWindowIterator[TransientRow](rv, start, step, end, window, rangeFunction, queryConfig) + +class ChunkedWindowIteratorH(rv: RawDataRangeVector, + start: Long, step: Long, end: Long, window: Long, + rangeFunction: ChunkedRangeFunction[TransientHistRow], + queryConfig: QueryConfig, + // put emitter here in constructor for faster access + var sampleToEmit: TransientHistRow = new TransientHistRow()) extends +ChunkedWindowIterator[TransientHistRow](rv, start, step, end, window, rangeFunction, queryConfig) + class QueueBasedWindow(q: IndexedArrayQueue[TransientRow]) extends Window { def size: Int = q.size def apply(i: Int): TransientRow = q(i) diff --git a/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala b/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala index 2b0db8bdcc..6a410e572d 100644 --- a/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala +++ b/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala @@ -8,7 +8,8 @@ import filodb.core.query._ import filodb.memory.format.RowReader import filodb.query.{BinaryOperator, InstantFunctionId, QueryConfig} import filodb.query.exec.binaryOp.BinaryOperatorFunction -import filodb.query.exec.rangefn.InstantFunction +import filodb.query.exec.rangefn.{DoubleInstantFunction, HistToDoubleIFunction, InstantFunction} +import filodb.query.InstantFunctionId.HistogramQuantile /** * Implementations can provide ways to transform RangeVector @@ -56,31 +57,75 @@ final case class InstantVectorFunctionMapper(function: InstantFunctionId, protected[exec] def args: String = s"function=$function, funcParams=$funcParams" - val instantFunction = InstantFunction(function, funcParams) - def apply(source: Observable[RangeVector], queryConfig: QueryConfig, limit: Int, sourceSchema: ResultSchema): Observable[RangeVector] = { - source.map { rv => - val resultIterator: Iterator[RowReader] = new Iterator[RowReader]() { - - private val rows = rv.rows - private val result = new TransientRow() - - override def hasNext: Boolean = rows.hasNext - - override def next(): RowReader = { - val next = rows.next() - val newValue = instantFunction(next.getDouble(1)) - result.setValues(next.getLong(0), newValue) - result + RangeVectorTransformer.valueColumnType(sourceSchema) match { + case ColumnType.HistogramColumn => + val instantFunction = InstantFunction.histogram(function, funcParams) + if (instantFunction.isHToDoubleFunc) { + source.map { rv => + IteratorBackedRangeVector(rv.key, new H2DoubleInstantFuncIterator(rv.rows, instantFunction.asHToDouble)) + } + } else { + throw new UnsupportedOperationException(s"Sorry, function $function is not supported right now") } - } - IteratorBackedRangeVector(rv.key, resultIterator) + case ColumnType.DoubleColumn => + if (function == HistogramQuantile) { + // Special mapper to pull all buckets together from different Prom-schema time series + val mapper = HistogramQuantileMapper(funcParams) + mapper.apply(source, queryConfig, limit, sourceSchema) + } else { + val instantFunction = InstantFunction.double(function, funcParams) + source.map { rv => + IteratorBackedRangeVector(rv.key, new DoubleInstantFuncIterator(rv.rows, instantFunction)) + } + } + case cType: ColumnType => + throw new UnsupportedOperationException(s"Column type $cType is not supported for instant functions") + } + } + + override def schema(dataset: Dataset, source: ResultSchema): ResultSchema = { + // if source is histogram, determine what output column type is + // otherwise pass along the source + RangeVectorTransformer.valueColumnType(source) match { + case ColumnType.HistogramColumn => + val instantFunction = InstantFunction.histogram(function, funcParams) + if (instantFunction.isHToDoubleFunc) { + // Hist to Double function, so output schema is double + source.copy(columns = Seq(source.columns.head, + source.columns(1).copy(colType = ColumnType.DoubleColumn))) + } else { source } + case cType: ColumnType => + source } } +} + +private class DoubleInstantFuncIterator(rows: Iterator[RowReader], + instantFunction: DoubleInstantFunction, + result: TransientRow = new TransientRow()) extends Iterator[RowReader] { + final def hasNext: Boolean = rows.hasNext + final def next(): RowReader = { + val next = rows.next() + val newValue = instantFunction(next.getDouble(1)) + result.setValues(next.getLong(0), newValue) + result + } +} +private class H2DoubleInstantFuncIterator(rows: Iterator[RowReader], + instantFunction: HistToDoubleIFunction, + result: TransientRow = new TransientRow()) extends Iterator[RowReader] { + final def hasNext: Boolean = rows.hasNext + final def next(): RowReader = { + val next = rows.next() + val newValue = instantFunction(next.getHistogram(1)) + result.setValues(next.getLong(0), newValue) + result + } } /** diff --git a/query/src/main/scala/filodb/query/exec/TransientRow.scala b/query/src/main/scala/filodb/query/exec/TransientRow.scala index afb5022a7b..adf3bd7da5 100644 --- a/query/src/main/scala/filodb/query/exec/TransientRow.scala +++ b/query/src/main/scala/filodb/query/exec/TransientRow.scala @@ -1,7 +1,6 @@ package filodb.query.exec -import filodb.memory.format.{RowReader, ZeroCopyUTF8String} - +import filodb.memory.format.{vectors => bv, RowReader, ZeroCopyUTF8String} trait MutableRowReader extends RowReader { def setLong(columnNo: Int, value: Long): Unit @@ -56,6 +55,38 @@ final class TransientRow(var timestamp: Long, var value: Double) extends Mutable override def toString: String = s"TransientRow(t=$timestamp, v=$value)" } +final class TransientHistRow(var timestamp: Long = 0L, + var value: bv.Histogram = bv.Histogram.empty) extends MutableRowReader { + def setValues(ts: Long, hist: bv.Histogram): Unit = { + timestamp = ts + value = hist + } + + def setLong(columnNo: Int, valu: Long): Unit = + if (columnNo == 0) timestamp = valu + else throw new IllegalArgumentException() + + def setDouble(columnNo: Int, valu: Double): Unit = throw new IllegalArgumentException() + def setString(columnNo: Int, value: ZeroCopyUTF8String): Unit = throw new IllegalArgumentException() + def setBlob(columnNo: Int, base: Array[Byte], offset: Int, length: Int): Unit = throw new IllegalArgumentException() + + def notNull(columnNo: Int): Boolean = columnNo < 2 + def getBoolean(columnNo: Int): Boolean = throw new IllegalArgumentException() + def getInt(columnNo: Int): Int = throw new IllegalArgumentException() + def getLong(columnNo: Int): Long = if (columnNo == 0) timestamp else throw new IllegalArgumentException() + def getDouble(columnNo: Int): Double = throw new IllegalArgumentException() + def getFloat(columnNo: Int): Float = throw new IllegalArgumentException() + def getString(columnNo: Int): String = throw new IllegalArgumentException() + override def getHistogram(col: Int): bv.Histogram = if (col == 1) value else throw new IllegalArgumentException() + def getAny(columnNo: Int): Any = throw new IllegalArgumentException() + + def getBlobBase(columnNo: Int): Any = throw new IllegalArgumentException() + def getBlobOffset(columnNo: Int): Long = throw new IllegalArgumentException() + def getBlobNumBytes(columnNo: Int): Int = throw new IllegalArgumentException() + + override def toString: String = s"TransientRow(t=$timestamp, v=$value)" +} + final class AvgAggTransientRow extends MutableRowReader { var timestamp: Long = _ var avg: Double = _ 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 1b9c924daa..27750e67a9 100644 --- a/query/src/main/scala/filodb/query/exec/rangefn/AggrOverTimeFunctions.scala +++ b/query/src/main/scala/filodb/query/exec/rangefn/AggrOverTimeFunctions.scala @@ -4,9 +4,10 @@ import java.lang.{Double => JLDouble} import java.util import filodb.core.store.ChunkSetInfo -import filodb.memory.format.{vectors => bv, BinaryVector} +import filodb.memory.format.{vectors => bv, BinaryVector, VectorDataReader} +import filodb.memory.format.BinaryVector.BinaryVectorPtr import filodb.query.QueryConfig -import filodb.query.exec.TransientRow +import filodb.query.exec.{TransientHistRow, TransientRow} class MinMaxOverTimeFunction(ord: Ordering[Double]) extends RangeFunction { val minMaxDeque = new util.ArrayDeque[TransientRow]() @@ -133,7 +134,7 @@ class SumOverTimeFunction(var sum: Double = Double.NaN, var count: Int = 0) exte } } -abstract class SumOverTimeChunkedFunction(var sum: Double = Double.NaN )extends ChunkedRangeFunction { +abstract class SumOverTimeChunkedFunction(var sum: Double = Double.NaN) extends ChunkedRangeFunction[TransientRow] { override final def reset(): Unit = { sum = Double.NaN } final def apply(endTimestamp: Long, sampleToEmit: TransientRow): Unit = { sampleToEmit.setValues(endTimestamp, sum) @@ -165,6 +166,28 @@ class SumOverTimeChunkedFunctionL extends SumOverTimeChunkedFunction() with Chun } } +class SumOverTimeChunkedFunctionH(var h: bv.Histogram = bv.Histogram.empty) +extends ChunkedRangeFunction[TransientHistRow] { + override final def reset(): Unit = { h = bv.Histogram.empty } + final def apply(endTimestamp: Long, sampleToEmit: TransientHistRow): Unit = { + sampleToEmit.setValues(endTimestamp, h) + } + + final def addChunks(tsVector: BinaryVectorPtr, tsReader: bv.LongVectorDataReader, + valueVector: BinaryVectorPtr, valueReader: VectorDataReader, + startTime: Long, endTime: Long, info: ChunkSetInfo, queryConfig: QueryConfig): Unit = { + val startRowNum = tsReader.binarySearch(tsVector, startTime) & 0x7fffffff + val endRowNum = Math.min(tsReader.ceilingIndex(tsVector, endTime), info.numRows - 1) + + val sum = valueReader.asHistReader.sum(startRowNum, endRowNum) + h match { + // sum is mutable histogram, copy to be sure it's our own copy + case hist if hist.numBuckets == 0 => h = sum.copy + case hist: bv.MutableHistogram => hist.add(sum) + } + } +} + class CountOverTimeFunction(var count: Double = Double.NaN) extends RangeFunction { override def addedToWindow(row: TransientRow, window: Window): Unit = { if (!JLDouble.isNaN(row.value)) { @@ -194,19 +217,17 @@ class CountOverTimeFunction(var count: Double = Double.NaN) extends RangeFunctio } } -class CountOverTimeChunkedFunction(var count: Int = 0) extends ChunkedRangeFunction { +class CountOverTimeChunkedFunction(var count: Int = 0) extends ChunkedRangeFunction[TransientRow] { override final def reset(): Unit = { count = 0 } final def apply(endTimestamp: Long, sampleToEmit: TransientRow): Unit = { sampleToEmit.setValues(endTimestamp, count.toDouble) } - final def addChunks(tsCol: Int, valueCol: Int, info: ChunkSetInfo, - startTime: Long, endTime: Long, queryConfig: QueryConfig): Unit = { - val timestampVector = info.vectorPtr(tsCol) - val tsReader = bv.LongBinaryVector(timestampVector) - + final def addChunks(tsVector: BinaryVectorPtr, tsReader: bv.LongVectorDataReader, + valueVector: BinaryVectorPtr, valueReader: VectorDataReader, + startTime: Long, endTime: Long, info: ChunkSetInfo, queryConfig: QueryConfig): Unit = { // First row >= startTime, so we can just drop bit 31 (dont care if it matches exactly) - val startRowNum = tsReader.binarySearch(timestampVector, startTime) & 0x7fffffff - val endRowNum = tsReader.ceilingIndex(timestampVector, endTime) + val startRowNum = tsReader.binarySearch(tsVector, startTime) & 0x7fffffff + val endRowNum = tsReader.ceilingIndex(tsVector, endTime) val numRows = Math.min(endRowNum, info.numRows - 1) - startRowNum + 1 count += numRows } @@ -260,7 +281,7 @@ class AvgOverTimeFunction(var sum: Double = Double.NaN, var count: Int = 0) exte } abstract class AvgOverTimeChunkedFunction(var sum: Double = Double.NaN, var count: Double = 0) - extends ChunkedRangeFunction { + extends ChunkedRangeFunction[TransientRow] { override final def reset(): Unit = { sum = Double.NaN; count = 0d diff --git a/query/src/main/scala/filodb/query/exec/rangefn/InstantFunction.scala b/query/src/main/scala/filodb/query/exec/rangefn/InstantFunction.scala index 24d6e471c6..40ce24889c 100644 --- a/query/src/main/scala/filodb/query/exec/rangefn/InstantFunction.scala +++ b/query/src/main/scala/filodb/query/exec/rangefn/InstantFunction.scala @@ -1,14 +1,13 @@ package filodb.query.exec.rangefn +import filodb.memory.format.vectors.Histogram import filodb.query.InstantFunctionId import filodb.query.InstantFunctionId.{Log2, Sqrt, _} /** - * All Instant Functions are implementation of this trait. - * Pass the sample value to `apply` and get the instant function value. + * Applies a function transforming a single value into another value, both of type Double. */ trait DoubleInstantFunction { - /** * Apply the required instant function against the given value. * @@ -16,30 +15,58 @@ trait DoubleInstantFunction { * @return Calculated value */ def apply(value: Double): Double - } trait EmptyParamsInstantFunction extends DoubleInstantFunction { - def funcParams: Seq[Any] /** * Validate the function before invoking the function. */ require(funcParams.isEmpty, "No additional parameters required for the instant function.") +} +sealed trait HistogramInstantFunction { + def isHToDoubleFunc: Boolean = this.isInstanceOf[HistToDoubleIFunction] + def asHToDouble: HistToDoubleIFunction = this.asInstanceOf[HistToDoubleIFunction] + def asHToH: HistToHistIFunction = this.asInstanceOf[HistToHistIFunction] } -object InstantFunction { +/** + * An instant function taking a histogram and returning a Double value + */ +trait HistToDoubleIFunction extends HistogramInstantFunction { + /** + * Apply the required instant function against the given value. + * + * @param value Sample against which the function will be applied + * @return Calculated value + */ + def apply(value: Histogram): Double +} +/** + * An instant function taking a histogram and returning another histogram + */ +trait HistToHistIFunction extends HistogramInstantFunction { + /** + * Apply the required instant function against the given value. + * + * @param value Sample against which the function will be applied + * @return Calculated value + */ + def apply(value: Histogram): Histogram +} + +object InstantFunction { /** - * This function returns a function that can be applied to generate the result. + * Returns the DoubleInstantFunction given the function ID and parameters. * * @param function to be invoked * @param funcParams - Additional required function parameters * @return the function */ - def apply(function: InstantFunctionId, funcParams: Seq[Any]): DoubleInstantFunction = { + def double(function: InstantFunctionId, funcParams: Seq[Any]): DoubleInstantFunction = { function match { case Abs => AbsImpl(funcParams) case Ceil => CeilImpl(funcParams) @@ -55,6 +82,14 @@ object InstantFunction { case _ => throw new UnsupportedOperationException(s"$function not supported.") } } + + /** + * Returns the HistogramInstantFunction given the function ID and parameters + */ + def histogram(function: InstantFunctionId, funcParams: Seq[Any]): HistogramInstantFunction = function match { + case HistogramQuantile => HistogramQuantileImpl(funcParams) + case _ => throw new UnsupportedOperationException(s"$function not supported.") + } } /** @@ -63,9 +98,7 @@ object InstantFunction { * @param funcParams - Additional function parameters */ case class AbsImpl(funcParams: Seq[Any]) extends EmptyParamsInstantFunction { - override def apply(value: Double): Double = scala.math.abs(value) - } /** @@ -74,9 +107,7 @@ case class AbsImpl(funcParams: Seq[Any]) extends EmptyParamsInstantFunction { * @param funcParams - Additional function parameters */ case class CeilImpl(funcParams: Seq[Any]) extends EmptyParamsInstantFunction { - override def apply(value: Double): Double = scala.math.ceil(value) - } /** @@ -85,7 +116,6 @@ case class CeilImpl(funcParams: Seq[Any]) extends EmptyParamsInstantFunction { * @param funcParams - Additional function parameters */ case class ClampMaxImpl(funcParams: Seq[Any]) extends DoubleInstantFunction { - /** * Validate the function before invoking the function. */ @@ -96,7 +126,6 @@ case class ClampMaxImpl(funcParams: Seq[Any]) extends DoubleInstantFunction { override def apply(value: Double): Double = scala.math.min(value, funcParams.head.asInstanceOf[Number].doubleValue()) - } /** @@ -105,7 +134,6 @@ case class ClampMaxImpl(funcParams: Seq[Any]) extends DoubleInstantFunction { * @param funcParams - Additional function parameters */ case class ClampMinImpl(funcParams: Seq[Any]) extends DoubleInstantFunction { - /** * Validate the function before invoking the function. */ @@ -116,7 +144,6 @@ case class ClampMinImpl(funcParams: Seq[Any]) extends DoubleInstantFunction { override def apply(value: Double): Double = scala.math.max(value, funcParams.head.asInstanceOf[Number].doubleValue()) - } /** @@ -125,9 +152,7 @@ case class ClampMinImpl(funcParams: Seq[Any]) extends DoubleInstantFunction { * @param funcParams - Additional function parameters */ case class ExpImpl(funcParams: Seq[Any]) extends EmptyParamsInstantFunction { - override def apply(value: Double): Double = scala.math.exp(value) - } /** @@ -136,9 +161,7 @@ case class ExpImpl(funcParams: Seq[Any]) extends EmptyParamsInstantFunction { * @param funcParams - Additional function parameters */ case class FloorImpl(funcParams: Seq[Any]) extends EmptyParamsInstantFunction { - override def apply(value: Double): Double = scala.math.floor(value) - } /** @@ -147,9 +170,7 @@ case class FloorImpl(funcParams: Seq[Any]) extends EmptyParamsInstantFunction { * @param funcParams - Additional function parameters */ case class LnImpl(funcParams: Seq[Any]) extends EmptyParamsInstantFunction { - override def apply(value: Double): Double = scala.math.log(value) - } /** @@ -158,14 +179,12 @@ case class LnImpl(funcParams: Seq[Any]) extends EmptyParamsInstantFunction { * @param funcParams - Additional function parameters */ case class Log10Impl(funcParams: Seq[Any]) extends EmptyParamsInstantFunction { - /** * Validate the function before invoking the function. */ require(funcParams.isEmpty, "No additional parameters required for Log10.") override def apply(value: Double): Double = scala.math.log10(value) - } /** @@ -174,10 +193,8 @@ case class Log10Impl(funcParams: Seq[Any]) extends EmptyParamsInstantFunction { * @param funcParams - Additional function parameters */ case class Log2Impl(funcParams: Seq[Any]) extends EmptyParamsInstantFunction { - override def apply(value: Double): Double = scala.math.log10(value)/scala.math.log10(2.0) - } /** @@ -189,7 +206,6 @@ case class Log2Impl(funcParams: Seq[Any]) extends EmptyParamsInstantFunction { * @param funcParams - Additional function parameters */ case class RoundImpl(funcParams: Seq[Any]) extends DoubleInstantFunction { - /** * Validate the function before invoking the function. */ @@ -212,7 +228,6 @@ case class RoundImpl(funcParams: Seq[Any]) extends DoubleInstantFunction { else scala.math.floor(value * toNearestInverse + 0.5) / toNearestInverse } - } /** @@ -220,7 +235,17 @@ case class RoundImpl(funcParams: Seq[Any]) extends DoubleInstantFunction { * @param funcParams - Additional function parameters */ case class SqrtImpl(funcParams: Seq[Any]) extends EmptyParamsInstantFunction { - override def apply(value: Double): Double = scala.math.sqrt(value) +} +/** + * Histogram quantile function for Histogram columns, where all buckets are together. + * @param funcParams - a single value between 0 and 1, the quantile to calculate. + */ +case class HistogramQuantileImpl(funcParams: Seq[Any]) extends HistToDoubleIFunction { + require(funcParams.length == 1, "Quantile (between 0 and 1) required for histogram quantile") + require(funcParams(0).isInstanceOf[Number], "histogram_quantile parameter must be a number") + val q = funcParams(0).asInstanceOf[Number].doubleValue() + + final def apply(value: Histogram): Double = value.quantile(q) } \ 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 972b620b03..a767c35ea4 100644 --- a/query/src/main/scala/filodb/query/exec/rangefn/RangeFunction.scala +++ b/query/src/main/scala/filodb/query/exec/rangefn/RangeFunction.scala @@ -2,7 +2,8 @@ package filodb.query.exec.rangefn import filodb.core.metadata.Column.ColumnType import filodb.core.store.ChunkSetInfo -import filodb.memory.format.{vectors => bv, BinaryVector} +import filodb.memory.format.{vectors => bv, BinaryVector, VectorDataReader} +import filodb.memory.format.BinaryVector.BinaryVectorPtr import filodb.query.{QueryConfig, RangeFunctionId} import filodb.query.exec._ import filodb.query.RangeFunctionId._ @@ -18,17 +19,22 @@ trait Window { def last: TransientRow } +// Just a marker trait for all RangeFunction implementations, sliding and chunked +sealed trait BaseRangeFunction { + def asSliding: RangeFunction = this.asInstanceOf[RangeFunction] + def asChunkedD: ChunkedRangeFunction[TransientRow] = this.asInstanceOf[ChunkedRangeFunction[TransientRow]] + def asChunkedH: ChunkedRangeFunction[TransientHistRow] = this.asInstanceOf[ChunkedRangeFunction[TransientHistRow]] +} + /** - * All Range Vector Functions are implementation of this trait. + * Double-based Range Vector Functions that work with the SlidingWindowIterator. * There are multiple choices for function implementation: * 1. Use the `addToWindow` and `removeFromWindow` events to evaluate the next value to emit. * This may result in O(n) complexity for emitting the entire range vector. * 2. Use the entire window content in `apply` to emit the next value. Depending on whether the * entire window is examined, this may result in O(n) or O(n-squared) for the entire range vector. - * 3. Use the addChunks() API of the ChunkedRangeFunction subtrait for more efficient proessing of windows - * in chunks of rows (rather than one at a time) */ -trait RangeFunction { +trait RangeFunction extends BaseRangeFunction { /** * Needs last sample prior to window start */ @@ -73,17 +79,10 @@ trait RangeFunction { } /** - * Improved RangeFunction API with direct chunk access for faster columnar/bulk operations + * Improved RangeFunction API with direct chunk access for faster columnar/bulk operations. + * It also allows for different MutableRows to be plugged in for different types. */ -trait ChunkedRangeFunction extends RangeFunction { - def addedToWindow(row: TransientRow, window: Window): Unit = {} - - def removedFromWindow(row: TransientRow, window: Window): Unit = {} - - def apply(startTimestamp: Long, endTimestamp: Long, window: Window, - sampleToEmit: TransientRow, - queryConfig: QueryConfig): Unit = {} - +trait ChunkedRangeFunction[R <: MutableRowReader] extends BaseRangeFunction { /** * Resets the state */ @@ -92,40 +91,40 @@ trait ChunkedRangeFunction extends RangeFunction { /** * Called by the ChunkedWindowIterator to add multiple rows at a time to the range function for efficiency. * The idea is to call chunk-based methods such as sum and binarySearch. - * @param tsCol ColumnID for timestamp column - * @param valueCol ColumnID for value column - * @param info ChunkSetInfo with information for specific chunks - * @param startTime starting timestamp in millis since Epoch for time window + * @param tsVector raw pointer to the timestamp BinaryVector + * @param tsReader a LongVectorDataReader for parsing the tsVector + * @param valueVector a raw pointer to the value BinaryVector + * @param valueReader a VectorDataReader for the value BinaryVector. Method must cast to appropriate type. + * @param startTime starting timestamp in millis since Epoch for time window, inclusive + * @param endTime ending timestamp in millis since Epoch for time window, inclusive */ - def addChunks(tsCol: Int, valueCol: Int, info: ChunkSetInfo, - startTime: Long, endTime: Long, queryConfig: QueryConfig): Unit + def addChunks(tsVector: BinaryVectorPtr, tsReader: bv.LongVectorDataReader, + valueVector: BinaryVectorPtr, valueReader: VectorDataReader, + startTime: Long, endTime: Long, info: ChunkSetInfo, queryConfig: QueryConfig): Unit /** * Return the computed result in the sampleToEmit */ - def apply(endTimestamp: Long, sampleToEmit: TransientRow): Unit + def apply(endTimestamp: Long, sampleToEmit: R): Unit } /** * Standard ChunkedRangeFunction implementation extracting the start and ending row numbers from the timestamp * and returning the double value vector and reader with the row numbers */ -trait ChunkedDoubleRangeFunction extends ChunkedRangeFunction { - def addChunks(tsCol: Int, valueCol: Int, info: ChunkSetInfo, - startTime: Long, endTime: Long, queryConfig: QueryConfig): Unit = { - val timestampVector = info.vectorPtr(tsCol) - val tsReader = bv.LongBinaryVector(timestampVector) - val doubleVector = info.vectorPtr(valueCol) - val dblReader = bv.DoubleVector(doubleVector) - +trait ChunkedDoubleRangeFunction extends ChunkedRangeFunction[TransientRow] { + final def addChunks(tsVector: BinaryVectorPtr, tsReader: bv.LongVectorDataReader, + valueVector: BinaryVectorPtr, valueReader: VectorDataReader, + startTime: Long, endTime: Long, info: ChunkSetInfo, queryConfig: QueryConfig): Unit = { // TODO: abstract this pattern of start/end row # out. Probably when cursors are implemented // First row >= startTime, so we can just drop bit 31 (dont care if it matches exactly) - val startRowNum = tsReader.binarySearch(timestampVector, startTime) & 0x7fffffff - val endRowNum = tsReader.ceilingIndex(timestampVector, endTime) + val startRowNum = tsReader.binarySearch(tsVector, startTime) & 0x7fffffff + val endRowNum = tsReader.ceilingIndex(tsVector, endTime) // At least one sample is present if (startRowNum <= endRowNum) { - addTimeDoubleChunks(doubleVector, dblReader, startRowNum, Math.min(endRowNum, info.numRows - 1)) + addTimeDoubleChunks(valueVector, valueReader.asDoubleReader, + startRowNum, Math.min(endRowNum, info.numRows - 1)) } } @@ -140,21 +139,18 @@ trait ChunkedDoubleRangeFunction extends ChunkedRangeFunction { endRowNum: Int): Unit } -trait ChunkedLongRangeFunction extends ChunkedRangeFunction { - def addChunks(tsCol: Int, valueCol: Int, info: ChunkSetInfo, - startTime: Long, endTime: Long, queryConfig: QueryConfig): Unit = { - val timestampVector = info.vectorPtr(tsCol) - val tsReader = bv.LongBinaryVector(timestampVector) - val longVector = info.vectorPtr(valueCol) - val longReader = bv.LongBinaryVector(longVector) - +trait ChunkedLongRangeFunction extends ChunkedRangeFunction[TransientRow] { + final def addChunks(tsVector: BinaryVectorPtr, tsReader: bv.LongVectorDataReader, + valueVector: BinaryVectorPtr, valueReader: VectorDataReader, + startTime: Long, endTime: Long, info: ChunkSetInfo, queryConfig: QueryConfig): Unit = { // TODO: abstract this pattern of start/end row # out. Probably when cursors are implemented // First row >= startTime, so we can just drop bit 31 (dont care if it matches exactly) - val startRowNum = tsReader.binarySearch(timestampVector, startTime) & 0x7fffffff - val endRowNum = tsReader.ceilingIndex(timestampVector, endTime) + val startRowNum = tsReader.binarySearch(tsVector, startTime) & 0x7fffffff + val endRowNum = tsReader.ceilingIndex(tsVector, endTime) if (startRowNum <= endRowNum) { - addTimeLongChunks(longVector, longReader, startRowNum, Math.min(endRowNum, info.numRows - 1)) + addTimeLongChunks(valueVector, valueReader.asLongReader, + startRowNum, Math.min(endRowNum, info.numRows - 1)) } } @@ -170,7 +166,7 @@ trait ChunkedLongRangeFunction extends ChunkedRangeFunction { } object RangeFunction { - type RangeFunctionGenerator = () => RangeFunction + type RangeFunctionGenerator = () => BaseRangeFunction /** * Returns a (probably new) instance of RangeFunction given the func ID and column type @@ -178,7 +174,7 @@ object RangeFunction { def apply(func: Option[RangeFunctionId], columnType: ColumnType, funcParams: Seq[Any] = Nil, - useChunked: Boolean): RangeFunction = + useChunked: Boolean): BaseRangeFunction = generatorFor(func, columnType, funcParams, useChunked)() /** @@ -192,6 +188,7 @@ object RangeFunction { case ColumnType.DoubleColumn => doubleChunkedFunction(func, funcParams) case ColumnType.LongColumn => longChunkedFunction(func, funcParams) case ColumnType.TimestampColumn => longChunkedFunction(func, funcParams) + case ColumnType.HistogramColumn => histChunkedFunction(func, funcParams) case other: ColumnType => throw new IllegalArgumentException(s"Column type $other not supported") } else { iteratingFunction(func, funcParams) @@ -229,6 +226,13 @@ object RangeFunction { case _ => iteratingFunction(func, funcParams) } + def histChunkedFunction(func: Option[RangeFunctionId], + funcParams: Seq[Any] = Nil): RangeFunctionGenerator = func match { + case None => () => new LastSampleChunkedFunctionH + case Some(SumOverTime) => () => new SumOverTimeChunkedFunctionH + case _ => ??? + } + /** * Returns a function to generate the RangeFunction for SlidingWindowIterator. * Note that these functions are Double-based, so a converting iterator eg LongToDoubleIterator may be needed. @@ -278,38 +282,55 @@ object LastSampleFunction extends RangeFunction { * Directly obtain the last sample from chunks for much much faster performance compared to above * TODO: one day we should expose higher level means of returning the actual Long value instead of converting to Double */ -abstract class LastSampleChunkedFunction(var timestamp: Long = -1L, - var value: Double = Double.NaN) extends ChunkedRangeFunction { - override final def reset(): Unit = { timestamp = -1L; value = Double.NaN } - final def apply(endTimestamp: Long, sampleToEmit: TransientRow): Unit = { - sampleToEmit.setValues(endTimestamp, value) - } +abstract class LastSampleChunkedFunction[R <: MutableRowReader](var timestamp: Long = -1L) +extends ChunkedRangeFunction[R] { // Add each chunk and update timestamp and value such that latest sample wins - final def addChunks(tsCol: Int, valueCol: Int, info: ChunkSetInfo, - startTime: Long, endTime: Long, queryConfig: QueryConfig): Unit = { - val timestampVector = info.vectorPtr(tsCol) - val tsReader = bv.LongBinaryVector(timestampVector) + final def addChunks(tsVector: BinaryVectorPtr, tsReader: bv.LongVectorDataReader, + valueVector: BinaryVectorPtr, valueReader: VectorDataReader, + startTime: Long, endTime: Long, info: ChunkSetInfo, queryConfig: QueryConfig): Unit = { // Just in case timestamp vectors are a bit longer than others. - val endRowNum = Math.min(tsReader.ceilingIndex(timestampVector, endTime), info.numRows - 1) + val endRowNum = Math.min(tsReader.ceilingIndex(tsVector, endTime), info.numRows - 1) // update timestamp only if // 1) endRowNum >= 0 (timestamp within chunk) // 2) timestamp is within stale window; AND // 3) timestamp is greater than current timestamp (for multiple chunk scenarios) if (endRowNum >= 0) { - val ts = tsReader(timestampVector, endRowNum) + val ts = tsReader(tsVector, endRowNum) if ((endTime - ts) <= queryConfig.staleSampleAfterMs && ts > timestamp) - updateValue(ts, info.vectorPtr(valueCol), endRowNum) + updateValue(ts, valueVector, valueReader, endRowNum) } } - def updateValue(ts: Long, dataVector: BinaryVector.BinaryVectorPtr, endRowNum: Int): Unit + def updateValue(ts: Long, valVector: BinaryVectorPtr, valReader: VectorDataReader, endRowNum: Int): Unit +} + +// LastSample functions with double value, based on TransientRow +abstract class LastSampleChunkedFuncDblVal(var value: Double = Double.NaN) +extends LastSampleChunkedFunction[TransientRow] { + override final def reset(): Unit = { timestamp = -1L; value = Double.NaN } + final def apply(endTimestamp: Long, sampleToEmit: TransientRow): Unit = { + sampleToEmit.setValues(endTimestamp, value) + } +} + +// LastSample function for Histogram columns +class LastSampleChunkedFunctionH(var value: bv.Histogram = bv.Histogram.empty) +extends LastSampleChunkedFunction[TransientHistRow] { + override final def reset(): Unit = { timestamp = -1L; value = bv.Histogram.empty } + final def apply(endTimestamp: Long, sampleToEmit: TransientHistRow): Unit = { + sampleToEmit.setValues(endTimestamp, value) + } + final def updateValue(ts: Long, valVector: BinaryVectorPtr, valReader: VectorDataReader, endRowNum: Int): Unit = { + timestamp = ts + value = valReader.asHistReader(endRowNum) + } } -class LastSampleChunkedFunctionD extends LastSampleChunkedFunction() { - final def updateValue(ts: Long, dataVector: BinaryVector.BinaryVectorPtr, endRowNum: Int): Unit = { - val dblReader = bv.DoubleVector(dataVector) - val doubleVal = dblReader(dataVector, endRowNum) +class LastSampleChunkedFunctionD extends LastSampleChunkedFuncDblVal() { + final def updateValue(ts: Long, valVector: BinaryVectorPtr, valReader: VectorDataReader, endRowNum: Int): Unit = { + val dblReader = valReader.asDoubleReader + val doubleVal = dblReader(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 @@ -317,7 +338,7 @@ class LastSampleChunkedFunctionD extends LastSampleChunkedFunction() { if (java.lang.Double.isNaN(doubleVal)) { if (endRowNum > 0) { timestamp = ts - value = dblReader(dataVector, endRowNum - 1) + value = dblReader(valVector, endRowNum - 1) } } else { timestamp = ts @@ -326,10 +347,10 @@ class LastSampleChunkedFunctionD extends LastSampleChunkedFunction() { } } -class LastSampleChunkedFunctionL extends LastSampleChunkedFunction() { - final def updateValue(ts: Long, dataVector: BinaryVector.BinaryVectorPtr, endRowNum: Int): Unit = { - val longReader = bv.LongBinaryVector(dataVector) +class LastSampleChunkedFunctionL extends LastSampleChunkedFuncDblVal() { + final def updateValue(ts: Long, valVector: BinaryVectorPtr, valReader: VectorDataReader, endRowNum: Int): Unit = { + val longReader = valReader.asLongReader timestamp = ts - value = longReader(dataVector, endRowNum).toDouble + value = longReader(valVector, endRowNum).toDouble } } diff --git a/query/src/test/scala/filodb/query/exec/LastSampleFunctionSpec.scala b/query/src/test/scala/filodb/query/exec/LastSampleFunctionSpec.scala index aac8f1da8f..cf89685df7 100644 --- a/query/src/test/scala/filodb/query/exec/LastSampleFunctionSpec.scala +++ b/query/src/test/scala/filodb/query/exec/LastSampleFunctionSpec.scala @@ -27,7 +27,7 @@ class LastSampleFunctionSpec extends RawDataWindowingSpec { val lastSamplesIter = new SlidingWindowIterator(rv.rows, start, step, end, 0, LastSampleFunction, queryConfig) validateLastSamples(samples, lastSamplesIter, start, end, step) - val chunkedIter = new ChunkedWindowIterator(rv, start, step, end, w, chunkedLSFunc, queryConfig) + val chunkedIter = new ChunkedWindowIteratorD(rv, start, step, end, w, chunkedLSFunc, queryConfig) validateLastSamples(samples, chunkedIter, start, end, step) } } @@ -39,7 +39,7 @@ class LastSampleFunctionSpec extends RawDataWindowingSpec { val lastSamplesIter = new SlidingWindowIterator(rv.rows, start, step, end, 0, LastSampleFunction, queryConfig) validateLastSamples(samples, lastSamplesIter, start, end, step) - val chunkedIter = new ChunkedWindowIterator(rv, start, step, end, w, chunkedLSFunc, queryConfig) + val chunkedIter = new ChunkedWindowIteratorD(rv, start, step, end, w, chunkedLSFunc, queryConfig) validateLastSamples(samples, chunkedIter, start, end, step) } } @@ -52,7 +52,7 @@ class LastSampleFunctionSpec extends RawDataWindowingSpec { val lastSamplesIter = new SlidingWindowIterator(rv.rows, start, step, end, 0, LastSampleFunction, queryConfig) validateLastSamples(samples, lastSamplesIter, start, end, step) - val chunkedIter = new ChunkedWindowIterator(rv, start, step, end, w, chunkedLSFunc, queryConfig) + val chunkedIter = new ChunkedWindowIteratorD(rv, start, step, end, w, chunkedLSFunc, queryConfig) validateLastSamples(samples, chunkedIter, start, end, step) } } @@ -69,7 +69,7 @@ class LastSampleFunctionSpec extends RawDataWindowingSpec { start, step, end, 0, LastSampleFunction, queryConfig) validateLastSamples(samplesWithLongGap, lastSamplesIter, start, end, step) - val chunkedIter = new ChunkedWindowIterator(rvWithLongGap, start, step, end, w, chunkedLSFunc, queryConfig) + val chunkedIter = new ChunkedWindowIteratorD(rvWithLongGap, start, step, end, w, chunkedLSFunc, queryConfig) validateLastSamples(samplesWithLongGap, chunkedIter, start, end, step) } @@ -84,7 +84,7 @@ class LastSampleFunctionSpec extends RawDataWindowingSpec { start, step, end, 0, LastSampleFunction, queryConfig) validateLastSamples(samplesWithLongGap, lastSamplesIter, start, end, step) - val chunkedIter = new ChunkedWindowIterator(rvWithLongGap, start, step, end, w, chunkedLSFunc, queryConfig) + val chunkedIter = new ChunkedWindowIteratorD(rvWithLongGap, start, step, end, w, chunkedLSFunc, queryConfig) validateLastSamples(samplesWithLongGap, chunkedIter, start, end, step) } @@ -99,7 +99,7 @@ class LastSampleFunctionSpec extends RawDataWindowingSpec { start, step, end, 0, LastSampleFunction, queryConfig) validateLastSamples(samplesWithLongGap, lastSamplesIter, start, end, step) - val chunkedIter = new ChunkedWindowIterator(rvWithLongGap, start, step, end, w, chunkedLSFunc, queryConfig) + val chunkedIter = new ChunkedWindowIteratorD(rvWithLongGap, start, step, end, w, chunkedLSFunc, queryConfig) validateLastSamples(samplesWithLongGap, chunkedIter, start, end, step) } diff --git a/query/src/test/scala/filodb/query/exec/SelectRawPartitionsExecSpec.scala b/query/src/test/scala/filodb/query/exec/SelectRawPartitionsExecSpec.scala index cdd87d3630..a4148ed647 100644 --- a/query/src/test/scala/filodb/query/exec/SelectRawPartitionsExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/SelectRawPartitionsExecSpec.scala @@ -11,11 +11,11 @@ import org.scalatest.concurrent.ScalaFutures import org.scalatest.time.{Millis, Seconds, Span} import filodb.core.MetricsTestData._ -import filodb.core.TestData +import filodb.core.{TestData, Types} import filodb.core.binaryrecord.BinaryRecord import filodb.core.binaryrecord2.RecordBuilder import filodb.core.memstore.{FixedMaxPartitionsEvictionPolicy, SomeData, TimeSeriesMemStore} -import filodb.core.metadata.Column.ColumnType.{DoubleColumn, TimestampColumn} +import filodb.core.metadata.Column.ColumnType.{DoubleColumn, HistogramColumn, TimestampColumn} import filodb.core.query.{ColumnFilter, Filter} import filodb.core.store.{InMemoryMetaStore, NullColumnStore} import filodb.memory.MemFactory @@ -51,6 +51,7 @@ class SelectRawPartitionsExecSpec extends FunSpec with Matchers with ScalaFuture val mmdBuilder = new RecordBuilder(MemFactory.onHeapFactory, MMD.dataset1.ingestionSchema) val mmdTuples = MMD.linearMultiSeries().take(100) val mmdSomeData = MMD.records(MMD.dataset1, mmdTuples) + val histData = MMD.linearHistSeries().take(100) implicit val execTimeout = 5.seconds @@ -59,8 +60,11 @@ class SelectRawPartitionsExecSpec extends FunSpec with Matchers with ScalaFuture memStore.ingest(timeseriesDataset.ref, 0, SomeData(container, 0)) memStore.setup(MMD.dataset1, 0, TestData.storeConf) memStore.ingest(MMD.dataset1.ref, 0, mmdSomeData) + memStore.setup(MMD.histDataset, 0, TestData.storeConf) + memStore.ingest(MMD.histDataset.ref, 0, MMD.records(MMD.histDataset, histData)) memStore.commitIndexForTesting(timeseriesDataset.ref) memStore.commitIndexForTesting(MMD.dataset1.ref) + memStore.commitIndexForTesting(MMD.histDataset.ref) } override def afterAll(): Unit = { @@ -126,6 +130,24 @@ class SelectRawPartitionsExecSpec extends FunSpec with Matchers with ScalaFuture dataRead shouldEqual mmdTuples.filter(_(5) == "Series 1").map(r => (r(0), r(4))).take(5) } + it ("should read raw Histogram samples from Memstore using IntervalSelector") { + import ZeroCopyUTF8String._ + val start: BinaryRecord = BinaryRecord(MMD.histDataset, Seq(100000L)) + val end: BinaryRecord = BinaryRecord(MMD.histDataset, Seq(150000L)) + + val filters = Seq(ColumnFilter("dc", Filter.Equals("0".utf8))) + val execPlan = SelectRawPartitionsExec("id1", now, numRawSamples, dummyDispatcher, MMD.histDataset.ref, 0, + filters, RowKeyInterval(start, end), Seq(0, 3)) + + val resp = execPlan.execute(memStore, MMD.histDataset, queryConfig).runAsync.futureValue + val result = resp.asInstanceOf[QueryResult] + result.resultSchema.columns.map(_.colType) shouldEqual Seq(TimestampColumn, HistogramColumn) + result.result.size shouldEqual 1 + val resultIt = result.result(0).rows.map(r=>(r.getLong(0), r.getHistogram(1))) + val orig = histData.filter(_(4).asInstanceOf[Types.UTF8Map]("dc".utf8) == "0".utf8).map(r => (r(0), r(3))).take(5) + resultIt.zip(orig.toIterator).foreach { case (res, origData) => res shouldEqual origData } + } + it ("should read periodic samples from Memstore") { import ZeroCopyUTF8String._ val filters = Seq (ColumnFilter("__name__", Filter.Equals("http_req_total".utf8)), @@ -176,7 +198,28 @@ class SelectRawPartitionsExecSpec extends FunSpec with Matchers with ScalaFuture val dataRead = result.result(0).rows.map(r=>(r.getLong(0), r.getDouble(1))).toList dataRead.map(_._1) shouldEqual (start to end by step) dataRead.map(_._2) shouldEqual (86 to 166).by(20) + } + it ("should read periodic Histogram samples from Memstore") { + import ZeroCopyUTF8String._ + val filters = Seq(ColumnFilter("dc", Filter.Equals("0".utf8))) + val execPlan = SelectRawPartitionsExec("id1", now, numRawSamples, dummyDispatcher, MMD.histDataset.ref, 0, + filters, AllChunks, Seq(0, 3)) + + val start = 105000L + val step = 20000L + val end = 185000L + execPlan.addRangeVectorTransformer(new PeriodicSamplesMapper(start, step, end, None, None, Nil)) + + val resp = execPlan.execute(memStore, MMD.histDataset, queryConfig).runAsync.futureValue + val result = resp.asInstanceOf[QueryResult] + result.resultSchema.columns.map(_.colType) shouldEqual Seq(TimestampColumn, HistogramColumn) + result.result.size shouldEqual 1 + val resultIt = result.result(0).rows.map(r=>(r.getLong(0), r.getHistogram(1))) + val orig = histData.filter(_(4).asInstanceOf[Types.UTF8Map]("dc".utf8) == "0".utf8) + .grouped(2).map(_.head) // Skip every other one, starting with second, since step=2x pace + .zip((start to end by step).toIterator).map { case (r, t) => (t, r(3)) } + resultIt.zip(orig.toIterator).foreach { case (res, origData) => res shouldEqual origData } } it ("should return correct result schema") { diff --git a/query/src/test/scala/filodb/query/exec/WindowIteratorSpec.scala b/query/src/test/scala/filodb/query/exec/WindowIteratorSpec.scala index 3cb87b0be4..d31d19947b 100644 --- a/query/src/test/scala/filodb/query/exec/WindowIteratorSpec.scala +++ b/query/src/test/scala/filodb/query/exec/WindowIteratorSpec.scala @@ -2,7 +2,7 @@ package filodb.query.exec import filodb.core.metadata.Column.ColumnType import filodb.query.RangeFunctionId -import filodb.query.exec.rangefn.{ChunkedRangeFunction, RangeFunction, RawDataWindowingSpec} +import filodb.query.exec.rangefn.{RangeFunction, RawDataWindowingSpec} /** * Tests both the SlidingWindowIterator and the ChunkedWindowIterator @@ -134,7 +134,7 @@ class WindowIteratorSpec extends RawDataWindowingSpec { it ("should ignore out of order samples for RateFunction") { val rawRows = counterSamples.map(s => new TransientRow(s._1, s._2)) val slidingWinIterator = new SlidingWindowIterator(rawRows.iterator, 1538416154000L, 20000, 1538416649000L,20000, - RangeFunction(Some(RangeFunctionId.Rate), ColumnType.DoubleColumn, useChunked = false), queryConfig) + RangeFunction(Some(RangeFunctionId.Rate), ColumnType.DoubleColumn, useChunked = false).asSliding, queryConfig) slidingWinIterator.foreach{ v => // if out of order samples are not removed, counter correction causes rate to spike up to very high value v.value should be < 10000d @@ -159,7 +159,7 @@ class WindowIteratorSpec extends RawDataWindowingSpec { val end = 1000L val step = 5 val slidingWinIterator = new SlidingWindowIterator(rawRows.iterator, start, step, - end, 0, RangeFunction(None, ColumnType.DoubleColumn, useChunked = false), queryConfig) + end, 0, RangeFunction(None, ColumnType.DoubleColumn, useChunked = false).asSliding, queryConfig) val result = slidingWinIterator.map(v => (v.timestamp, v.value)).toSeq result.map(_._1) shouldEqual (start to end).by(step) result.foreach{ v => @@ -193,12 +193,15 @@ class WindowIteratorSpec extends RawDataWindowingSpec { 750000->17.0 ) val slidingWinIterator = new SlidingWindowIterator(rv.rows, 50000L, 100000, 1100000L, 100000, - RangeFunction(Some(RangeFunctionId.SumOverTime), ColumnType.DoubleColumn, useChunked = false), queryConfig) + RangeFunction(Some(RangeFunctionId.SumOverTime), ColumnType.DoubleColumn, useChunked = false).asSliding, + queryConfig) + // NOTE: dum_over_time sliding iterator does not handle the NaN at the end correctly! + // slidingWinIterator.map(r => (r.getLong(0), r.getDouble(1))).toList shouldEqual windowResults slidingWinIterator.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual windowResults - val chunkedIt = new ChunkedWindowIterator(rv, 50000L, 100000, 1100000L, 100000, - RangeFunction(Some(RangeFunctionId.SumOverTime), ColumnType.DoubleColumn, useChunked = true) - .asInstanceOf[ChunkedRangeFunction], queryConfig) + val chunkedIt = new ChunkedWindowIteratorD(rv, 50000L, 100000, 1100000L, 100000, + RangeFunction(Some(RangeFunctionId.SumOverTime), ColumnType.DoubleColumn, useChunked = true).asChunkedD, + queryConfig) chunkedIt.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual windowResults } @@ -223,7 +226,7 @@ class WindowIteratorSpec extends RawDataWindowingSpec { ) val rawRows = samples.map(s => new TransientRow(s._1, s._2)) val slidingWinIterator = new SlidingWindowIterator(rawRows.iterator, 1548191496000L, 15000, 1548191796000L,300000, - RangeFunction(Some(RangeFunctionId.Rate), ColumnType.DoubleColumn, useChunked = false), queryConfig) + RangeFunction(Some(RangeFunctionId.Rate), ColumnType.DoubleColumn, useChunked = false).asSliding, queryConfig) slidingWinIterator.foreach{ v => windowResults.find(a => a._1 == v.timestamp).foreach(b => Math.abs(b._2 - v.value) should be < 0.0000000001) } @@ -283,15 +286,15 @@ class WindowIteratorSpec extends RawDataWindowingSpec { 1540850640000L->330) // 330 becomes stale now. val slidingWinIterator = new SlidingWindowIterator(rv.rows, 1540845090000L, - 15000, 1540855905000L, 0, - RangeFunction(None, ColumnType.DoubleColumn, useChunked = false), - queryConfig) + 15000, 1540855905000L, 0, + RangeFunction(None, ColumnType.DoubleColumn, useChunked = false).asSliding, + queryConfig) slidingWinIterator.map(r => (r.getLong(0), r.getDouble(1))).toList.filter(!_._2.isNaN) shouldEqual windowResults - val chunkedWinIt = new ChunkedWindowIterator(rv, 1540845090000L, - 15000, 1540855905000L, queryConfig.staleSampleAfterMs, - RangeFunction(None, ColumnType.DoubleColumn, useChunked = true) - .asInstanceOf[ChunkedRangeFunction], queryConfig) + val chunkedWinIt = new ChunkedWindowIteratorD(rv, 1540845090000L, + 15000, 1540855905000L, queryConfig.staleSampleAfterMs, + RangeFunction(None, ColumnType.DoubleColumn, useChunked = true).asChunkedD, + queryConfig) chunkedWinIt.map(r => (r.getLong(0), r.getDouble(1))).toList.filter(!_._2.isNaN) shouldEqual windowResults } @@ -312,15 +315,14 @@ class WindowIteratorSpec extends RawDataWindowingSpec { val slidingWinIterator = new SlidingWindowIterator(rv.rows, 100000L, 100000, 600000L, 0, - RangeFunction(None, ColumnType.DoubleColumn, useChunked = false), + RangeFunction(None, ColumnType.DoubleColumn, useChunked = false).asSliding, queryConfig) slidingWinIterator.map(r => (r.getLong(0), r.getDouble(1))).toList.filter(!_._2.isNaN) shouldEqual windowResults // ChunkedWindowIterator requires window to be staleSampleAfterMs + 1 when window of SlidingWindowIterator is 0 - val chunkedWinIt = new ChunkedWindowIterator(rv, 100000L, + val chunkedWinIt = new ChunkedWindowIteratorD(rv, 100000L, 100000, 600000L, queryConfig.staleSampleAfterMs + 1, - RangeFunction(None, ColumnType.DoubleColumn, useChunked = true) - .asInstanceOf[ChunkedRangeFunction], queryConfig) + RangeFunction(None, ColumnType.DoubleColumn, useChunked = true).asChunkedD, queryConfig) chunkedWinIt.map(r => (r.getLong(0), r.getDouble(1))).toList.filter(!_._2.isNaN) shouldEqual windowResults } @@ -348,17 +350,17 @@ class WindowIteratorSpec extends RawDataWindowingSpec { ) val slidingWinIterator = new SlidingWindowIterator(rv.rows, 50000L, 100000, 700000L, 100000, - RangeFunction(Some(RangeFunctionId.AvgOverTime), ColumnType.DoubleColumn, useChunked = false), queryConfig) + RangeFunction(Some(RangeFunctionId.AvgOverTime), ColumnType.DoubleColumn, useChunked = false).asSliding, + queryConfig) slidingWinIterator.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual windowResults - val chunkedIt = new ChunkedWindowIterator(rv, 50000L, 100000, 700000L, 100000, - RangeFunction(Some(RangeFunctionId.AvgOverTime), ColumnType.DoubleColumn, useChunked = true) - .asInstanceOf[ChunkedRangeFunction], queryConfig) + val chunkedIt = new ChunkedWindowIteratorD(rv, 50000L, 100000, 700000L, 100000, + RangeFunction(Some(RangeFunctionId.AvgOverTime), ColumnType.DoubleColumn, useChunked = true).asChunkedD, + queryConfig) chunkedIt.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual windowResults } it("should calculate CountOverTime correctly even for windows with no values") { - val samples = Seq( 100000L -> 1d, 153000L -> 2d, @@ -381,18 +383,18 @@ class WindowIteratorSpec extends RawDataWindowingSpec { ) val slidingWinIterator = new SlidingWindowIterator(rv.rows, 50000L, 100000, 700000L, 100000, - RangeFunction(Some(RangeFunctionId.CountOverTime), ColumnType.DoubleColumn, useChunked = false), queryConfig) + RangeFunction(Some(RangeFunctionId.CountOverTime), ColumnType.DoubleColumn, useChunked = false).asSliding, + queryConfig) slidingWinIterator.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual windowResults - val chunkedIt = new ChunkedWindowIterator(rv, 50000L, 100000, 700000L, 100000, - RangeFunction(Some(RangeFunctionId.CountOverTime), ColumnType.DoubleColumn, useChunked = true) - .asInstanceOf[ChunkedRangeFunction], queryConfig) + val chunkedIt = new ChunkedWindowIteratorD(rv, 50000L, 100000, 700000L, 100000, + RangeFunction(Some(RangeFunctionId.CountOverTime), ColumnType.DoubleColumn, useChunked = true).asChunkedD, + queryConfig) chunkedIt.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual windowResults } it("should calculate MinOverTime correctly even for windows with no values") { - val samples = Seq( 100000L -> 1d, 153000L -> 2d, @@ -415,18 +417,18 @@ class WindowIteratorSpec extends RawDataWindowingSpec { ) val slidingWinIterator = new SlidingWindowIterator(rv.rows, 50000L, 100000, 700000L, 100000, - RangeFunction(Some(RangeFunctionId.MinOverTime), ColumnType.DoubleColumn, useChunked = false), queryConfig) + RangeFunction(Some(RangeFunctionId.MinOverTime), ColumnType.DoubleColumn, useChunked = false).asSliding, + queryConfig) slidingWinIterator.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual windowResults - val chunkedIt = new ChunkedWindowIterator(rv, 50000L, 100000, 700000L, 100000, - RangeFunction(Some(RangeFunctionId.MinOverTime), ColumnType.DoubleColumn, useChunked = true) - .asInstanceOf[ChunkedRangeFunction], queryConfig) + val chunkedIt = new ChunkedWindowIteratorD(rv, 50000L, 100000, 700000L, 100000, + RangeFunction(Some(RangeFunctionId.MinOverTime), ColumnType.DoubleColumn, useChunked = true).asChunkedD, + queryConfig) chunkedIt.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual windowResults } it("should calculate MaxOverTime correctly even for windows with no values") { - val samples = Seq( 100000L -> 1d, 153000L -> 2d, @@ -449,12 +451,13 @@ class WindowIteratorSpec extends RawDataWindowingSpec { ) val slidingWinIterator = new SlidingWindowIterator(rv.rows, 50000L, 100000, 700000L, 100000, - RangeFunction(Some(RangeFunctionId.MaxOverTime), ColumnType.DoubleColumn, useChunked = false), queryConfig) + RangeFunction(Some(RangeFunctionId.MaxOverTime), ColumnType.DoubleColumn, useChunked = false).asSliding, + queryConfig) slidingWinIterator.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual windowResults - val chunkedIt = new ChunkedWindowIterator(rv, 50000L, 100000, 700000L, 100000, - RangeFunction(Some(RangeFunctionId.MaxOverTime), ColumnType.DoubleColumn, useChunked = true) - .asInstanceOf[ChunkedRangeFunction], queryConfig) + val chunkedIt = new ChunkedWindowIteratorD(rv, 50000L, 100000, 700000L, 100000, + RangeFunction(Some(RangeFunctionId.MaxOverTime), ColumnType.DoubleColumn, useChunked = true).asChunkedD, + queryConfig) chunkedIt.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual windowResults } } diff --git a/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala b/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala index 5fc7336a42..9fb6537bd1 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala @@ -8,11 +8,11 @@ import org.scalatest.{BeforeAndAfterAll, FunSpec, Matchers} import filodb.core.memstore.{TimeSeriesPartitionSpec, WriteBufferPool} import filodb.core.query.RawDataRangeVector import filodb.core.store.AllChunkScan -import filodb.core.{MetricsTestData, TestData} +import filodb.core.{MachineMetricsData => MMD, MetricsTestData, TestData} import filodb.memory._ -import filodb.memory.format.TupleRowReader +import filodb.memory.format.{vectors => bv, TupleRowReader} import filodb.query.QueryConfig -import filodb.query.exec.{ChunkedWindowIterator, SlidingWindowIterator} +import filodb.query.exec._ /** * A common trait for windowing query tests which uses real chunks and real RawDataRangeVectors @@ -41,7 +41,7 @@ trait RawDataWindowingSpec extends FunSpec with Matchers with BeforeAndAfterAll val queryConfig = new QueryConfig(config.getConfig("query")) // windowSize and step are in number of elements of the data - def numWindows(data: Seq[Double], windowSize: Int, step: Int): Int = data.sliding(windowSize, step).length + def numWindows(data: Seq[Any], windowSize: Int, step: Int): Int = data.sliding(windowSize, step).length // Creates a RawDataRangeVector using Prometheus time-value schema and a given chunk size etc. def timeValueRV(tuples: Seq[(Long, Double)]): RawDataRangeVector = { @@ -59,16 +59,46 @@ trait RawDataWindowingSpec extends FunSpec with Matchers with BeforeAndAfterAll timeValueRV(tuples) } + // Call this only after calling histogramRV + def emptyAggHist: bv.MutableHistogram = bv.MutableHistogram.empty(MMD.histBucketScheme) + + protected val histIngestBH = new BlockMemFactory(blockStore, None, MMD.histDataset.blockMetaSize, true) + protected val histBufferPool = new WriteBufferPool(TestData.nativeMem, MMD.histDataset, 100, 10) + + // Designed explicitly to work with linearHistSeries records and histDataset from MachineMetricsData + def histogramRV(numSamples: Int = 100, numBuckets: Int = 8): + (Stream[Seq[Any]], RawDataRangeVector) = { + val histData = MMD.linearHistSeries(defaultStartTS, 1, pubFreq.toInt, numBuckets).take(numSamples) + val container = MMD.records(MMD.histDataset, histData).records + val part = TimeSeriesPartitionSpec.makePart(0, MMD.histDataset, bufferPool = histBufferPool) + container.iterate(MMD.histDataset.ingestionSchema).foreach { row => part.ingest(row, histIngestBH) } + // Now flush and ingest the rest to ensure two separate chunks + part.switchBuffers(histIngestBH, encode = true) + (histData, RawDataRangeVector(null, part, AllChunkScan, Array(0, 3))) // select timestamp and histogram columns only + } + def chunkedWindowIt(data: Seq[Double], rv: RawDataRangeVector, - func: ChunkedRangeFunction, + func: ChunkedRangeFunction[TransientRow], windowSize: Int, - step: Int): ChunkedWindowIterator = { + step: Int): ChunkedWindowIteratorD = { val windowTime = (windowSize.toLong - 1) * pubFreq val windowStartTS = defaultStartTS + windowTime val stepTimeMillis = step.toLong * pubFreq val windowEndTS = windowStartTS + (numWindows(data, windowSize, step) - 1) * stepTimeMillis - new ChunkedWindowIterator(rv, windowStartTS, stepTimeMillis, windowEndTS, windowTime, func, queryConfig) + new ChunkedWindowIteratorD(rv, windowStartTS, stepTimeMillis, windowEndTS, windowTime, func, queryConfig) + } + + def chunkedWindowItHist(data: Seq[Seq[Any]], + rv: RawDataRangeVector, + func: ChunkedRangeFunction[TransientHistRow], + windowSize: Int, + step: Int): ChunkedWindowIteratorH = { + val windowTime = (windowSize.toLong - 1) * pubFreq + val windowStartTS = defaultStartTS + windowTime + val stepTimeMillis = step.toLong * pubFreq + val windowEndTS = windowStartTS + (numWindows(data, windowSize, step) - 1) * stepTimeMillis + new ChunkedWindowIteratorH(rv, windowStartTS, stepTimeMillis, windowEndTS, windowTime, func, queryConfig) } def slidingWindowIt(data: Seq[Double], @@ -92,7 +122,6 @@ class AggrOverTimeFunctionsSpec extends RawDataWindowingSpec { it("should correctly aggregate sum_over_time using both chunked and sliding iterators") { val data = (1 to 240).map(_.toDouble) - val chunkSize = 40 val rv = timeValueRV(data) (0 until numIterations).foreach { x => val windowSize = rand.nextInt(100) + 10 @@ -110,6 +139,23 @@ class AggrOverTimeFunctionsSpec extends RawDataWindowingSpec { } } + it("should correctly aggregate sum_over_time for histogram RVs") { + val (data, rv) = histogramRV(numSamples = 150) + (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") + + 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]) + .foldLeft(emptyAggHist) { case (agg, h) => agg.add(h); agg } + aggHist shouldEqual sumRawHist + } + } + } + it("should correctly aggregate min_over_time / max_over_time using both chunked and sliding iterators") { val data = (1 to 240).map(_.toDouble) val chunkSize = 40 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 30931a1b83..18d570ffb7 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/InstantFunctionSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/InstantFunctionSpec.scala @@ -2,22 +2,20 @@ package filodb.query.exec.rangefn import scala.util.Random -import com.typesafe.config.{Config, ConfigFactory} import monix.execution.Scheduler.Implicits.global import monix.reactive.Observable -import org.scalatest.{FunSpec, Matchers} import org.scalatest.concurrent.ScalaFutures -import filodb.core.MetricsTestData +import filodb.core.{MachineMetricsData => MMD, MetricsTestData} import filodb.core.query.{CustomRangeVectorKey, RangeVector, RangeVectorKey, ResultSchema} import filodb.memory.format.{RowReader, ZeroCopyUTF8String} import filodb.query._ import filodb.query.exec.TransientRow -class InstantFunctionSpec extends FunSpec with Matchers with ScalaFutures { +class InstantFunctionSpec extends RawDataWindowingSpec with ScalaFutures { - val config: Config = ConfigFactory.load("application_test.conf").getConfig("filodb") val resultSchema = ResultSchema(MetricsTestData.timeseriesDataset.infosFromIDs(0 to 1), 1) + val histSchema = ResultSchema(MMD.histDataset.infosFromIDs(Seq(0, 3)), 1) val ignoreKey = CustomRangeVectorKey( Map(ZeroCopyUTF8String("ignore") -> ZeroCopyUTF8String("ignore"))) val sampleBase: Array[RangeVector] = Array( @@ -33,7 +31,6 @@ class InstantFunctionSpec extends FunSpec with Matchers with ScalaFutures { new TransientRow(3L, 3239.3423d), new TransientRow(4L, 94935.1523d)).iterator }) - val queryConfig = new QueryConfig(config.getConfig("query")) val rand = new Random() val error = 0.00000001d @@ -203,6 +200,16 @@ class InstantFunctionSpec extends FunSpec with Matchers with ScalaFutures { instantVectorFnMapper5(Observable.fromIterable(sampleBase), queryConfig, 1000, resultSchema) } should have message "requirement failed: Only one optional parameters allowed for Round." + // histogram quantile + the[IllegalArgumentException] thrownBy { + val ivMapper = exec.InstantVectorFunctionMapper(InstantFunctionId.HistogramQuantile) + ivMapper(Observable.fromIterable(sampleBase), queryConfig, 1000, histSchema) + } should have message "requirement failed: Quantile (between 0 and 1) required for histogram quantile" + + the[IllegalArgumentException] thrownBy { + val ivMapper = exec.InstantVectorFunctionMapper(InstantFunctionId.HistogramQuantile, Seq("b012")) + ivMapper(Observable.fromIterable(sampleBase), queryConfig, 1000, histSchema) + } should have message "requirement failed: histogram_quantile parameter must be a number" } it ("should fail with wrong calculation") { @@ -221,10 +228,18 @@ class InstantFunctionSpec extends FunSpec with Matchers with ScalaFutures { } } - private def applyFunctionAndAssertResult(samples: Array[RangeVector], expectedVal: Array[Iterator[Double]], - instantFunctionId: InstantFunctionId, funcParams: Seq[Any] = Nil): Unit = { + it("should compute histogram_quantile on Histogram RV") { + val (data, histRV) = histogramRV(numSamples = 10) + val expected = Seq(0.8, 1.6, 2.4, 3.2, 4.0, 5.6, 7.2, 9.6) + applyFunctionAndAssertResult(Array(histRV), Array(expected.toIterator), + InstantFunctionId.HistogramQuantile, Seq(0.4), histSchema) + } + + private def applyFunctionAndAssertResult(samples: Array[RangeVector], expectedVal: Array[Iterator[Double]], + instantFunctionId: InstantFunctionId, funcParams: Seq[Any] = Nil, + schema: ResultSchema = resultSchema): Unit = { val instantVectorFnMapper = exec.InstantVectorFunctionMapper(instantFunctionId, funcParams) - val resultObs = instantVectorFnMapper(Observable.fromIterable(samples), queryConfig, 1000, resultSchema) + val resultObs = instantVectorFnMapper(Observable.fromIterable(samples), queryConfig, 1000, schema) val result = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) expectedVal.zip(result).foreach { case (ex, res) => { @@ -232,7 +247,7 @@ class InstantFunctionSpec extends FunSpec with Matchers with ScalaFutures { case (val1, val2) => if (val1.isInfinity) val2.isInfinity shouldEqual true else if (val1.isNaN) val2.isNaN shouldEqual true - else val1 shouldEqual val2 + else val1 shouldEqual val2 +- 0.0001 } } } diff --git a/spark/src/main/scala/filodb.spark/TypeConverters.scala b/spark/src/main/scala/filodb.spark/TypeConverters.scala index ad81d0e9a6..77175c4c3d 100644 --- a/spark/src/main/scala/filodb.spark/TypeConverters.scala +++ b/spark/src/main/scala/filodb.spark/TypeConverters.scala @@ -10,7 +10,6 @@ object TypeConverters { Column.ColumnType.DoubleColumn -> DoubleType, Column.ColumnType.LongColumn -> LongType, Column.ColumnType.StringColumn -> StringType, - Column.ColumnType.BitmapColumn -> BooleanType, Column.ColumnType.TimestampColumn -> TimestampType ) From 0ca0fd8dd250c61ba35c2e99420bcdbe6f28c702 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Thu, 28 Feb 2019 14:19:14 -0800 Subject: [PATCH 02/41] bug(prometheus): Use a bigger dummy step for instant queries (#262) Instant queries used a smaller dummy step that would not pass validation in PeriodicSamplesMapper. --- .../filodb/prometheus/parse/Parser.scala | 4 +- .../filodb/prometheus/parse/ParserSpec.scala | 68 +++++++++---------- 2 files changed, 37 insertions(+), 35 deletions(-) diff --git a/prometheus/src/main/scala/filodb/prometheus/parse/Parser.scala b/prometheus/src/main/scala/filodb/prometheus/parse/Parser.scala index ec76f0f641..4b42315b70 100644 --- a/prometheus/src/main/scala/filodb/prometheus/parse/Parser.scala +++ b/prometheus/src/main/scala/filodb/prometheus/parse/Parser.scala @@ -325,7 +325,9 @@ object Parser extends Expression { } def queryToLogicalPlan(query: String, queryTimestamp: Long): LogicalPlan = { - val defaultQueryParams = TimeStepParams(queryTimestamp, 1, queryTimestamp) + // step does not matter here in instant query. + // Since we have some min validation on the server, pass Long.MaxValue / 1000 seconds for step. + val defaultQueryParams = TimeStepParams(queryTimestamp, Long.MaxValue / 1000, queryTimestamp) queryRangeToLogicalPlan(query, defaultQueryParams) } diff --git a/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala b/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala index c9ab822d44..d18ef6a7d4 100644 --- a/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala +++ b/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala @@ -245,15 +245,15 @@ class ParserSpec extends FunSpec with Matchers { it("Should be able to make logical plans for Series Expressions") { val queryToLpString = Map( "primary:instance-001:no.ofrequests{job=\"my-job\"}" -> - "PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(job,Equals(my-job)), ColumnFilter(__name__,Equals(primary:instance-001:no.ofrequests))),List()),1524855988000,1000,1524855988000)", + "PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(job,Equals(my-job)), ColumnFilter(__name__,Equals(primary:instance-001:no.ofrequests))),List()),1524855988000,9223372036854775000,1524855988000)", "absent(nonexistent{job=\"myjob\"})" -> - "ApplyInstantFunction(PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(job,Equals(myjob)), ColumnFilter(__name__,Equals(nonexistent))),List()),1524855988000,1000,1524855988000),Absent,List())", + "ApplyInstantFunction(PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(job,Equals(myjob)), ColumnFilter(__name__,Equals(nonexistent))),List()),1524855988000,9223372036854775000,1524855988000),Absent,List())", "rate(http_requests_total[5m] offset 1w)" -> - "PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000,1524855988000,300000,Rate,List())", + "PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,9223372036854775000,1524855988000,300000,Rate,List())", "http_requests_total{job=\"prometheus\",group=\"canary\"}" -> - "PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(job,Equals(prometheus)), ColumnFilter(group,Equals(canary)), ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000,1524855988000)", + "PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(job,Equals(prometheus)), ColumnFilter(group,Equals(canary)), ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,9223372036854775000,1524855988000)", "http_requests_total{job=\"prometheus\",__col__=\"min\"}" -> - "PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(job,Equals(prometheus)), ColumnFilter(__name__,Equals(http_requests_total))),List(min)),1524855988000,1000,1524855988000)", + "PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(job,Equals(prometheus)), ColumnFilter(__name__,Equals(http_requests_total))),List(min)),1524855988000,9223372036854775000,1524855988000)", // Internal FiloDB debug function "_filodb_chunkmeta_all(http_requests_total{job=\"prometheus\"})" -> "RawChunkMeta(IntervalSelector(List(1524855988000),List(1524855988000)),List(ColumnFilter(job,Equals(prometheus)), ColumnFilter(__name__,Equals(http_requests_total))),)", @@ -261,67 +261,67 @@ class ParserSpec extends FunSpec with Matchers { "RawChunkMeta(IntervalSelector(List(1524855988000),List(1524855988000)),List(ColumnFilter(job,Equals(prometheus)), ColumnFilter(__name__,Equals(http_requests_total))),avg)", "sum(http_requests_total) by (application, group)" -> - "Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000,1524855988000),List(),List(application, group),List())", + "Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,9223372036854775000,1524855988000),List(),List(application, group),List())", "sum(http_requests_total) without (instance)" -> - "Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000,1524855988000),List(),List(),List(instance))", + "Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,9223372036854775000,1524855988000),List(),List(),List(instance))", "count_values(\"version\", build_version)" -> - "Aggregate(CountValues,PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(__name__,Equals(build_version))),List()),1524855988000,1000,1524855988000),List(\"version\"),List(),List())", + "Aggregate(CountValues,PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(__name__,Equals(build_version))),List()),1524855988000,9223372036854775000,1524855988000),List(\"version\"),List(),List())", "label_replace(up{job=\"api-server\",service=\"a:c\"}, \"foo\", \"$1\", \"service\", \"(.*):.*\")" -> - "ApplyInstantFunction(PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(job,Equals(api-server)), ColumnFilter(service,Equals(a:c)), ColumnFilter(__name__,Equals(up))),List()),1524855988000,1000,1524855988000),LabelReplace,List())", + "ApplyInstantFunction(PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(job,Equals(api-server)), ColumnFilter(service,Equals(a:c)), ColumnFilter(__name__,Equals(up))),List()),1524855988000,9223372036854775000,1524855988000),LabelReplace,List())", "sum(http_requests_total)" -> - "Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000,1524855988000),List(),List(),List())", + "Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,9223372036854775000,1524855988000),List(),List(),List())", "label_join(up{job=\"api-server\",src1=\"a\",src2=\"b\",src3=\"c\"}, \"foo\", \",\", \"src1\", \"src2\", \"src3\")" -> - "ApplyInstantFunction(PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(job,Equals(api-server)), ColumnFilter(src1,Equals(a)), ColumnFilter(src2,Equals(b)), ColumnFilter(src3,Equals(c)), ColumnFilter(__name__,Equals(up))),List()),1524855988000,1000,1524855988000),LabelJoin,List())", + "ApplyInstantFunction(PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(job,Equals(api-server)), ColumnFilter(src1,Equals(a)), ColumnFilter(src2,Equals(b)), ColumnFilter(src3,Equals(c)), ColumnFilter(__name__,Equals(up))),List()),1524855988000,9223372036854775000,1524855988000),LabelJoin,List())", "histogram_quantile(0.9, sum(rate(http_request_duration_seconds_bucket[10m])) by (le))" -> - "ApplyInstantFunction(Aggregate(Sum,PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(List(1524855388000),List(1524855988000)),List(ColumnFilter(__name__,Equals(http_request_duration_seconds_bucket))),List()),1524855988000,1000,1524855988000,600000,Rate,List()),List(),List(le),List()),HistogramQuantile,List(0.9))", + "ApplyInstantFunction(Aggregate(Sum,PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(List(1524855388000),List(1524855988000)),List(ColumnFilter(__name__,Equals(http_request_duration_seconds_bucket))),List()),1524855988000,9223372036854775000,1524855988000,600000,Rate,List()),List(),List(le),List()),HistogramQuantile,List(0.9))", "delta(cpu_temp_celsius{host=\"zeus\"}[2h])" -> - "PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(List(1524848788000),List(1524855988000)),List(ColumnFilter(host,Equals(zeus)), ColumnFilter(__name__,Equals(cpu_temp_celsius))),List()),1524855988000,1000,1524855988000,7200000,Delta,List())", + "PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(List(1524848788000),List(1524855988000)),List(ColumnFilter(host,Equals(zeus)), ColumnFilter(__name__,Equals(cpu_temp_celsius))),List()),1524855988000,9223372036854775000,1524855988000,7200000,Delta,List())", "method_code:http_errors:rate5m{code=\"500\"} / ignoring(code) method:http_requests:rate5m" -> - "BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(code,Equals(500)), ColumnFilter(__name__,Equals(method_code:http_errors:rate5m))),List()),1524855988000,1000,1524855988000),DIV,OneToOne,PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(__name__,Equals(method:http_requests:rate5m))),List()),1524855988000,1000,1524855988000),List(),List())", + "BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(code,Equals(500)), ColumnFilter(__name__,Equals(method_code:http_errors:rate5m))),List()),1524855988000,9223372036854775000,1524855988000),DIV,OneToOne,PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(__name__,Equals(method:http_requests:rate5m))),List()),1524855988000,9223372036854775000,1524855988000),List(),List())", "histogram_quantile(0.9, rate(http_request_duration_seconds_bucket[10m]))" -> - "ApplyInstantFunction(PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(List(1524855388000),List(1524855988000)),List(ColumnFilter(__name__,Equals(http_request_duration_seconds_bucket))),List()),1524855988000,1000,1524855988000,600000,Rate,List()),HistogramQuantile,List(0.9))", + "ApplyInstantFunction(PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(List(1524855388000),List(1524855988000)),List(ColumnFilter(__name__,Equals(http_request_duration_seconds_bucket))),List()),1524855988000,9223372036854775000,1524855988000,600000,Rate,List()),HistogramQuantile,List(0.9))", "histogram_quantile(0.9, sum(rate(http_request_duration_seconds_bucket[10m])) by (job, le))" -> - "ApplyInstantFunction(Aggregate(Sum,PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(List(1524855388000),List(1524855988000)),List(ColumnFilter(__name__,Equals(http_request_duration_seconds_bucket))),List()),1524855988000,1000,1524855988000,600000,Rate,List()),List(),List(job, le),List()),HistogramQuantile,List(0.9))", + "ApplyInstantFunction(Aggregate(Sum,PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(List(1524855388000),List(1524855988000)),List(ColumnFilter(__name__,Equals(http_request_duration_seconds_bucket))),List()),1524855988000,9223372036854775000,1524855988000,600000,Rate,List()),List(),List(job, le),List()),HistogramQuantile,List(0.9))", "http_requests_total" -> - "PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000,1524855988000)", + "PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,9223372036854775000,1524855988000)", "http_requests_total ^ 5" -> - "ScalarVectorBinaryOperation(POW,5.0,PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000,1524855988000),false)", + "ScalarVectorBinaryOperation(POW,5.0,PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,9223372036854775000,1524855988000),false)", //FIXME Operator precedence is not implemented "10 + http_requests_total * 5" -> - "ScalarVectorBinaryOperation(ADD,10.0,ScalarVectorBinaryOperation(MUL,5.0,PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000,1524855988000),false),true)", + "ScalarVectorBinaryOperation(ADD,10.0,ScalarVectorBinaryOperation(MUL,5.0,PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,9223372036854775000,1524855988000),false),true)", "10 + (http_requests_total * 5)" -> - "ScalarVectorBinaryOperation(ADD,10.0,ScalarVectorBinaryOperation(MUL,5.0,PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000,1524855988000),false),true)", + "ScalarVectorBinaryOperation(ADD,10.0,ScalarVectorBinaryOperation(MUL,5.0,PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,9223372036854775000,1524855988000),false),true)", "(10 + http_requests_total) * 5" -> - "ScalarVectorBinaryOperation(MUL,5.0,ScalarVectorBinaryOperation(ADD,10.0,PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000,1524855988000),true),false)", + "ScalarVectorBinaryOperation(MUL,5.0,ScalarVectorBinaryOperation(ADD,10.0,PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,9223372036854775000,1524855988000),true),false)", "topk(5, http_requests_total)" -> - "Aggregate(TopK,PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000,1524855988000),List(5.0),List(),List())", + "Aggregate(TopK,PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,9223372036854775000,1524855988000),List(5.0),List(),List())", "irate(http_requests_total{job=\"api-server\"}[5m])" -> - "PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(job,Equals(api-server)), ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000,1524855988000,300000,Irate,List())", + "PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(job,Equals(api-server)), ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,9223372036854775000,1524855988000,300000,Irate,List())", "idelta(http_requests_total{job=\"api-server\"}[5m])" -> - "PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(job,Equals(api-server)), ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000,1524855988000,300000,Idelta,List())", + "PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(job,Equals(api-server)), ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,9223372036854775000,1524855988000,300000,Idelta,List())", "resets(http_requests_total{job=\"api-server\"}[5m])" -> - "PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(job,Equals(api-server)), ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000,1524855988000,300000,Resets,List())", + "PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(job,Equals(api-server)), ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,9223372036854775000,1524855988000,300000,Resets,List())", "deriv(http_requests_total{job=\"api-server\"}[5m])" -> - "PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(job,Equals(api-server)), ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000,1524855988000,300000,Deriv,List())", + "PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(job,Equals(api-server)), ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,9223372036854775000,1524855988000,300000,Deriv,List())", "rate(http_requests_total{job=\"api-server\"}[5m])" -> - "PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(job,Equals(api-server)), ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000,1524855988000,300000,Rate,List())", + "PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(job,Equals(api-server)), ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,9223372036854775000,1524855988000,300000,Rate,List())", "http_requests_total{job=\"prometheus\"}[5m]" -> "RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(job,Equals(prometheus)), ColumnFilter(__name__,Equals(http_requests_total))),List())", "http_requests_total offset 5m" -> - "PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000,1524855988000)", + "PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,9223372036854775000,1524855988000)", "http_requests_total{environment=~\"staging|testing|development\",method!=\"GET\"}" -> - "PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(environment,EqualsRegex(staging|testing|development)), ColumnFilter(method,NotEquals(GET)), ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000,1524855988000)", + "PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(environment,EqualsRegex(staging|testing|development)), ColumnFilter(method,NotEquals(GET)), ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,9223372036854775000,1524855988000)", "method_code:http_errors:rate5m / ignoring(code) group_left method:http_requests:rate5m" -> - "BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(__name__,Equals(method_code:http_errors:rate5m))),List()),1524855988000,1000,1524855988000),DIV,OneToMany,PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(__name__,Equals(method:http_requests:rate5m))),List()),1524855988000,1000,1524855988000),List(),List())", + "BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(__name__,Equals(method_code:http_errors:rate5m))),List()),1524855988000,9223372036854775000,1524855988000),DIV,OneToMany,PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(__name__,Equals(method:http_requests:rate5m))),List()),1524855988000,9223372036854775000,1524855988000),List(),List())", "increase(http_requests_total{job=\"api-server\"}[5m])" -> - "PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(job,Equals(api-server)), ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000,1524855988000,300000,Increase,List())", + "PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(job,Equals(api-server)), ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,9223372036854775000,1524855988000,300000,Increase,List())", "sum(http_requests_total{method=\"GET\"} offset 5m)" -> - "Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(method,Equals(GET)), ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000,1524855988000),List(),List(),List())", + "Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(method,Equals(GET)), ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,9223372036854775000,1524855988000),List(),List(),List())", "absent(nonexistent{job=\"myjob\",instance=~\".*\"})" -> - "ApplyInstantFunction(PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(job,Equals(myjob)), ColumnFilter(instance,EqualsRegex(.*)), ColumnFilter(__name__,Equals(nonexistent))),List()),1524855988000,1000,1524855988000),Absent,List())", + "ApplyInstantFunction(PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(job,Equals(myjob)), ColumnFilter(instance,EqualsRegex(.*)), ColumnFilter(__name__,Equals(nonexistent))),List()),1524855988000,9223372036854775000,1524855988000),Absent,List())", "absent(sum(nonexistent{job=\"myjob\"}))" -> - "ApplyInstantFunction(Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(job,Equals(myjob)), ColumnFilter(__name__,Equals(nonexistent))),List()),1524855988000,1000,1524855988000),List(),List(),List()),Absent,List())" + "ApplyInstantFunction(Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(job,Equals(myjob)), ColumnFilter(__name__,Equals(nonexistent))),List()),1524855988000,9223372036854775000,1524855988000),List(),List(),List()),Absent,List())" ) val qts: Long = 1524855988L From a90d02c7d362d43de59557b42ec02fafca4c1139 Mon Sep 17 00:00:00 2001 From: "Brian S. O'Neill" Date: Tue, 5 Mar 2019 14:11:45 -0800 Subject: [PATCH 03/41] misc: Run Travis builds with openjdk11 (#264) --- .travis.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.travis.yml b/.travis.yml index 2fbb1de748..3266eedd79 100644 --- a/.travis.yml +++ b/.travis.yml @@ -5,7 +5,7 @@ env: scala: - 2.11.12 jdk: - - oraclejdk8 + - openjdk11 # These directories are cached to S3 at the end of the build cache: From 5c24fa686fa64593270bcdfc3687599c5b7bdb51 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Wed, 6 Mar 2019 11:53:55 -0800 Subject: [PATCH 04/41] fix(coordinator): Use random Akka port in some tests to reduce failures (#266) --- .../scala/filodb.coordinator/FilodbClusterNodeSpec.scala | 9 +++++---- .../filodb.coordinator/ShardSubscriptionsSpec.scala | 4 +--- 2 files changed, 6 insertions(+), 7 deletions(-) diff --git a/coordinator/src/test/scala/filodb.coordinator/FilodbClusterNodeSpec.scala b/coordinator/src/test/scala/filodb.coordinator/FilodbClusterNodeSpec.scala index 03fb8001c9..3491301513 100644 --- a/coordinator/src/test/scala/filodb.coordinator/FilodbClusterNodeSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/FilodbClusterNodeSpec.scala @@ -34,10 +34,13 @@ trait SocketChecker { } trait FilodbClusterNodeSpec extends AbstractSpec with FilodbClusterNode with ScalaFutures { + val port = 22552 + util.Random.nextInt(200) + // Ensure that CoordinatedShutdown does not shutdown the whole test JVM, otherwise Travis CI/CD fails override protected lazy val roleConfig = ConfigFactory.parseString( - """akka.coordinated-shutdown.run-by-jvm-shutdown-hook=off + s"""akka.coordinated-shutdown.run-by-jvm-shutdown-hook=off |akka.coordinated-shutdown.exit-jvm = off + |akka.remote.netty.tcp.port=$port """.stripMargin) implicit abstract override val patienceConfig: PatienceConfig = @@ -133,7 +136,7 @@ class ClusterNodeServerSpec extends FilodbClusterNodeSpec { * Initiates cluster singleton recovery sequence by populating guardian with some initial non-empty * shardmap and subscription state, and checking that it is recovered properly on startup */ -class ClusterNodeRecoverySpec extends FilodbClusterNodeSpec with SocketChecker { +class ClusterNodeRecoverySpec extends FilodbClusterNodeSpec { import scala.collection.immutable import scala.concurrent.duration._ @@ -145,8 +148,6 @@ class ClusterNodeRecoverySpec extends FilodbClusterNodeSpec with SocketChecker { import NodeClusterActor._ import NodeProtocol._ - waitSocketOpen(2552) - override val role = ClusterRole.Server override protected lazy val roleConfig: Config = AkkaSpec.settings.allConfig diff --git a/coordinator/src/test/scala/filodb.coordinator/ShardSubscriptionsSpec.scala b/coordinator/src/test/scala/filodb.coordinator/ShardSubscriptionsSpec.scala index ef0cf90a8b..6f4b1ed8f6 100644 --- a/coordinator/src/test/scala/filodb.coordinator/ShardSubscriptionsSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/ShardSubscriptionsSpec.scala @@ -4,7 +4,7 @@ import akka.testkit.TestProbe import filodb.core.DatasetRef -class ShardSubscriptionsSpec extends AkkaSpec with SocketChecker { +class ShardSubscriptionsSpec extends AkkaSpec { private val extension = FilodbCluster(system) @@ -13,8 +13,6 @@ class ShardSubscriptionsSpec extends AkkaSpec with SocketChecker { private val subscribers = Set(TestProbe().ref, TestProbe().ref) - waitSocketOpen(2552) - "ShardSubscription" must { "add unseen subscribers, not add seen subscribers" in { var subscription = ShardSubscription(dataset1, Set.empty) From ef8cc28e58cf08540e0ba936e0ccdc6a2c94476f Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Thu, 7 Mar 2019 09:51:35 -0800 Subject: [PATCH 05/41] fix(core): option to disable new endMap() bulk hash calculation; PromInputRecord preserves old hash calculation (#270) --- .../filodb.core/binaryrecord2/RecordBuilder.scala | 10 +++++++--- .../scala/filodb/gateway/conversion/InputRecord.scala | 2 +- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/filodb.core/binaryrecord2/RecordBuilder.scala b/core/src/main/scala/filodb.core/binaryrecord2/RecordBuilder.scala index 808495336e..6b3b826006 100644 --- a/core/src/main/scala/filodb.core/binaryrecord2/RecordBuilder.scala +++ b/core/src/main/scala/filodb.core/binaryrecord2/RecordBuilder.scala @@ -325,10 +325,14 @@ final class RecordBuilder(memFactory: MemFactory, /** * Ends creation of a map field. Recompute the hash for all fields at once. + * @param bulkHash if true (default), computes the hash for all key/values. + * Some users use the older alternate, sortAndComputeHashes() - then set this to false. */ - final def endMap(): Unit = { - val mapHash = BinaryRegion.hash32(curBase, mapOffset, (curRecEndOffset - mapOffset).toInt) - updatePartitionHash(mapHash) + final def endMap(bulkHash: Boolean = true): Unit = { + if (bulkHash) { + val mapHash = BinaryRegion.hash32(curBase, mapOffset, (curRecEndOffset - mapOffset).toInt) + updatePartitionHash(mapHash) + } mapOffset = -1L fieldNo += 1 } diff --git a/gateway/src/main/scala/filodb/gateway/conversion/InputRecord.scala b/gateway/src/main/scala/filodb/gateway/conversion/InputRecord.scala index 59c04c9a9e..c6107ca38d 100644 --- a/gateway/src/main/scala/filodb/gateway/conversion/InputRecord.scala +++ b/gateway/src/main/scala/filodb/gateway/conversion/InputRecord.scala @@ -79,7 +79,7 @@ case class PrometheusInputRecord(tags: Map[String, String], builder.addMapKeyValue(k.getBytes, v.getBytes) builder.updatePartitionHash(hashes(i)) } - builder.endMap() + builder.endMap(bulkHash = false) builder.endRecord() } From 8db790ceccf99f2fa9257d7d0856e488844d4597 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Thu, 7 Mar 2019 11:04:10 -0800 Subject: [PATCH 06/41] feat(query): Handling spread changes over time in Query Engine - Iteration 1 (#268) * spreadFunc in QueryOptions now receives all changes in spread as opposed to the current value * Query Planner schedules the RV stitching prior to a join, and finally just before sending out final result. We return an NaN instead of a wrong result if overlap is seen during stitching. * Expectation for gateway to schedule spread changes for a future timestamp Known Limitations in this First Iteration Solution: Time windows aka lookback windows in queries may now span multiple shards due to spread change. This involves a more detailed solution and is out of the scope of this PR. As a first solution, we rely on spread change to be scheduled at the 0th minute of the hour so that the time window spanning across shards are minimized. Other fixes in commit: * Return correct future in some C* Store methods * Use correct dummy step for fix for instant queries. Earlier fix didn't work. --- .../columnstore/CassandraColumnStore.scala | 6 +- .../client/QueryCommands.scala | 23 ++- .../queryengine/Utils.scala | 2 +- .../queryengine2/QueryEngine.scala | 115 ++++++++---- .../client/SerializationSpec.scala | 2 +- .../queryengine2/QueryEngineSpec.scala | 50 +++++- .../memstore/OnDemandPagingShard.scala | 4 +- .../scala/filodb.core/query/RangeVector.scala | 4 +- .../timeseries/TestTimeseriesProducer.scala | 13 +- .../filodb/http/PrometheusApiRoute.scala | 4 +- .../scala/filodb/prometheus/ast/Base.scala | 2 +- .../filodb/prometheus/parse/Parser.scala | 5 +- .../prometheus/query/PrometheusModel.scala | 2 +- .../filodb/prometheus/parse/ParserSpec.scala | 74 ++++---- .../main/scala/filodb/query/LogicalPlan.scala | 2 +- .../filodb/query/exec/BinaryJoinExec.scala | 4 +- .../filodb/query/exec/DistConcatExec.scala | 11 +- .../query/exec/HistogramQuantileMapper.scala | 6 +- .../query/exec/PeriodicSamplesMapper.scala | 2 +- .../scala/filodb/query/exec/RowKeyRange.scala | 12 ++ .../filodb/query/exec/StitchRvsExec.scala | 108 +++++++++++ .../filodb/query/exec/StitchRvsExecSpec.scala | 170 ++++++++++++++++++ .../scala/filodb.stress/MemStoreStress.scala | 2 +- 23 files changed, 503 insertions(+), 120 deletions(-) create mode 100644 query/src/main/scala/filodb/query/exec/RowKeyRange.scala create mode 100644 query/src/main/scala/filodb/query/exec/StitchRvsExec.scala create mode 100644 query/src/test/scala/filodb/query/exec/StitchRvsExecSpec.scala diff --git a/cassandra/src/main/scala/filodb.cassandra/columnstore/CassandraColumnStore.scala b/cassandra/src/main/scala/filodb.cassandra/columnstore/CassandraColumnStore.scala index 0312d7159b..301fdda53b 100644 --- a/cassandra/src/main/scala/filodb.cassandra/columnstore/CassandraColumnStore.scala +++ b/cassandra/src/main/scala/filodb.cassandra/columnstore/CassandraColumnStore.scala @@ -68,7 +68,7 @@ extends ColumnStore with CassandraChunkSource with StrictLogging { clusterMeta.checkSchemaAgreement() for { ctResp <- chunkTable.initialize() rmtResp <- indexTable.initialize() - pitResp <- partIndexTable.initialize() } yield rmtResp + pitResp <- partIndexTable.initialize() } yield pitResp } def truncate(dataset: DatasetRef): Future[Response] = { @@ -79,7 +79,7 @@ extends ColumnStore with CassandraChunkSource with StrictLogging { clusterMeta.checkSchemaAgreement() for { ctResp <- chunkTable.clearAll() rmtResp <- indexTable.clearAll() - pitResp <- partIndexTable.clearAll() } yield rmtResp + pitResp <- partIndexTable.clearAll() } yield pitResp } def dropDataset(dataset: DatasetRef): Future[Response] = { @@ -94,7 +94,7 @@ extends ColumnStore with CassandraChunkSource with StrictLogging { chunkTableCache.remove(dataset) indexTableCache.remove(dataset) partitionIndexTableCache.remove(dataset) - rmtResp + pitResp } } diff --git a/coordinator/src/main/scala/filodb.coordinator/client/QueryCommands.scala b/coordinator/src/main/scala/filodb.coordinator/client/QueryCommands.scala index d1b9647b65..534aa3b27a 100644 --- a/coordinator/src/main/scala/filodb.coordinator/client/QueryCommands.scala +++ b/coordinator/src/main/scala/filodb.coordinator/client/QueryCommands.scala @@ -32,7 +32,14 @@ object QueryCommands { limit: Int = 100, submitTime: Long = System.currentTimeMillis()) extends QueryCommand - final case class QueryOptions(spreadFunc: Seq[ColumnFilter] => Int = { x => 1 }, + + final case class SpreadChange(time: Long = 0L, spread: Int = 1) + + /** + * This class provides general query processing parameters + * @param spreadFunc a function that returns chronologically ordered spread changes for the filter + */ + final case class QueryOptions(spreadFunc: Seq[ColumnFilter] => Seq[SpreadChange] = { _ => Seq(SpreadChange()) }, parallelism: Int = 16, queryTimeoutSecs: Int = 30, sampleLimit: Int = 1000000, @@ -40,7 +47,7 @@ object QueryCommands { object QueryOptions { def apply(constSpread: Int, sampleLimit: Int): QueryOptions = - QueryOptions(spreadFunc = { x => constSpread}, sampleLimit = sampleLimit) + QueryOptions(spreadFunc = { _ => Seq(SpreadChange(spread = constSpread))}, sampleLimit = sampleLimit) /** * Creates a spreadFunc that looks for a particular filter with keyName Equals a value, and then maps values @@ -48,20 +55,20 @@ object QueryCommands { */ def simpleMapSpreadFunc(keyName: String, spreadMap: collection.Map[String, Int], - defaultSpread: Int): Seq[ColumnFilter] => Int = { + defaultSpread: Int): Seq[ColumnFilter] => Seq[SpreadChange] = { filters: Seq[ColumnFilter] => - filters.collect { + filters.collectFirst { case ColumnFilter(key, Filter.Equals(filtVal: String)) if key == keyName => filtVal - }.headOption.map { tagValue => - spreadMap.getOrElse(tagValue, defaultSpread) - }.getOrElse(defaultSpread) + }.map { tagValue => + Seq(SpreadChange(spread = spreadMap.getOrElse(tagValue, defaultSpread))) + }.getOrElse(Seq(SpreadChange(defaultSpread))) } import collection.JavaConverters._ def simpleMapSpreadFunc(keyName: String, spreadMap: java.util.Map[String, Int], - defaultSpread: Int): Seq[ColumnFilter] => Int = + defaultSpread: Int): Seq[ColumnFilter] => Seq[SpreadChange] = simpleMapSpreadFunc(keyName, spreadMap.asScala, defaultSpread) } diff --git a/coordinator/src/main/scala/filodb.coordinator/queryengine/Utils.scala b/coordinator/src/main/scala/filodb.coordinator/queryengine/Utils.scala index 0a4e5599c3..4df2701277 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryengine/Utils.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryengine/Utils.scala @@ -74,7 +74,7 @@ object Utils extends StrictLogging { val shardCols = dataset.options.shardKeyColumns if (shardCols.length > 0) { shardHashFromFilters(filters, shardCols, dataset) match { - case Some(shardHash) => shardMap.queryShards(shardHash, options.spreadFunc(filters)) + case Some(shardHash) => shardMap.queryShards(shardHash, options.spreadFunc(filters).last.spread) case None => throw new IllegalArgumentException(s"Must specify filters for $shardCols") } } else { diff --git a/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala b/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala index 61906f5508..2096e84c08 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala @@ -30,6 +30,14 @@ class QueryEngine(dataset: Dataset, shardMapperFunc: => ShardMapper) extends StrictLogging { + /** + * Intermediate Plan Result includes the exec plan(s) along with any state to be passed up the + * plan building call tree during query planning. + * + * Not for runtime use. + */ + private case class PlanResult(plans: Seq[ExecPlan], needsStitch: Boolean = false) + private val mdNoShardKeyFilterRequests = Kamon.counter("queryengine-metadata-no-shardkey-requests") /** @@ -52,14 +60,18 @@ class QueryEngine(dataset: Dataset, options: QueryOptions): ExecPlan = { val queryId = UUID.randomUUID().toString val materialized = walkLogicalPlanTree(rootLogicalPlan, queryId, System.currentTimeMillis(), options) match { - case Seq(justOne) => + case PlanResult(Seq(justOne), stitch) => + if (stitch) justOne.addRangeVectorTransformer(new StitchRvsMapper()) justOne - case many => + case PlanResult(many, stitch) => val targetActor = pickDispatcher(many) many(0) match { - case lve: LabelValuesExec => LabelValuesDistConcatExec(queryId, targetActor, many) + case lve: LabelValuesExec =>LabelValuesDistConcatExec(queryId, targetActor, many) case ske: PartKeysExec => PartKeysDistConcatExec(queryId, targetActor, many) - case ep: ExecPlan => DistConcatExec(queryId, targetActor, many) + case ep: ExecPlan => + val topPlan = DistConcatExec(queryId, targetActor, many) + if (stitch) topPlan.addRangeVectorTransformer(new StitchRvsMapper()) + topPlan } } logger.debug(s"Materialized logical plan for dataset=${dataset.ref}:" + @@ -88,13 +100,13 @@ class QueryEngine(dataset: Dataset, s"$shardCol, shard key hashing disabled") } } - val metric = shardVals.filter(_._1 == dataset.options.metricColumn).headOption + val metric = shardVals.find(_._1 == dataset.options.metricColumn) .map(_._2) .getOrElse(throw new BadQueryException(s"Could not find metric value")) val shardValues = shardVals.filterNot(_._1 == dataset.options.metricColumn).map(_._2) logger.debug(s"For shardColumns $shardColumns, extracted metric $metric and shard values $shardValues") val shardHash = RecordBuilder.shardKeyHash(shardValues, metric) - shardMapperFunc.queryShards(shardHash, options.spreadFunc(filters)) + shardMapperFunc.queryShards(shardHash, options.spreadFunc(filters).last.spread) } } @@ -112,7 +124,7 @@ class QueryEngine(dataset: Dataset, private def walkLogicalPlanTree(logicalPlan: LogicalPlan, queryId: String, submitTime: Long, - options: QueryOptions): Seq[ExecPlan] = { + options: QueryOptions): PlanResult = { logicalPlan match { case lp: RawSeries => materializeRawSeries(queryId, submitTime, options, lp) case lp: RawChunkMeta => materializeRawChunkMeta(queryId, submitTime, options, lp) @@ -130,50 +142,74 @@ class QueryEngine(dataset: Dataset, private def materializeScalarVectorBinOp(queryId: String, submitTime: Long, options: QueryOptions, - lp: ScalarVectorBinaryOperation): Seq[ExecPlan] = { + lp: ScalarVectorBinaryOperation): PlanResult = { val vectors = walkLogicalPlanTree(lp.vector, queryId, submitTime, options) - vectors.foreach(_.addRangeVectorTransformer(ScalarOperationMapper(lp.operator, lp.scalar, lp.scalarIsLhs))) + vectors.plans.foreach(_.addRangeVectorTransformer(ScalarOperationMapper(lp.operator, lp.scalar, lp.scalarIsLhs))) vectors } private def materializeBinaryJoin(queryId: String, submitTime: Long, options: QueryOptions, - lp: BinaryJoin): Seq[ExecPlan] = { + lp: BinaryJoin): PlanResult = { val lhs = walkLogicalPlanTree(lp.lhs, queryId, submitTime, options) - val rhs = walkLogicalPlanTree(lp.rhs, queryId, submitTime, options) - val targetActor = pickDispatcher(lhs ++ rhs) - Seq(BinaryJoinExec(queryId, targetActor, lhs, rhs, lp.operator, lp.cardinality, lp.on, lp.ignoring)) + val stitchedLhs = if (lhs.needsStitch) Seq(StitchRvsExec(queryId, pickDispatcher(lhs.plans), lhs.plans)) + else lhs.plans + val rhs = walkLogicalPlanTree(lp.rhs, queryId, submitTime, options) + val stitchedRhs = if (rhs.needsStitch) Seq(StitchRvsExec(queryId, 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 + // and transform child results individually before composing child results together. + // In theory, more efficient to use transformer than to have separate exec plan node to avoid IO. + // In the interest of keeping it simple, deferring decorations to the ExecPlan. Add only if needed after measuring. + + val targetActor = pickDispatcher(stitchedLhs ++ stitchedRhs) + val joined = Seq(BinaryJoinExec(queryId, targetActor, stitchedLhs, stitchedRhs, lp.operator, lp.cardinality, + lp.on, lp.ignoring)) + PlanResult(joined, false) } private def materializeAggregate(queryId: String, submitTime: Long, options: QueryOptions, - lp: Aggregate): Seq[ExecPlan] = { - val toReduce = walkLogicalPlanTree(lp.vectors, queryId, submitTime, options) // Now we have one exec plan per shard - toReduce.foreach(_.addRangeVectorTransformer(AggregateMapReduce(lp.operator, lp.params, lp.without, lp.by))) + lp: Aggregate): PlanResult = { + val toReduce = walkLogicalPlanTree(lp.vectors, queryId, submitTime, options) + // 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, + * one of the following must happen + * 1. Step instants must be chosen so time windows dont span shards. + * 2. We pump data into multiple shards for sometime so atleast one shard will fully contain any time window + * + * Pulling all data into one node and stich before reducing (not feasible, doesnt scale). So we will + * not stitch + * + * Starting off with solution 1 first until (2) or some other approach is decided on. + */ + toReduce.plans.foreach(_.addRangeVectorTransformer(AggregateMapReduce(lp.operator, lp.params, lp.without, lp.by))) // One could do another level of aggregation per node too. Ignoring for now - val reduceDispatcher = pickDispatcher(toReduce) - val reducer = ReduceAggregateExec(queryId, reduceDispatcher, toReduce, lp.operator, lp.params) + val reduceDispatcher = pickDispatcher(toReduce.plans) + val reducer = ReduceAggregateExec(queryId, reduceDispatcher, toReduce.plans, lp.operator, lp.params) reducer.addRangeVectorTransformer(AggregatePresenter(lp.operator, lp.params)) - Seq(reducer) + PlanResult(Seq(reducer), false) // since we have aggregated, no stitching } private def materializeApplyInstantFunction(queryId: String, submitTime: Long, options: QueryOptions, - lp: ApplyInstantFunction): Seq[ExecPlan] = { + lp: ApplyInstantFunction): PlanResult = { val vectors = walkLogicalPlanTree(lp.vectors, queryId, submitTime, options) - vectors.foreach(_.addRangeVectorTransformer(InstantVectorFunctionMapper(lp.function, lp.functionArgs))) + vectors.plans.foreach(_.addRangeVectorTransformer(InstantVectorFunctionMapper(lp.function, lp.functionArgs))) vectors } private def materializePeriodicSeriesWithWindowing(queryId: String, submitTime: Long, options: QueryOptions, - lp: PeriodicSeriesWithWindowing): Seq[ExecPlan] = { + lp: PeriodicSeriesWithWindowing): PlanResult ={ val rawSeries = walkLogicalPlanTree(lp.rawSeries, queryId, submitTime, options) - rawSeries.foreach(_.addRangeVectorTransformer(PeriodicSamplesMapper(lp.start, lp.step, + rawSeries.plans.foreach(_.addRangeVectorTransformer(PeriodicSamplesMapper(lp.start, lp.step, lp.end, Some(lp.window), Some(lp.function), lp.functionArgs))) rawSeries } @@ -181,9 +217,9 @@ class QueryEngine(dataset: Dataset, private def materializePeriodicSeries(queryId: String, submitTime: Long, options: QueryOptions, - lp: PeriodicSeries): Seq[ExecPlan] = { + lp: PeriodicSeries): PlanResult = { val rawSeries = walkLogicalPlanTree(lp.rawSeries, queryId, submitTime, options) - rawSeries.foreach(_.addRangeVectorTransformer(PeriodicSamplesMapper(lp.start, lp.step, lp.end, + rawSeries.plans.foreach(_.addRangeVectorTransformer(PeriodicSamplesMapper(lp.start, lp.step, lp.end, None, None, Nil))) rawSeries } @@ -191,20 +227,26 @@ class QueryEngine(dataset: Dataset, private def materializeRawSeries(queryId: String, submitTime: Long, options: QueryOptions, - lp: RawSeries): Seq[ExecPlan] = { + lp: RawSeries): PlanResult = { val colIDs = getColumnIDs(dataset, lp.columns) val renamedFilters = renameMetricFilter(lp.filters) - shardsFromFilters(renamedFilters, options).map { shard => + val spreadChanges = options.spreadFunc(renamedFilters) + val needsStitch = lp.rangeSelector match { + case IntervalSelector(from, to) => spreadChanges.exists(c => c.time >= from && c.time <= to) + case _ => false + } + val execPlans = shardsFromFilters(renamedFilters, options).map { shard => val dispatcher = dispatcherForShard(shard) SelectRawPartitionsExec(queryId, submitTime, options.sampleLimit, dispatcher, dataset.ref, shard, renamedFilters, toRowKeyRange(lp.rangeSelector), colIDs) } + PlanResult(execPlans, needsStitch) } private def materializeLabelValues(queryId: String, submitTime: Long, options: QueryOptions, - lp: LabelValues): Seq[LabelValuesExec] = { + lp: LabelValues): PlanResult = { val filters = lp.labelConstraints.map { case (k, v) => new ColumnFilter(k, Filter.Equals(v)) }.toSeq @@ -220,17 +262,18 @@ class QueryEngine(dataset: Dataset, mdNoShardKeyFilterRequests.increment() shardMapperFunc.assignedShards } - shardsToHit.map { shard => + val metaExec = shardsToHit.map { shard => val dispatcher = dispatcherForShard(shard) exec.LabelValuesExec(queryId, submitTime, options.sampleLimit, dispatcher, dataset.ref, shard, filters, labelNames, lp.lookbackTimeInMillis) } + PlanResult(metaExec, false) } private def materializeSeriesKeysByFilters(queryId: String, submitTime: Long, options: QueryOptions, - lp: SeriesKeysByFilters): Seq[PartKeysExec] = { + lp: SeriesKeysByFilters): PlanResult = { val renamedFilters = renameMetricFilter(lp.filters) val filterCols = lp.filters.map(_.column).toSet val shardsToHit = if (shardColumns.toSet.subsetOf(filterCols)) { @@ -239,26 +282,28 @@ class QueryEngine(dataset: Dataset, mdNoShardKeyFilterRequests.increment() shardMapperFunc.assignedShards } - shardsToHit.map { shard => + val metaExec = shardsToHit.map { shard => val dispatcher = dispatcherForShard(shard) PartKeysExec(queryId, submitTime, options.sampleLimit, dispatcher, dataset.ref, shard, renamedFilters, lp.start, lp.end) } + PlanResult(metaExec, false) } private def materializeRawChunkMeta(queryId: String, submitTime: Long, options: QueryOptions, - lp: RawChunkMeta): Seq[ExecPlan] = { + lp: RawChunkMeta): PlanResult = { // Translate column name to ID and validate here val colName = if (lp.column.isEmpty) dataset.options.valueColumn else lp.column val colID = dataset.colIDs(colName).get.head val renamedFilters = renameMetricFilter(lp.filters) - shardsFromFilters(renamedFilters, options).map { shard => + val metaExec = shardsFromFilters(renamedFilters, options).map { shard => val dispatcher = dispatcherForShard(shard) SelectChunkInfosExec(queryId, submitTime, options.sampleLimit, dispatcher, dataset.ref, shard, renamedFilters, toRowKeyRange(lp.rangeSelector), colID) } + PlanResult(metaExec, false) } /** @@ -291,8 +336,8 @@ class QueryEngine(dataset: Dataset, private def toRowKeyRange(rangeSelector: RangeSelector): RowKeyRange = { rangeSelector match { - case IntervalSelector(from, to) => RowKeyInterval(BinaryRecord(dataset, from), - BinaryRecord(dataset, to)) + case IntervalSelector(from, to) => RowKeyInterval(BinaryRecord(dataset, Seq(from)), + BinaryRecord(dataset, Seq(to))) case AllChunksSelector => AllChunks case EncodedChunksSelector => EncodedChunks case WriteBufferSelector => WriteBuffers diff --git a/coordinator/src/test/scala/filodb.coordinator/client/SerializationSpec.scala b/coordinator/src/test/scala/filodb.coordinator/client/SerializationSpec.scala index 24a8d45642..86eb14a8c6 100644 --- a/coordinator/src/test/scala/filodb.coordinator/client/SerializationSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/client/SerializationSpec.scala @@ -189,7 +189,7 @@ class SerializationSpec extends ActorTest(SerializationSpecConfig.getNewSystem) val to = System.currentTimeMillis() val from = to - 50000 - val intervalSelector = IntervalSelector(Seq(from), Seq(to)) + val intervalSelector = IntervalSelector(from, to) val raw1 = RawSeries(rangeSelector = intervalSelector, filters= f1, columns = Seq("value")) val windowed1 = PeriodicSeriesWithWindowing(raw1, from, 1000, to, 5000, RangeFunctionId.Rate) diff --git a/coordinator/src/test/scala/filodb.coordinator/queryengine2/QueryEngineSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryengine2/QueryEngineSpec.scala index dd746368a1..be2000da6d 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryengine2/QueryEngineSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryengine2/QueryEngineSpec.scala @@ -5,9 +5,11 @@ import akka.testkit.TestProbe import org.scalatest.{FunSpec, Matchers} import filodb.coordinator.ShardMapper -import filodb.coordinator.client.QueryCommands.QueryOptions +import filodb.coordinator.client.QueryCommands.{QueryOptions, SpreadChange} import filodb.core.MetricsTestData import filodb.core.query.{ColumnFilter, Filter} +import filodb.prometheus.ast.TimeStepParams +import filodb.prometheus.parse.Parser import filodb.query._ import filodb.query.exec._ @@ -46,7 +48,7 @@ class QueryEngineSpec extends FunSpec with Matchers { val to = System.currentTimeMillis() val from = to - 50000 - val intervalSelector = IntervalSelector(Seq(from), Seq(to)) + val intervalSelector = IntervalSelector(from, to) val raw1 = RawSeries(rangeSelector = intervalSelector, filters= f1, columns = Seq("value")) val windowed1 = PeriodicSeriesWithWindowing(raw1, from, 1000, to, 5000, RangeFunctionId.Rate) @@ -140,4 +142,48 @@ class QueryEngineSpec extends FunSpec with Matchers { reduceAggPlan.children should have length (4) // spread=2 means 4 shards } } + + it("should stitch results when spread changes during query range") { + val lp = Parser.queryRangeToLogicalPlan("""foo{job="bar"}""", TimeStepParams(20000, 100, 30000)) + def spread(filter: Seq[ColumnFilter]): Seq[SpreadChange] = { + Seq(SpreadChange(0, 1), SpreadChange(25000000, 2)) // spread change time is in ms + } + val execPlan = engine.materialize(lp, QueryOptions(spread)) + execPlan.rangeVectorTransformers.head.isInstanceOf[StitchRvsMapper] shouldEqual true + } + + it("should not stitch results when spread has not changed in query range") { + val lp = Parser.queryRangeToLogicalPlan("""foo{job="bar"}""", TimeStepParams(20000, 100, 30000)) + def spread(filter: Seq[ColumnFilter]): Seq[SpreadChange] = { + Seq(SpreadChange(0, 1), SpreadChange(35000000, 2)) + } + val execPlan = engine.materialize(lp, QueryOptions(spread)) + execPlan.rangeVectorTransformers.isEmpty shouldEqual true + } + + it("should stitch results before binary join when spread changed in query range") { + val lp = Parser.queryRangeToLogicalPlan("""count(foo{job="bar"} + baz{job="bar"})""", + TimeStepParams(20000, 100, 30000)) + def spread(filter: Seq[ColumnFilter]): Seq[SpreadChange] = { + Seq(SpreadChange(0, 1), SpreadChange(25000000, 2)) + } + val execPlan = engine.materialize(lp, QueryOptions(spread)) + val binaryJoinNode = execPlan.children(0) + binaryJoinNode.isInstanceOf[BinaryJoinExec] shouldEqual true + binaryJoinNode.children.size shouldEqual 2 + binaryJoinNode.children.foreach(_.isInstanceOf[StitchRvsExec] shouldEqual true) + } + + it("should not stitch results before binary join when spread has not changed in query range") { + val lp = Parser.queryRangeToLogicalPlan("""count(foo{job="bar"} + baz{job="bar"})""", + TimeStepParams(20000, 100, 30000)) + def spread(filter: Seq[ColumnFilter]): Seq[SpreadChange] = { + Seq(SpreadChange(0, 1), SpreadChange(35000000, 2)) + } + val execPlan = engine.materialize(lp, QueryOptions(spread)) + val binaryJoinNode = execPlan.children(0) + binaryJoinNode.isInstanceOf[BinaryJoinExec] shouldEqual true + binaryJoinNode.children.foreach(_.isInstanceOf[StitchRvsExec] should not equal true) + } + } diff --git a/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala b/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala index 7d4375984b..ca92b0c700 100644 --- a/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala +++ b/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala @@ -94,7 +94,7 @@ TimeSeriesShard(dataset, storeConfig, shardNum, rawStore, metastore, evictionPol // NOTE: this executes the partMaker single threaded. Needed for now due to concurrency constraints. // In the future optimize this if needed. .mapAsync { rawPart => partitionMaker.populateRawChunks(rawPart).executeOn(singleThreadPool) } - .doOnComplete(() => span.finish()) + .doOnTerminate(ex => span.finish()) // This is needed so future computations happen in a different thread .asyncBoundary(strategy) } else { Observable.empty } @@ -111,7 +111,7 @@ TimeSeriesShard(dataset, storeConfig, shardNum, rawStore, metastore, evictionPol .defaultIfEmpty(getPartition(partBytes).get) .headL } - .doOnComplete(() => span.finish()) + .doOnTerminate(ex => span.finish()) } else { Observable.empty } diff --git a/core/src/main/scala/filodb.core/query/RangeVector.scala b/core/src/main/scala/filodb.core/query/RangeVector.scala index 80aca5eea8..bb246d80d0 100644 --- a/core/src/main/scala/filodb.core/query/RangeVector.scala +++ b/core/src/main/scala/filodb.core/query/RangeVector.scala @@ -14,7 +14,9 @@ import filodb.memory.data.ChunkMap import filodb.memory.format.{RowReader, ZeroCopyUTF8String => UTF8Str} /** - * Identifier for a single RangeVector + * Identifier for a single RangeVector. + * Sub-classes must be a case class or override equals/hashcode since this class is used in a + * hash table. */ trait RangeVectorKey extends java.io.Serializable { def labelValues: Map[UTF8Str, UTF8Str] diff --git a/gateway/src/main/scala/filodb/timeseries/TestTimeseriesProducer.scala b/gateway/src/main/scala/filodb/timeseries/TestTimeseriesProducer.scala index 0e13907f36..ddcfba0c7d 100644 --- a/gateway/src/main/scala/filodb/timeseries/TestTimeseriesProducer.scala +++ b/gateway/src/main/scala/filodb/timeseries/TestTimeseriesProducer.scala @@ -100,17 +100,20 @@ object TestTimeseriesProducer extends StrictLogging { s"""--promql 'heap_usage{dc="DC0",_ns="App-0"}' --start $startQuery --end $endQuery --limit 15""" logger.info(s"Periodic Samples CLI Query : \n$query") - val q = URLEncoder.encode("""heap_usage{dc="DC0",_ns="App-0"}[2m]""", StandardCharsets.UTF_8.toString) + val periodQuery = URLEncoder.encode("""heap_usage{dc="DC0",_ns="App-0"}""", StandardCharsets.UTF_8.toString) val periodicSamplesUrl = s"http://localhost:8080/promql/prometheus/api/v1/query_range?" + - s"query=$q&start=$startQuery&end=$endQuery&step=15" + s"query=$periodQuery&start=$startQuery&end=$endQuery&step=15" logger.info(s"Periodic Samples query URL: \n$periodicSamplesUrl") - val q2 = URLEncoder.encode("""heap_usage{dc="DC0",_ns="App-0",__col__="sum"}[2m]""", + val rawQuery = URLEncoder.encode("""heap_usage{dc="DC0",_ns="App-0"}[2m]""", StandardCharsets.UTF_8.toString) - val rawSamplesUrl = s"http://localhost:8080/promql/prometheus/api/v1/query?query=$q2&time=$endQuery" + val rawSamplesUrl = s"http://localhost:8080/promql/prometheus/api/v1/query?query=$rawQuery&time=$endQuery" logger.info(s"Raw Samples query URL: \n$rawSamplesUrl") + + val downsampleSumQuery = URLEncoder.encode("""heap_usage{dc="DC0",_ns="App-0",__col__="sum"}[2m]""", + StandardCharsets.UTF_8.toString) val downsampledSamplesUrl = s"http://localhost:8080/promql/prometheus_ds_1m/api/v1/query?" + - s"query=$q2&time=$endQuery" + s"query=$downsampleSumQuery&time=$endQuery" logger.info(s"Downsampled Samples query URL: \n$downsampledSamplesUrl") } diff --git a/http/src/main/scala/filodb/http/PrometheusApiRoute.scala b/http/src/main/scala/filodb/http/PrometheusApiRoute.scala index 085d87f4fe..37cb4427ca 100644 --- a/http/src/main/scala/filodb/http/PrometheusApiRoute.scala +++ b/http/src/main/scala/filodb/http/PrometheusApiRoute.scala @@ -14,7 +14,7 @@ import org.xerial.snappy.Snappy import remote.RemoteStorage.ReadRequest import filodb.coordinator.client.IngestionCommands.UnknownDataset -import filodb.coordinator.client.QueryCommands.{LogicalPlan2Query, QueryOptions} +import filodb.coordinator.client.QueryCommands.{LogicalPlan2Query, QueryOptions, SpreadChange} import filodb.core.DatasetRef import filodb.prometheus.ast.TimeStepParams import filodb.prometheus.parse.Parser @@ -29,7 +29,7 @@ class PrometheusApiRoute(nodeCoord: ActorRef, settings: HttpSettings)(implicit a import filodb.coordinator.client.Client._ import filodb.prometheus.query.PrometheusModel._ - val queryOptions = QueryOptions(spreadFunc = { _ => settings.queryDefaultSpread }, + val queryOptions = QueryOptions(spreadFunc = { _ => Seq(SpreadChange(settings.queryDefaultSpread)) }, sampleLimit = settings.querySampleLimit) val route = pathPrefix( "promql" / Segment) { dataset => diff --git a/prometheus/src/main/scala/filodb/prometheus/ast/Base.scala b/prometheus/src/main/scala/filodb/prometheus/ast/Base.scala index 82ad8a6891..ca56ddc6c6 100644 --- a/prometheus/src/main/scala/filodb/prometheus/ast/Base.scala +++ b/prometheus/src/main/scala/filodb/prometheus/ast/Base.scala @@ -44,7 +44,7 @@ trait Base { * timeParam.start is in seconds, startOffset is in millis */ def timeParamToSelector(timeParam: TimeRangeParams, startOffset: Long): RangeSelector = timeParam match { - case TimeStepParams(start, step, end) => IntervalSelector(Seq(start * 1000 - startOffset), Seq(end * 1000)) + case TimeStepParams(start, step, end) => IntervalSelector(start * 1000 - startOffset, end * 1000) case InMemoryParam(_) => InMemoryChunksSelector case WriteBuffersParam(_) => WriteBufferSelector } diff --git a/prometheus/src/main/scala/filodb/prometheus/parse/Parser.scala b/prometheus/src/main/scala/filodb/prometheus/parse/Parser.scala index 4b42315b70..1bfaff3537 100644 --- a/prometheus/src/main/scala/filodb/prometheus/parse/Parser.scala +++ b/prometheus/src/main/scala/filodb/prometheus/parse/Parser.scala @@ -325,9 +325,8 @@ object Parser extends Expression { } def queryToLogicalPlan(query: String, queryTimestamp: Long): LogicalPlan = { - // step does not matter here in instant query. - // Since we have some min validation on the server, pass Long.MaxValue / 1000 seconds for step. - val defaultQueryParams = TimeStepParams(queryTimestamp, Long.MaxValue / 1000, queryTimestamp) + // step does not matter here in instant query - just use a dummy value more than minStep + val defaultQueryParams = TimeStepParams(queryTimestamp, 1000, queryTimestamp) queryRangeToLogicalPlan(query, defaultQueryParams) } diff --git a/prometheus/src/main/scala/filodb/prometheus/query/PrometheusModel.scala b/prometheus/src/main/scala/filodb/prometheus/query/PrometheusModel.scala index b79ce007e8..99be888bba 100644 --- a/prometheus/src/main/scala/filodb/prometheus/query/PrometheusModel.scala +++ b/prometheus/src/main/scala/filodb/prometheus/query/PrometheusModel.scala @@ -32,7 +32,7 @@ object PrometheusModel { def toFiloDBLogicalPlans(readRequest: ReadRequest): Seq[LogicalPlan] = { for { i <- 0 until readRequest.getQueriesCount } yield { val q = readRequest.getQueries(i) - val interval = IntervalSelector(Seq(q.getStartTimestampMs), Seq(q.getEndTimestampMs)) + val interval = IntervalSelector(q.getStartTimestampMs, q.getEndTimestampMs) val filters = for { j <- 0 until q.getMatchersCount } yield { val m = q.getMatchers(j) val filter = m.getType match { diff --git a/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala b/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala index d18ef6a7d4..2144274f00 100644 --- a/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala +++ b/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala @@ -245,83 +245,83 @@ class ParserSpec extends FunSpec with Matchers { it("Should be able to make logical plans for Series Expressions") { val queryToLpString = Map( "primary:instance-001:no.ofrequests{job=\"my-job\"}" -> - "PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(job,Equals(my-job)), ColumnFilter(__name__,Equals(primary:instance-001:no.ofrequests))),List()),1524855988000,9223372036854775000,1524855988000)", + "PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(job,Equals(my-job)), ColumnFilter(__name__,Equals(primary:instance-001:no.ofrequests))),List()),1524855988000,1000000,1524855988000)", "absent(nonexistent{job=\"myjob\"})" -> - "ApplyInstantFunction(PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(job,Equals(myjob)), ColumnFilter(__name__,Equals(nonexistent))),List()),1524855988000,9223372036854775000,1524855988000),Absent,List())", + "ApplyInstantFunction(PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(job,Equals(myjob)), ColumnFilter(__name__,Equals(nonexistent))),List()),1524855988000,1000000,1524855988000),Absent,List())", "rate(http_requests_total[5m] offset 1w)" -> - "PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,9223372036854775000,1524855988000,300000,Rate,List())", + "PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000,300000,Rate,List())", "http_requests_total{job=\"prometheus\",group=\"canary\"}" -> - "PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(job,Equals(prometheus)), ColumnFilter(group,Equals(canary)), ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,9223372036854775000,1524855988000)", + "PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(job,Equals(prometheus)), ColumnFilter(group,Equals(canary)), ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000)", "http_requests_total{job=\"prometheus\",__col__=\"min\"}" -> - "PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(job,Equals(prometheus)), ColumnFilter(__name__,Equals(http_requests_total))),List(min)),1524855988000,9223372036854775000,1524855988000)", + "PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(job,Equals(prometheus)), ColumnFilter(__name__,Equals(http_requests_total))),List(min)),1524855988000,1000000,1524855988000)", // Internal FiloDB debug function "_filodb_chunkmeta_all(http_requests_total{job=\"prometheus\"})" -> - "RawChunkMeta(IntervalSelector(List(1524855988000),List(1524855988000)),List(ColumnFilter(job,Equals(prometheus)), ColumnFilter(__name__,Equals(http_requests_total))),)", + "RawChunkMeta(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(job,Equals(prometheus)), ColumnFilter(__name__,Equals(http_requests_total))),)", "_filodb_chunkmeta_all(http_requests_total{job=\"prometheus\",__col__=\"avg\"})" -> - "RawChunkMeta(IntervalSelector(List(1524855988000),List(1524855988000)),List(ColumnFilter(job,Equals(prometheus)), ColumnFilter(__name__,Equals(http_requests_total))),avg)", + "RawChunkMeta(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(job,Equals(prometheus)), ColumnFilter(__name__,Equals(http_requests_total))),avg)", "sum(http_requests_total) by (application, group)" -> - "Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,9223372036854775000,1524855988000),List(),List(application, group),List())", + "Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000),List(),List(application, group),List())", "sum(http_requests_total) without (instance)" -> - "Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,9223372036854775000,1524855988000),List(),List(),List(instance))", + "Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000),List(),List(),List(instance))", "count_values(\"version\", build_version)" -> - "Aggregate(CountValues,PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(__name__,Equals(build_version))),List()),1524855988000,9223372036854775000,1524855988000),List(\"version\"),List(),List())", + "Aggregate(CountValues,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(build_version))),List()),1524855988000,1000000,1524855988000),List(\"version\"),List(),List())", "label_replace(up{job=\"api-server\",service=\"a:c\"}, \"foo\", \"$1\", \"service\", \"(.*):.*\")" -> - "ApplyInstantFunction(PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(job,Equals(api-server)), ColumnFilter(service,Equals(a:c)), ColumnFilter(__name__,Equals(up))),List()),1524855988000,9223372036854775000,1524855988000),LabelReplace,List())", + "ApplyInstantFunction(PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(job,Equals(api-server)), ColumnFilter(service,Equals(a:c)), ColumnFilter(__name__,Equals(up))),List()),1524855988000,1000000,1524855988000),LabelReplace,List())", "sum(http_requests_total)" -> - "Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,9223372036854775000,1524855988000),List(),List(),List())", + "Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000),List(),List(),List())", "label_join(up{job=\"api-server\",src1=\"a\",src2=\"b\",src3=\"c\"}, \"foo\", \",\", \"src1\", \"src2\", \"src3\")" -> - "ApplyInstantFunction(PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(job,Equals(api-server)), ColumnFilter(src1,Equals(a)), ColumnFilter(src2,Equals(b)), ColumnFilter(src3,Equals(c)), ColumnFilter(__name__,Equals(up))),List()),1524855988000,9223372036854775000,1524855988000),LabelJoin,List())", + "ApplyInstantFunction(PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(job,Equals(api-server)), ColumnFilter(src1,Equals(a)), ColumnFilter(src2,Equals(b)), ColumnFilter(src3,Equals(c)), ColumnFilter(__name__,Equals(up))),List()),1524855988000,1000000,1524855988000),LabelJoin,List())", "histogram_quantile(0.9, sum(rate(http_request_duration_seconds_bucket[10m])) by (le))" -> - "ApplyInstantFunction(Aggregate(Sum,PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(List(1524855388000),List(1524855988000)),List(ColumnFilter(__name__,Equals(http_request_duration_seconds_bucket))),List()),1524855988000,9223372036854775000,1524855988000,600000,Rate,List()),List(),List(le),List()),HistogramQuantile,List(0.9))", + "ApplyInstantFunction(Aggregate(Sum,PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(1524855388000,1524855988000),List(ColumnFilter(__name__,Equals(http_request_duration_seconds_bucket))),List()),1524855988000,1000000,1524855988000,600000,Rate,List()),List(),List(le),List()),HistogramQuantile,List(0.9))", "delta(cpu_temp_celsius{host=\"zeus\"}[2h])" -> - "PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(List(1524848788000),List(1524855988000)),List(ColumnFilter(host,Equals(zeus)), ColumnFilter(__name__,Equals(cpu_temp_celsius))),List()),1524855988000,9223372036854775000,1524855988000,7200000,Delta,List())", + "PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(1524848788000,1524855988000),List(ColumnFilter(host,Equals(zeus)), ColumnFilter(__name__,Equals(cpu_temp_celsius))),List()),1524855988000,1000000,1524855988000,7200000,Delta,List())", "method_code:http_errors:rate5m{code=\"500\"} / ignoring(code) method:http_requests:rate5m" -> - "BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(code,Equals(500)), ColumnFilter(__name__,Equals(method_code:http_errors:rate5m))),List()),1524855988000,9223372036854775000,1524855988000),DIV,OneToOne,PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(__name__,Equals(method:http_requests:rate5m))),List()),1524855988000,9223372036854775000,1524855988000),List(),List())", + "BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(code,Equals(500)), ColumnFilter(__name__,Equals(method_code:http_errors:rate5m))),List()),1524855988000,1000000,1524855988000),DIV,OneToOne,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(method:http_requests:rate5m))),List()),1524855988000,1000000,1524855988000),List(),List())", "histogram_quantile(0.9, rate(http_request_duration_seconds_bucket[10m]))" -> - "ApplyInstantFunction(PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(List(1524855388000),List(1524855988000)),List(ColumnFilter(__name__,Equals(http_request_duration_seconds_bucket))),List()),1524855988000,9223372036854775000,1524855988000,600000,Rate,List()),HistogramQuantile,List(0.9))", + "ApplyInstantFunction(PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(1524855388000,1524855988000),List(ColumnFilter(__name__,Equals(http_request_duration_seconds_bucket))),List()),1524855988000,1000000,1524855988000,600000,Rate,List()),HistogramQuantile,List(0.9))", "histogram_quantile(0.9, sum(rate(http_request_duration_seconds_bucket[10m])) by (job, le))" -> - "ApplyInstantFunction(Aggregate(Sum,PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(List(1524855388000),List(1524855988000)),List(ColumnFilter(__name__,Equals(http_request_duration_seconds_bucket))),List()),1524855988000,9223372036854775000,1524855988000,600000,Rate,List()),List(),List(job, le),List()),HistogramQuantile,List(0.9))", + "ApplyInstantFunction(Aggregate(Sum,PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(1524855388000,1524855988000),List(ColumnFilter(__name__,Equals(http_request_duration_seconds_bucket))),List()),1524855988000,1000000,1524855988000,600000,Rate,List()),List(),List(job, le),List()),HistogramQuantile,List(0.9))", "http_requests_total" -> - "PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,9223372036854775000,1524855988000)", + "PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000)", "http_requests_total ^ 5" -> - "ScalarVectorBinaryOperation(POW,5.0,PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,9223372036854775000,1524855988000),false)", + "ScalarVectorBinaryOperation(POW,5.0,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000),false)", //FIXME Operator precedence is not implemented "10 + http_requests_total * 5" -> - "ScalarVectorBinaryOperation(ADD,10.0,ScalarVectorBinaryOperation(MUL,5.0,PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,9223372036854775000,1524855988000),false),true)", + "ScalarVectorBinaryOperation(ADD,10.0,ScalarVectorBinaryOperation(MUL,5.0,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000),false),true)", "10 + (http_requests_total * 5)" -> - "ScalarVectorBinaryOperation(ADD,10.0,ScalarVectorBinaryOperation(MUL,5.0,PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,9223372036854775000,1524855988000),false),true)", + "ScalarVectorBinaryOperation(ADD,10.0,ScalarVectorBinaryOperation(MUL,5.0,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000),false),true)", "(10 + http_requests_total) * 5" -> - "ScalarVectorBinaryOperation(MUL,5.0,ScalarVectorBinaryOperation(ADD,10.0,PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,9223372036854775000,1524855988000),true),false)", + "ScalarVectorBinaryOperation(MUL,5.0,ScalarVectorBinaryOperation(ADD,10.0,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000),true),false)", "topk(5, http_requests_total)" -> - "Aggregate(TopK,PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,9223372036854775000,1524855988000),List(5.0),List(),List())", + "Aggregate(TopK,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000),List(5.0),List(),List())", "irate(http_requests_total{job=\"api-server\"}[5m])" -> - "PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(job,Equals(api-server)), ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,9223372036854775000,1524855988000,300000,Irate,List())", + "PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(job,Equals(api-server)), ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000,300000,Irate,List())", "idelta(http_requests_total{job=\"api-server\"}[5m])" -> - "PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(job,Equals(api-server)), ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,9223372036854775000,1524855988000,300000,Idelta,List())", + "PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(job,Equals(api-server)), ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000,300000,Idelta,List())", "resets(http_requests_total{job=\"api-server\"}[5m])" -> - "PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(job,Equals(api-server)), ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,9223372036854775000,1524855988000,300000,Resets,List())", + "PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(job,Equals(api-server)), ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000,300000,Resets,List())", "deriv(http_requests_total{job=\"api-server\"}[5m])" -> - "PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(job,Equals(api-server)), ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,9223372036854775000,1524855988000,300000,Deriv,List())", + "PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(job,Equals(api-server)), ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000,300000,Deriv,List())", "rate(http_requests_total{job=\"api-server\"}[5m])" -> - "PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(job,Equals(api-server)), ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,9223372036854775000,1524855988000,300000,Rate,List())", + "PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(job,Equals(api-server)), ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000,300000,Rate,List())", "http_requests_total{job=\"prometheus\"}[5m]" -> - "RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(job,Equals(prometheus)), ColumnFilter(__name__,Equals(http_requests_total))),List())", + "RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(job,Equals(prometheus)), ColumnFilter(__name__,Equals(http_requests_total))),List())", "http_requests_total offset 5m" -> - "PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,9223372036854775000,1524855988000)", + "PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000)", "http_requests_total{environment=~\"staging|testing|development\",method!=\"GET\"}" -> - "PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(environment,EqualsRegex(staging|testing|development)), ColumnFilter(method,NotEquals(GET)), ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,9223372036854775000,1524855988000)", + "PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(environment,EqualsRegex(staging|testing|development)), ColumnFilter(method,NotEquals(GET)), ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000)", "method_code:http_errors:rate5m / ignoring(code) group_left method:http_requests:rate5m" -> - "BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(__name__,Equals(method_code:http_errors:rate5m))),List()),1524855988000,9223372036854775000,1524855988000),DIV,OneToMany,PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(__name__,Equals(method:http_requests:rate5m))),List()),1524855988000,9223372036854775000,1524855988000),List(),List())", + "BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(method_code:http_errors:rate5m))),List()),1524855988000,1000000,1524855988000),DIV,OneToMany,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(method:http_requests:rate5m))),List()),1524855988000,1000000,1524855988000),List(),List())", "increase(http_requests_total{job=\"api-server\"}[5m])" -> - "PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(job,Equals(api-server)), ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,9223372036854775000,1524855988000,300000,Increase,List())", + "PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(job,Equals(api-server)), ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000,300000,Increase,List())", "sum(http_requests_total{method=\"GET\"} offset 5m)" -> - "Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(method,Equals(GET)), ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,9223372036854775000,1524855988000),List(),List(),List())", + "Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(method,Equals(GET)), ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000),List(),List(),List())", "absent(nonexistent{job=\"myjob\",instance=~\".*\"})" -> - "ApplyInstantFunction(PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(job,Equals(myjob)), ColumnFilter(instance,EqualsRegex(.*)), ColumnFilter(__name__,Equals(nonexistent))),List()),1524855988000,9223372036854775000,1524855988000),Absent,List())", + "ApplyInstantFunction(PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(job,Equals(myjob)), ColumnFilter(instance,EqualsRegex(.*)), ColumnFilter(__name__,Equals(nonexistent))),List()),1524855988000,1000000,1524855988000),Absent,List())", "absent(sum(nonexistent{job=\"myjob\"}))" -> - "ApplyInstantFunction(Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(List(1524855688000),List(1524855988000)),List(ColumnFilter(job,Equals(myjob)), ColumnFilter(__name__,Equals(nonexistent))),List()),1524855988000,9223372036854775000,1524855988000),List(),List(),List()),Absent,List())" + "ApplyInstantFunction(Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(job,Equals(myjob)), ColumnFilter(__name__,Equals(nonexistent))),List()),1524855988000,1000000,1524855988000),List(),List(),List()),Absent,List())" ) 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 b1b7164bed..c449249a7e 100644 --- a/query/src/main/scala/filodb/query/LogicalPlan.scala +++ b/query/src/main/scala/filodb/query/LogicalPlan.scala @@ -26,7 +26,7 @@ case object AllChunksSelector extends RangeSelector case object WriteBufferSelector extends RangeSelector case object InMemoryChunksSelector extends RangeSelector case object EncodedChunksSelector extends RangeSelector -case class IntervalSelector(from: Seq[Any], to: Seq[Any]) extends RangeSelector +case class IntervalSelector(from: Long, to: Long) extends RangeSelector /** * Concrete logical plan to query for raw data in a given range diff --git a/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala b/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala index 68f37a8c5b..b480daa54a 100644 --- a/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala +++ b/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala @@ -65,8 +65,8 @@ final case class BinaryJoinExec(id: String, case (QueryError(_, ex), _) => throw ex }.toListL.map { resp => require(resp.size == lhs.size + rhs.size, "Did not get sufficient responses for LHS and RHS") - val lhsRvs = resp.filter(_._2 < lhs.size).map(_._1).flatten - val rhsRvs = resp.filter(_._2 >= lhs.size).map(_._1).flatten + val lhsRvs = resp.filter(_._2 < lhs.size).flatMap(_._1) + val rhsRvs = resp.filter(_._2 >= lhs.size).flatMap(_._1) // figure out which side is the "one" side val (oneSide, otherSide, lhsIsOneSide) = diff --git a/query/src/main/scala/filodb/query/exec/DistConcatExec.scala b/query/src/main/scala/filodb/query/exec/DistConcatExec.scala index 9795d50f7d..25afaa5b69 100644 --- a/query/src/main/scala/filodb/query/exec/DistConcatExec.scala +++ b/query/src/main/scala/filodb/query/exec/DistConcatExec.scala @@ -2,27 +2,18 @@ package filodb.query.exec import monix.reactive.Observable -import filodb.core.binaryrecord.BinaryRecord import filodb.core.metadata.Dataset import filodb.core.query._ import filodb.query._ import filodb.query.Query.qLogger -sealed trait RowKeyRange - -case class RowKeyInterval(from: BinaryRecord, to: BinaryRecord) extends RowKeyRange -case object AllChunks extends RowKeyRange -case object WriteBuffers extends RowKeyRange -case object InMemoryChunks extends RowKeyRange -case object EncodedChunks extends RowKeyRange - /** * Simply concatenate results from child ExecPlan objects */ final case class DistConcatExec(id: String, dispatcher: PlanDispatcher, children: Seq[ExecPlan]) extends NonLeafExecPlan { - require(!children.isEmpty) + require(children.nonEmpty) protected def args: String = "" diff --git a/query/src/main/scala/filodb/query/exec/HistogramQuantileMapper.scala b/query/src/main/scala/filodb/query/exec/HistogramQuantileMapper.scala index 69e9a6f6a3..6a3d5d50fd 100644 --- a/query/src/main/scala/filodb/query/exec/HistogramQuantileMapper.scala +++ b/query/src/main/scala/filodb/query/exec/HistogramQuantileMapper.scala @@ -31,7 +31,7 @@ case class HistogramQuantileMapper(funcParams: Seq[Any]) extends RangeVectorTran * @param le the less-than-equals boundary for histogram bucket * @param rate number of occurrences for the bucket per second */ - case class Bucket(val le: Double, var rate: Double) { + case class Bucket(le: Double, var rate: Double) { override def toString: String = s"$le->$rate" } @@ -75,7 +75,7 @@ case class HistogramQuantileMapper(funcParams: Seq[Any]) extends RangeVectorTran val row = new TransientRow() override def hasNext: Boolean = samples.forall(_.hasNext) override def next(): RowReader = { - for { i <- 0 until samples.length } { + for { i <- samples.indices } { val nxt = samples(i).next() buckets(i).rate = nxt.getDouble(1) row.timestamp = nxt.getLong(0) @@ -115,7 +115,7 @@ case class HistogramQuantileMapper(funcParams: Seq[Any]) extends RangeVectorTran * Similar to prometheus implementation for consistent results. */ private def histogramQuantile(q: Double, buckets: Array[Bucket]): Double = { - if (!buckets.last.le.isPosInfinity) return Double.NaN + if (!buckets.last.le.isPosInfinity) Double.NaN else { makeMonotonic(buckets) PromRateHistogram(buckets).quantile(q) diff --git a/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala b/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala index b70072950d..bb1a8b9351 100644 --- a/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala +++ b/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala @@ -49,7 +49,7 @@ final case class PeriodicSamplesMapper(start: Long, // Generate one range function to check if it is chunked val sampleRangeFunc = rangeFuncGen() // Really, use the stale lookback window size, not 0 which doesn't make sense - val windowLength = window.getOrElse(if (functionId == None) queryConfig.staleSampleAfterMs else 0L) + val windowLength = window.getOrElse(if (functionId.isEmpty) queryConfig.staleSampleAfterMs else 0L) sampleRangeFunc match { case c: ChunkedRangeFunction[_] if valColType == ColumnType.HistogramColumn => diff --git a/query/src/main/scala/filodb/query/exec/RowKeyRange.scala b/query/src/main/scala/filodb/query/exec/RowKeyRange.scala new file mode 100644 index 0000000000..0047c9f442 --- /dev/null +++ b/query/src/main/scala/filodb/query/exec/RowKeyRange.scala @@ -0,0 +1,12 @@ +package filodb.query.exec + +import filodb.core.binaryrecord.BinaryRecord + +sealed trait RowKeyRange + +case class RowKeyInterval(from: BinaryRecord, to: BinaryRecord) extends RowKeyRange +case object AllChunks extends RowKeyRange +case object WriteBuffers extends RowKeyRange +case object InMemoryChunks extends RowKeyRange +case object EncodedChunks extends RowKeyRange + diff --git a/query/src/main/scala/filodb/query/exec/StitchRvsExec.scala b/query/src/main/scala/filodb/query/exec/StitchRvsExec.scala new file mode 100644 index 0000000000..32ec5307af --- /dev/null +++ b/query/src/main/scala/filodb/query/exec/StitchRvsExec.scala @@ -0,0 +1,108 @@ +package filodb.query.exec + +import scala.collection.mutable + +import monix.reactive.Observable + +import filodb.core.metadata.Dataset +import filodb.core.query._ +import filodb.memory.format.RowReader +import filodb.query._ +import filodb.query.Query.qLogger + +object StitchRvsExec { + + def merge(vectors: Seq[Iterator[RowReader]]): Iterator[RowReader] = { + // This is an n-way merge without using a heap. + // Heap is not used since n is expected to be very small (almost always just 1 or 2) + new Iterator[RowReader] { + val bVectors = vectors.map(_.buffered) + val mins = new mutable.ArrayBuffer[BufferedIterator[RowReader]](2) + val noResult = new TransientRow(0, 0) + override def hasNext: Boolean = bVectors.exists(_.hasNext) + + override def next(): RowReader = { + mins.clear() + var minTime = Long.MaxValue + bVectors.foreach { r => + if (r.hasNext) { + val t = r.head.getLong(0) + if (mins.size == 0) { + minTime = t + mins += r + } + else if (t < minTime) { + mins.clear() + mins += r + minTime = t + } else if (t == minTime) { + mins += r + } + } + } + if (mins.size == 1) mins.head.next() + else if (mins.isEmpty) throw new IllegalStateException("next was called when no element") + else { + mins.foreach(it => if (it.hasNext) it.next()) // move iterator forward + noResult.timestamp = minTime + noResult.value = Double.NaN // until we have a different indicator for "unable-to-calculate" use NaN + noResult + } + } + } + } +} + +/** + * Use when data for same time series spans multiple shards, or clusters. + */ +final case class StitchRvsExec(id: String, + dispatcher: PlanDispatcher, + children: Seq[ExecPlan]) extends NonLeafExecPlan { + require(children.nonEmpty) + + protected def args: String = "" + + protected def schemaOfCompose(dataset: Dataset): ResultSchema = children.head.schema(dataset) + + protected def compose(childResponses: Observable[(QueryResponse, Int)], + queryConfig: QueryConfig): Observable[RangeVector] = { + qLogger.debug(s"StitchRvsExec: Stitching results:") + val stitched = childResponses.map { + case (QueryResult(_, _, result), _) => result + case (QueryError(_, ex), _) => throw ex + }.toListL.map(_.flatten).map { srvs => + val groups = srvs.groupBy(_.key) + groups.mapValues { toMerge => + val rows = StitchRvsExec.merge(toMerge.map(_.rows)) + val key = toMerge.head.key + IteratorBackedRangeVector(key, rows) + }.values + }.map(Observable.fromIterable) + Observable.fromTask(stitched).flatten + } +} + +/** + * Range Vector Transformer version of StitchRvsExec + */ +final case class StitchRvsMapper() extends RangeVectorTransformer { + + def apply(source: Observable[RangeVector], + queryConfig: QueryConfig, + limit: Int, + sourceSchema: ResultSchema): Observable[RangeVector] = { + qLogger.debug(s"StitchRvsMapper: Stitching results:") + val stitched = source.toListL.map { rvs => + val groups = rvs.groupBy(_.key) + groups.mapValues { toMerge => + val rows = StitchRvsExec.merge(toMerge.map(_.rows)) + val key = toMerge.head.key + IteratorBackedRangeVector(key, rows) + }.values + }.map(Observable.fromIterable) + Observable.fromTask(stitched).flatten + } + + override protected[query] def args: String = "" +} diff --git a/query/src/test/scala/filodb/query/exec/StitchRvsExecSpec.scala b/query/src/test/scala/filodb/query/exec/StitchRvsExecSpec.scala new file mode 100644 index 0000000000..b15ad85758 --- /dev/null +++ b/query/src/test/scala/filodb/query/exec/StitchRvsExecSpec.scala @@ -0,0 +1,170 @@ +package filodb.query.exec + +import scala.annotation.tailrec + +import org.scalatest.{FunSpec, Matchers} + +// scalastyle:off null +class StitchRvsExecSpec extends FunSpec with Matchers { + val error = 0.0000001d + + it ("should merge with two overlapping RVs correctly") { + val rvs = Seq ( + Seq( (10L, 3d), + (20L, 3d), + (30L, 3d), + (40L, 3d), + (50L, 3d) + ), + Seq( (30L, 4d), + (50L, 4d), + (60L, 3d), + (70L, 3d), + (80L, 3d), + (90L, 3d), + (100L, 3d) + ) + ) + val expected = + Seq( (10L, 3d), + (20L, 3d), + (30L, Double.NaN), + (40L, 3d), + (50L, Double.NaN), + (60L, 3d), + (70L, 3d), + (80L, 3d), + (90L, 3d), + (100L, 3d) + ) + mergeAndValidate(rvs, expected) + } + + it ("should merge one RV correctly") { + val input = Seq( (10L, 3d), + (20L, 3d), + (30L, Double.NaN), + (40L, 3d), + (50L, Double.NaN), + (60L, 3d), + (70L, 3d), + (80L, 3d), + (90L, 3d), + (100L, 3d) + ) + mergeAndValidate(Seq(input), input) + } + it ("should merge with three overlapping RVs correctly") { + val rvs = Seq ( + Seq( (10L, 3d), + (20L, 3d), + (30L, 3d), + (40L, 3d), + (50L, 3d) + ), + Seq( (30L, 4d), + (50L, 4d), + (60L, 3d), + (70L, 3d), + (80L, 3d), + (90L, 3d), + (100L, 3d) + ), + Seq( (30L, 4d), + (55L, 3d) + ) + ) + val expected = + Seq( (10L, 3d), + (20L, 3d), + (30L, Double.NaN), + (40L, 3d), + (50L, Double.NaN), + (55L, 3d), + (60L, 3d), + (70L, 3d), + (80L, 3d), + (90L, 3d), + (100L, 3d) + ) + mergeAndValidate(rvs, expected) + } + + + it ("should merge with no overlap correctly") { + val rvs = Seq ( + Seq( + (60L, 3d), + (70L, 3d), + (80L, 3d), + (90L, 3d), + (100L, 3d) + ), + Seq( (10L, 3d), + (20L, 3d), + (30L, 3d), + (40L, 3d), + (50L, 3d) + ) + ) + val expected = + Seq( (10L, 3d), + (20L, 3d), + (30L, 3d), + (40L, 3d), + (50L, 3d), + (60L, 3d), + (70L, 3d), + (80L, 3d), + (90L, 3d), + (100L, 3d) + ) + mergeAndValidate(rvs, expected) + } + + + it ("should merge with one empty rv correctly") { + val rvs = Seq ( + Seq( + (60L, 3d), + (70L, 3d), + (80L, 3d), + (90L, 3d), + (100L, 3d) + ), + Seq() + ) + val expected = + Seq( + (60L, 3d), + (70L, 3d), + (80L, 3d), + (90L, 3d), + (100L, 3d) + ) + mergeAndValidate(rvs, expected) + } + + def mergeAndValidate(rvs: Seq[Seq[(Long, Double)]], expected: Seq[(Long, Double)]): Unit = { + val inputSeq = rvs.map { rows => rows.iterator.map(r => new TransientRow(r._1, r._2)) } + val result = StitchRvsExec.merge(inputSeq).map(r => (r.getLong(0), r.getDouble(1))) + compareIter(result, expected.toIterator) + } + + @tailrec + final private def compareIter(it1: Iterator[(Long, Double)], it2: Iterator[(Long, Double)]) : Unit = { + (it1.hasNext, it2.hasNext) match{ + case (true, true) => + val v1 = it1.next() + val v2 = it2.next() + v1._1 shouldEqual v2._1 + if (v1._2.isNaN) v2._2.isNaN shouldEqual true + else Math.abs(v1._2-v2._2) should be < error + compareIter(it1, it2) + case (false, false) => Unit + case _ => fail("Unequal lengths") + } + } + +} + diff --git a/stress/src/main/scala/filodb.stress/MemStoreStress.scala b/stress/src/main/scala/filodb.stress/MemStoreStress.scala index 2253c81493..5211903bca 100644 --- a/stress/src/main/scala/filodb.stress/MemStoreStress.scala +++ b/stress/src/main/scala/filodb.stress/MemStoreStress.scala @@ -68,7 +68,7 @@ object MemStoreStress extends App { // use observables (a stream of queries) to handle queries val startTime = DateTime.parse("2013-01-01T00Z").getMillis val endTime = DateTime.parse("2013-02-01T00Z").getMillis - val interval = IntervalSelector(Seq(startTime), Seq(endTime)) + val interval = IntervalSelector(startTime, endTime) val ref = DatasetRef("nyc_taxi") var startMs = 0L var endMs = 0L From 6daa6422be1d08befac12829a2b626b42ae89317 Mon Sep 17 00:00:00 2001 From: TanviBhavsar Date: Thu, 7 Mar 2019 22:27:28 -0800 Subject: [PATCH 07/41] feat(query): Label Replace function support (#257) --- .../queryengine2/QueryEngine.scala | 10 + .../scala/filodb.core/query/RangeVector.scala | 4 +- .../filodb/prometheus/ast/Functions.scala | 21 +- .../filodb/prometheus/parse/ParserSpec.scala | 6 +- .../main/scala/filodb/query/LogicalPlan.scala | 7 + .../main/scala/filodb/query/PlanEnums.scala | 55 ++- .../query/exec/RangeVectorTransformer.scala | 29 +- .../exec/rangefn/MiscellaneousFunction.scala | 80 ++++ .../exec/rangefn/InstantFunctionSpec.scala | 5 +- .../query/exec/rangefn/LabelReplaceSpec.scala | 410 ++++++++++++++++++ 10 files changed, 589 insertions(+), 38 deletions(-) create mode 100644 query/src/main/scala/filodb/query/exec/rangefn/MiscellaneousFunction.scala create mode 100644 query/src/test/scala/filodb/query/exec/rangefn/LabelReplaceSpec.scala diff --git a/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala b/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala index 2096e84c08..2f8f18499c 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala @@ -136,6 +136,7 @@ class QueryEngine(dataset: Dataset, case lp: ScalarVectorBinaryOperation => materializeScalarVectorBinOp(queryId, submitTime, options, lp) case lp: LabelValues => materializeLabelValues(queryId, submitTime, options, lp) case lp: SeriesKeysByFilters => materializeSeriesKeysByFilters(queryId, submitTime, options, lp) + case lp: ApplyMiscellaneousFunction => materializeApplyMiscellaneousFunction(queryId, submitTime, options, lp) } } @@ -306,6 +307,15 @@ class QueryEngine(dataset: Dataset, PlanResult(metaExec, false) } + private def materializeApplyMiscellaneousFunction(queryId: String, + submitTime: Long, + options: QueryOptions, + lp: ApplyMiscellaneousFunction): PlanResult = { + val vectors = walkLogicalPlanTree(lp.vectors, queryId, submitTime, options) + vectors.plans.foreach(_.addRangeVectorTransformer(MiscellaneousFunctionMapper(lp.function, lp.functionArgs))) + vectors + } + /** * Renames Prom AST __name__ metric name filters to one based on the actual metric column of the dataset, * if it is not the prometheus standard diff --git a/core/src/main/scala/filodb.core/query/RangeVector.scala b/core/src/main/scala/filodb.core/query/RangeVector.scala index bb246d80d0..6c75046b4b 100644 --- a/core/src/main/scala/filodb.core/query/RangeVector.scala +++ b/core/src/main/scala/filodb.core/query/RangeVector.scala @@ -59,8 +59,8 @@ final case class PartitionRangeVectorKey(partBase: Array[Byte], override def toString: String = s"/shard:$sourceShard/${partSchema.stringify(partBase, partOffset)} [grp$groupNum]" } -final case class CustomRangeVectorKey(labelValues: Map[UTF8Str, UTF8Str]) extends RangeVectorKey { - val sourceShards: Seq[Int] = Nil +final case class CustomRangeVectorKey(labelValues: Map[UTF8Str, UTF8Str], sourceShards: Seq[Int] = Nil) + extends RangeVectorKey { } object CustomRangeVectorKey { diff --git a/prometheus/src/main/scala/filodb/prometheus/ast/Functions.scala b/prometheus/src/main/scala/filodb/prometheus/ast/Functions.scala index a7f411a6af..5270bd1228 100644 --- a/prometheus/src/main/scala/filodb/prometheus/ast/Functions.scala +++ b/prometheus/src/main/scala/filodb/prometheus/ast/Functions.scala @@ -10,15 +10,24 @@ trait Functions extends Base with Operators with Vectors { if (!ignoreChecks && InstantFunctionId.withNameLowercaseOnlyOption(name.toLowerCase).isEmpty && RangeFunctionId.withNameLowercaseOnlyOption(name.toLowerCase).isEmpty && - FiloFunctionId.withNameLowercaseOnlyOption(name.toLowerCase).isEmpty) { + FiloFunctionId.withNameLowercaseOnlyOption(name.toLowerCase).isEmpty && + MiscellaneousFunctionId.withNameLowercaseOnlyOption(name.toLowerCase).isEmpty) { throw new IllegalArgumentException(s"Invalid function name [$name]") } def toPeriodicSeriesPlan(timeParams: TimeRangeParams): PeriodicSeriesPlan = { val seriesParam = allParams.filter(_.isInstanceOf[Series]).head.asInstanceOf[Series] - val otherParams = allParams.filter(!_.isInstanceOf[Series]).map(_.asInstanceOf[Scalar].toScalar) + val otherParams = allParams.filter(!_.equals(seriesParam)).map(_ match { + case num: ScalarExpression => num.toScalar + case s: InstantExpression => s.metricName.replaceAll("^\"|\"$", "") + case _ => + throw new IllegalArgumentException("Parameters can be a string or number") + } + ) + val instantFunctionIdOpt = InstantFunctionId.withNameInsensitiveOption(name) val filoFunctionIdOpt = FiloFunctionId.withNameInsensitiveOption(name) + val miscellaneousFunctionIdOpt = MiscellaneousFunctionId.withNameInsensitiveOption(name) if (instantFunctionIdOpt.isDefined) { val instantFunctionId = instantFunctionIdOpt.get @@ -37,7 +46,13 @@ trait Functions extends Base with Operators with Vectors { case FiloFunctionId.ChunkMetaAll => // Just get the raw chunk metadata RawChunkMeta(rangeSelector, filters, columns.headOption.getOrElse("")) } - } else { + } else if (miscellaneousFunctionIdOpt.isDefined) { + val miscellaneousFunctionId = miscellaneousFunctionIdOpt.get + val periodicSeriesPlan = seriesParam.asInstanceOf[PeriodicSeries].toPeriodicSeriesPlan(timeParams) + + ApplyMiscellaneousFunction(periodicSeriesPlan, miscellaneousFunctionId, otherParams) + } + else { val rangeFunctionId = RangeFunctionId.withNameInsensitiveOption(name).get val rangeExpression = seriesParam.asInstanceOf[RangeExpression] diff --git a/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala b/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala index 2144274f00..ad3ecfb7f6 100644 --- a/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala +++ b/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala @@ -267,11 +267,11 @@ class ParserSpec extends FunSpec with Matchers { "count_values(\"version\", build_version)" -> "Aggregate(CountValues,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(build_version))),List()),1524855988000,1000000,1524855988000),List(\"version\"),List(),List())", "label_replace(up{job=\"api-server\",service=\"a:c\"}, \"foo\", \"$1\", \"service\", \"(.*):.*\")" -> - "ApplyInstantFunction(PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(job,Equals(api-server)), ColumnFilter(service,Equals(a:c)), ColumnFilter(__name__,Equals(up))),List()),1524855988000,1000000,1524855988000),LabelReplace,List())", + "ApplyMiscellaneousFunction(PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(job,Equals(api-server)), ColumnFilter(service,Equals(a:c)), ColumnFilter(__name__,Equals(up))),List()),1524855988000,1000000,1524855988000),LabelReplace,List(foo, $1, service, (.*):.*))", "sum(http_requests_total)" -> "Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000),List(),List(),List())", "label_join(up{job=\"api-server\",src1=\"a\",src2=\"b\",src3=\"c\"}, \"foo\", \",\", \"src1\", \"src2\", \"src3\")" -> - "ApplyInstantFunction(PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(job,Equals(api-server)), ColumnFilter(src1,Equals(a)), ColumnFilter(src2,Equals(b)), ColumnFilter(src3,Equals(c)), ColumnFilter(__name__,Equals(up))),List()),1524855988000,1000000,1524855988000),LabelJoin,List())", + "ApplyMiscellaneousFunction(PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(job,Equals(api-server)), ColumnFilter(src1,Equals(a)), ColumnFilter(src2,Equals(b)), ColumnFilter(src3,Equals(c)), ColumnFilter(__name__,Equals(up))),List()),1524855988000,1000000,1524855988000),LabelJoin,List(foo, ,, src1, src2, src3))", "histogram_quantile(0.9, sum(rate(http_request_duration_seconds_bucket[10m])) by (le))" -> "ApplyInstantFunction(Aggregate(Sum,PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(1524855388000,1524855988000),List(ColumnFilter(__name__,Equals(http_request_duration_seconds_bucket))),List()),1524855988000,1000000,1524855988000,600000,Rate,List()),List(),List(le),List()),HistogramQuantile,List(0.9))", "delta(cpu_temp_celsius{host=\"zeus\"}[2h])" -> @@ -331,7 +331,7 @@ class ParserSpec extends FunSpec with Matchers { lp.toString shouldEqual (e) } } - + private def parseSuccessfully(query: String) = { Parser.parseQuery(query) } diff --git a/query/src/main/scala/filodb/query/LogicalPlan.scala b/query/src/main/scala/filodb/query/LogicalPlan.scala index c449249a7e..7f79f6d5e0 100644 --- a/query/src/main/scala/filodb/query/LogicalPlan.scala +++ b/query/src/main/scala/filodb/query/LogicalPlan.scala @@ -128,4 +128,11 @@ case class ScalarVectorBinaryOperation(operator: BinaryOperator, */ case class ApplyInstantFunction(vectors: PeriodicSeriesPlan, function: InstantFunctionId, + functionArgs: Seq[Any] = Nil) extends PeriodicSeriesPlan + +/** + * Apply Miscellaneous Function to a collection of RangeVectors + */ +case class ApplyMiscellaneousFunction(vectors: PeriodicSeriesPlan, + function: MiscellaneousFunctionId, functionArgs: Seq[Any] = Nil) extends PeriodicSeriesPlan \ No newline at end of file diff --git a/query/src/main/scala/filodb/query/PlanEnums.scala b/query/src/main/scala/filodb/query/PlanEnums.scala index 49d8f85487..976f678d66 100644 --- a/query/src/main/scala/filodb/query/PlanEnums.scala +++ b/query/src/main/scala/filodb/query/PlanEnums.scala @@ -18,46 +18,22 @@ object InstantFunctionId extends Enum[InstantFunctionId] { case object ClampMin extends InstantFunctionId("clamp_min") - case object DaysInMonth extends InstantFunctionId("days_in_month") - - case object DaysOfMonth extends InstantFunctionId("day_of_month") - - case object DayOfWeek extends InstantFunctionId("day_of_Week") - case object Exp extends InstantFunctionId("exp") case object Floor extends InstantFunctionId("floor") case object HistogramQuantile extends InstantFunctionId("histogram_quantile") - case object Hour extends InstantFunctionId("hour") - - case object LabelReplace extends InstantFunctionId("label_replace") - - case object LabelJoin extends InstantFunctionId("label_join") - case object Ln extends InstantFunctionId("ln") case object Log10 extends InstantFunctionId("log10") case object Log2 extends InstantFunctionId("log2") - case object Minute extends InstantFunctionId("minute") - - case object Month extends InstantFunctionId("month") - case object Round extends InstantFunctionId("round") - case object Sort extends InstantFunctionId("sort") - - case object SortDesc extends InstantFunctionId("sort_desc") - case object Sqrt extends InstantFunctionId("sqrt") - case object Timestamp extends InstantFunctionId("timestamp") - - case object Year extends InstantFunctionId("year") - // TODO time, vector, scalar } @@ -204,6 +180,37 @@ object Cardinality extends Enum[Cardinality] { } +sealed abstract class MiscellaneousFunctionId(override val entryName: String) extends EnumEntry + +object MiscellaneousFunctionId extends Enum[MiscellaneousFunctionId] { + val values = findValues + + case object DaysInMonth extends MiscellaneousFunctionId("days_in_month") + + case object DaysOfMonth extends MiscellaneousFunctionId("day_of_month") + + case object DayOfWeek extends MiscellaneousFunctionId("day_of_week") + + case object Hour extends MiscellaneousFunctionId("hour") + + case object LabelReplace extends MiscellaneousFunctionId("label_replace") + + case object LabelJoin extends MiscellaneousFunctionId("label_join") + + case object Minute extends MiscellaneousFunctionId("minute") + + case object Month extends MiscellaneousFunctionId("month") + + case object Sort extends MiscellaneousFunctionId("sort") + + case object SortDesc extends MiscellaneousFunctionId("sort_desc") + + case object Timestamp extends MiscellaneousFunctionId("timestamp") + + case object Year extends MiscellaneousFunctionId("year") + +} + //scalastyle:on diff --git a/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala b/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala index 6a410e572d..947fd44d0f 100644 --- a/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala +++ b/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala @@ -6,10 +6,13 @@ import filodb.core.metadata.Column.ColumnType import filodb.core.metadata.Dataset import filodb.core.query._ import filodb.memory.format.RowReader -import filodb.query.{BinaryOperator, InstantFunctionId, QueryConfig} -import filodb.query.exec.binaryOp.BinaryOperatorFunction -import filodb.query.exec.rangefn.{DoubleInstantFunction, HistToDoubleIFunction, InstantFunction} +import filodb.query.{BinaryOperator, InstantFunctionId, MiscellaneousFunctionId, QueryConfig} import filodb.query.InstantFunctionId.HistogramQuantile +import filodb.query.MiscellaneousFunctionId.LabelReplace +import filodb.query.exec.binaryOp.BinaryOperatorFunction +import filodb.query.exec.rangefn._ + + /** * Implementations can provide ways to transform RangeVector @@ -168,3 +171,23 @@ final case class ScalarOperationMapper(operator: BinaryOperator, // TODO all operation defs go here and get invoked from mapRangeVector } + +final case class MiscellaneousFunctionMapper(function: MiscellaneousFunctionId, + funcParams: Seq[Any] = Nil) extends RangeVectorTransformer { + protected[exec] def args: String = + s"function=$function, funcParams=$funcParams" + + val miscFunction: MiscellaneousFunction = { + function match { + case LabelReplace => LabelReplaceFunction(funcParams) + case _ => throw new UnsupportedOperationException(s"$function not supported.") + } + } + + def apply(source: Observable[RangeVector], + queryConfig: QueryConfig, + limit: Int, + sourceSchema: ResultSchema): Observable[RangeVector] = { + miscFunction.execute(source) + } +} \ No newline at end of file diff --git a/query/src/main/scala/filodb/query/exec/rangefn/MiscellaneousFunction.scala b/query/src/main/scala/filodb/query/exec/rangefn/MiscellaneousFunction.scala new file mode 100644 index 0000000000..3856ca180b --- /dev/null +++ b/query/src/main/scala/filodb/query/exec/rangefn/MiscellaneousFunction.scala @@ -0,0 +1,80 @@ +package filodb.query.exec.rangefn + +import java.util.regex.{Pattern, PatternSyntaxException} + +import monix.reactive.Observable + +import filodb.core.query.{CustomRangeVectorKey, IteratorBackedRangeVector, RangeVector, RangeVectorKey} +import filodb.memory.format.ZeroCopyUTF8String + +trait MiscellaneousFunction { + def execute(source: Observable[RangeVector]): Observable[RangeVector] +} + +case class LabelReplaceFunction(funcParams: Seq[Any]) + extends MiscellaneousFunction { + + val labelIdentifier: String = "[a-zA-Z_][a-zA-Z0-9_:\\-\\.]*" + + require(funcParams.size == 4, + "Cannot use LabelReplace without function parameters: " + + "instant-vector, dst_label string, replacement string, src_label string, regex string") + + val dstLabel: String = funcParams(0).asInstanceOf[String] + val replacementString: String = funcParams(1).asInstanceOf[String] + val srcLabel: String = funcParams(2).asInstanceOf[String] + val regexString: String = funcParams(3).asInstanceOf[String] + + require(dstLabel.matches(labelIdentifier), "Invalid destination label name") + + try { + Pattern.compile(regexString) + } + catch { + case ex: PatternSyntaxException => { + throw new IllegalArgumentException("Invalid Regular Expression for label_replace", ex) + } + } + + override def execute(source: Observable[RangeVector]): Observable[RangeVector] = { + source.map { rv => + val newLabel = labelReplaceImpl(rv.key, funcParams) + IteratorBackedRangeVector(newLabel, rv.rows) + } + } + + def labelReplaceImpl(rangeVectorKey: RangeVectorKey, funcParams: Seq[Any]): RangeVectorKey = { + + val value: ZeroCopyUTF8String = if (rangeVectorKey.labelValues.contains(ZeroCopyUTF8String(srcLabel))) { + rangeVectorKey.labelValues.get(ZeroCopyUTF8String(srcLabel)).get + } + else { + // Assign dummy value as label_replace should overwrite destination label if the source label is empty but matched + ZeroCopyUTF8String.empty + } + + // Pattern is not deserialized correctly if it is a data member + val pattern = Pattern.compile(regexString) + val matcher = pattern.matcher(value.toString) + if (matcher.matches()) { + var labelReplaceValue = replacementString + for (index <- 1 to matcher.groupCount()) { + labelReplaceValue = labelReplaceValue.replace(s"$$$index", matcher.group(index)) + } + // Remove groups which are not present + labelReplaceValue = labelReplaceValue.replaceAll("\\$[A-Za-z0-9]+", "") + + if (labelReplaceValue.length > 0) { + return CustomRangeVectorKey(rangeVectorKey.labelValues. + updated(ZeroCopyUTF8String(dstLabel), ZeroCopyUTF8String(labelReplaceValue)), rangeVectorKey.sourceShards) + } + else { + // Drop label if new value is empty + return CustomRangeVectorKey(rangeVectorKey.labelValues - + ZeroCopyUTF8String(dstLabel), rangeVectorKey.sourceShards) + } + } + + return rangeVectorKey; + } +} 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 18d570ffb7..41aa70edf5 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/InstantFunctionSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/InstantFunctionSpec.scala @@ -159,8 +159,8 @@ class InstantFunctionSpec extends RawDataWindowingSpec with ScalaFutures { it ("should handle unknown functions") { // sort_desc the[UnsupportedOperationException] thrownBy { - val instantVectorFnMapper = exec.InstantVectorFunctionMapper(InstantFunctionId.SortDesc) - instantVectorFnMapper(Observable.fromIterable(sampleBase), queryConfig, 1000, resultSchema) + val miscellaneousVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.SortDesc) + miscellaneousVectorFnMapper(Observable.fromIterable(sampleBase), queryConfig, 1000, resultSchema) } should have message "SortDesc not supported." } @@ -252,5 +252,4 @@ class InstantFunctionSpec extends RawDataWindowingSpec with ScalaFutures { } } } - } \ No newline at end of file diff --git a/query/src/test/scala/filodb/query/exec/rangefn/LabelReplaceSpec.scala b/query/src/test/scala/filodb/query/exec/rangefn/LabelReplaceSpec.scala new file mode 100644 index 0000000000..274375b387 --- /dev/null +++ b/query/src/test/scala/filodb/query/exec/rangefn/LabelReplaceSpec.scala @@ -0,0 +1,410 @@ +package filodb.query.exec.rangefn + +import com.typesafe.config.{Config, ConfigFactory} +import monix.execution.Scheduler.Implicits.global +import monix.reactive.Observable +import org.scalatest.{FunSpec, Matchers} +import org.scalatest.concurrent.ScalaFutures + +import filodb.core.MetricsTestData +import filodb.core.query.{CustomRangeVectorKey, RangeVector, RangeVectorKey, ResultSchema} +import filodb.memory.format.{RowReader, ZeroCopyUTF8String} +import filodb.query._ +import filodb.query.exec.TransientRow + +class LabelReplaceSpec extends FunSpec with Matchers with ScalaFutures { + + val config: Config = ConfigFactory.load("application_test.conf").getConfig("filodb") + val resultSchema = ResultSchema(MetricsTestData.timeseriesDataset.infosFromIDs(0 to 1), 1) + val ignoreKey = CustomRangeVectorKey( + Map(ZeroCopyUTF8String("ignore") -> ZeroCopyUTF8String("ignore"))) + + val testKey1 = CustomRangeVectorKey( + Map(ZeroCopyUTF8String("src") -> ZeroCopyUTF8String("source-value-10"), + ZeroCopyUTF8String("dst") -> ZeroCopyUTF8String("original-destination-value"))) + + val testKey2 = CustomRangeVectorKey( + Map(ZeroCopyUTF8String("src") -> ZeroCopyUTF8String("source-value-20"), + ZeroCopyUTF8String("dst") -> ZeroCopyUTF8String("original-destination-value"))) + + val testSample: Array[RangeVector] = Array( + new RangeVector { + override def key: RangeVectorKey = testKey1 + + override def rows: Iterator[RowReader] = Seq( + new TransientRow(1L, 3.3d), + new TransientRow(2L, 5.1d)).iterator + }, + new RangeVector { + override def key: RangeVectorKey = testKey2 + + override def rows: Iterator[RowReader] = Seq( + new TransientRow(3L, 100d), + new TransientRow(4L, 200d)).iterator + }) + + val queryConfig = new QueryConfig(config.getConfig("query")) + + it("should replace label only when match is found in label replace") { + val sampleKey1 = CustomRangeVectorKey( + Map(ZeroCopyUTF8String("instance") -> ZeroCopyUTF8String("demo.io:9090"), + ZeroCopyUTF8String("job") -> ZeroCopyUTF8String("test"))) + val sampleKey2 = CustomRangeVectorKey( + Map(ZeroCopyUTF8String("instance") -> ZeroCopyUTF8String("demo.io:8080"))) + + val sampleWithKey: Array[RangeVector] = Array( + new RangeVector { + override def key: RangeVectorKey = sampleKey1 + + override def rows: Iterator[RowReader] = Seq( + new TransientRow(1L, 3.3d), + new TransientRow(2L, 5.1d)).iterator + }, + new RangeVector { + override def key: RangeVectorKey = sampleKey2 + + override def rows: Iterator[RowReader] = Seq( + new TransientRow(3L, 100d), + new TransientRow(4L, 200d)).iterator + }) + + val expectedLabels = List(Map(ZeroCopyUTF8String("instance") -> ZeroCopyUTF8String("demo.io new Label Value 90"), + ZeroCopyUTF8String("job") -> ZeroCopyUTF8String("test")), + Map(ZeroCopyUTF8String("instance") -> ZeroCopyUTF8String("demo.io:8080"))) // will not be replaced + + val funcParams = Seq("instance", "$1 new Label Value $2", "instance", "(.*):90(.*)") + val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, funcParams) + val resultObs = labelVectorFnMapper(Observable.fromIterable(sampleWithKey), queryConfig, 1000, resultSchema) + val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) + val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) + + resultLabelValues.sameElements(expectedLabels) shouldEqual true + + //label_replace should not change rows + sampleWithKey.map(_.rows.map(_.getDouble(1))).zip(resultRows).foreach { + case (ex, res) => { + ex.zip(res).foreach { + case (val1, val2) => + val1 shouldEqual val2 + } + } + } + } + + it("should add new label when dst_label does not exist") { + val sampleKey1 = CustomRangeVectorKey( + Map(ZeroCopyUTF8String("instance") -> ZeroCopyUTF8String("Instance-100"), + ZeroCopyUTF8String("job") -> ZeroCopyUTF8String("test"))) + + val sampleWithKey: Array[RangeVector] = Array( + new RangeVector { + override def key: RangeVectorKey = sampleKey1 + + override def rows: Iterator[RowReader] = Seq( + new TransientRow(1L, 3.3d), + new TransientRow(2L, 5.1d)).iterator + }, + new RangeVector { + override def key: RangeVectorKey = ignoreKey + + override def rows: Iterator[RowReader] = Seq( + new TransientRow(3L, 100d), + new TransientRow(4L, 200d)).iterator + }) + + val expectedLabels = List(Map(ZeroCopyUTF8String("instance") -> ZeroCopyUTF8String("Instance-100"), + ZeroCopyUTF8String("job") -> ZeroCopyUTF8String("test"), + ZeroCopyUTF8String("instanceNew") -> ZeroCopyUTF8String("Instance-10-Instance-10")), + Map(ZeroCopyUTF8String("ignore") -> ZeroCopyUTF8String("ignore"))) + + val funcParams = Seq("instanceNew", "$1-$1", "instance", "(.*)\\d") + + val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, funcParams) + val resultObs = labelVectorFnMapper(Observable.fromIterable(sampleWithKey), queryConfig, 1000, resultSchema) + val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) + val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) + + resultLabelValues.sameElements(expectedLabels) shouldEqual true + + sampleWithKey.map(_.rows.map(_.getDouble(1))).zip(resultRows).foreach { + case (ex, res) => { + ex.zip(res).foreach { + case (val1, val2) => + val1 shouldEqual val2 + } + } + } + } + + it("should not change sample when entire regex does not match") { + val sampleKey1 = CustomRangeVectorKey( + Map(ZeroCopyUTF8String("instance") -> ZeroCopyUTF8String("Instance-9090"), + ZeroCopyUTF8String("job") -> ZeroCopyUTF8String("test"))) + + val sampleWithKey: Array[RangeVector] = Array( + new RangeVector { + override def key: RangeVectorKey = sampleKey1 + + override def rows: Iterator[RowReader] = Seq( + new TransientRow(1L, 3.3d), + new TransientRow(2L, 5.1d)).iterator + }, + new RangeVector { + override def key: RangeVectorKey = ignoreKey + + override def rows: Iterator[RowReader] = Seq( + new TransientRow(3L, 100d), + new TransientRow(4L, 200d)).iterator + }) + + val expectedLabels = sampleWithKey.toList.map(_.key.labelValues) + val funcParams = Seq("instance", "$1", "instance", "(.*)9") + + val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, funcParams) + val resultObs = labelVectorFnMapper(Observable.fromIterable(sampleWithKey), queryConfig, 1000, resultSchema) + val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) + val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) + + resultLabelValues.sameElements(expectedLabels) shouldEqual true + + sampleWithKey.map(_.rows.map(_.getDouble(1))).zip(resultRows).foreach { + case (ex, res) => { + ex.zip(res).foreach { + case (val1, val2) => + val1 shouldEqual val2 + } + } + } + } + + it("should validate invalid function params") { + + val funcParams = Seq("instance", "$1", "instance", "(.*)9(") + + the[IllegalArgumentException] thrownBy { + val miscellaneousFunctionMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, + funcParams) + miscellaneousFunctionMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema) + } should have message "Invalid Regular Expression for label_replace" + + the[IllegalArgumentException] thrownBy { + val miscellaneousFunctionMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, + Seq("instance", "$1")) + miscellaneousFunctionMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema) + } should have message "requirement failed: " + + "Cannot use LabelReplace without function parameters: " + + "instant-vector, dst_label string, replacement string, src_label string, regex string" + + the[IllegalArgumentException] thrownBy { + val miscellaneousFunctionMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, + Seq("$instance", "$1", "instance", "(.*)9(")) + miscellaneousFunctionMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema) + } should have message "requirement failed: Invalid destination label name" + } + + it("should do a full-string match and replace") { + + val expectedLabels = List(Map(ZeroCopyUTF8String("src") -> ZeroCopyUTF8String("source-value-10"), + ZeroCopyUTF8String("dst") -> ZeroCopyUTF8String("destination-value-10")), + Map(ZeroCopyUTF8String("src") -> ZeroCopyUTF8String("source-value-20"), + ZeroCopyUTF8String("dst") -> ZeroCopyUTF8String("destination-value-20"))) + + val funcParams = Seq("dst", "destination-value-$1", "src", "source-value-(.*)") + + val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, funcParams) + val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema) + val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) + val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) + + resultLabelValues.sameElements(expectedLabels) shouldEqual true + + testSample.map(_.rows.map(_.getDouble(1))).zip(resultRows).foreach { + case (ex, res) => { + ex.zip(res).foreach { + case (val1, val2) => + val1 shouldEqual val2 + } + } + } + } + + it("should not do a sub-string match") { + + val expectedLabels = List(Map(ZeroCopyUTF8String("src") -> ZeroCopyUTF8String("source-value-10"), + ZeroCopyUTF8String("dst") -> ZeroCopyUTF8String("original-destination-value")), + Map(ZeroCopyUTF8String("src") -> ZeroCopyUTF8String("source-value-20"), + ZeroCopyUTF8String("dst") -> ZeroCopyUTF8String("original-destination-value"))) + + val funcParams = Seq("dst", "destination-value-$1", "src", "value-(.*)") + + val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, funcParams) + val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema) + val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) + val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) + + resultLabelValues.sameElements(expectedLabels) shouldEqual true + + testSample.map(_.rows.map(_.getDouble(1))).zip(resultRows).foreach { + case (ex, res) => { + ex.zip(res).foreach { + case (val1, val2) => + val1 shouldEqual val2 + } + } + } + } + + it("should works with multiple groups and remove groups which do not exist from the replacement string") { + + val expectedLabels = List(Map(ZeroCopyUTF8String("src") -> ZeroCopyUTF8String("source-value-10"), + ZeroCopyUTF8String("dst") -> ZeroCopyUTF8String("source-value-10 ")), + Map(ZeroCopyUTF8String("src") -> ZeroCopyUTF8String("source-value-20"), + ZeroCopyUTF8String("dst") -> ZeroCopyUTF8String("source-value-20 "))) + + val funcParams = Seq("dst", "$1-value-$2 $3$67", "src", "(.*)-value-(.*)") + + val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, funcParams) + val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema) + val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) + val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) + + resultLabelValues.sameElements(expectedLabels) shouldEqual true + + testSample.map(_.rows.map(_.getDouble(1))).zip(resultRows).foreach { + case (ex, res) => { + ex.zip(res).foreach { + case (val1, val2) => + val1 shouldEqual val2 + } + } + } + } + + it("should not overwrite the destination label if the source label does not exist") { + + val expectedLabels = List(Map(ZeroCopyUTF8String("src") -> ZeroCopyUTF8String("source-value-10"), + ZeroCopyUTF8String("dst") -> ZeroCopyUTF8String("original-destination-value")), + Map(ZeroCopyUTF8String("src") -> ZeroCopyUTF8String("source-value-20"), + ZeroCopyUTF8String("dst") -> ZeroCopyUTF8String("original-destination-value"))) + + val funcParams = Seq("dst", "value-$1", "nonexistent-src", "source-value-(.*)") + + val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, funcParams) + val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema) + val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) + val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) + + resultLabelValues.sameElements(expectedLabels) shouldEqual true + + testSample.map(_.rows.map(_.getDouble(1))).zip(resultRows).foreach { + case (ex, res) => { + ex.zip(res).foreach { + case (val1, val2) => + val1 shouldEqual val2 + } + } + } + } + + it("should overwrite destination label if the source label is empty but matched") { + + val expectedLabels = List(Map(ZeroCopyUTF8String("src") -> ZeroCopyUTF8String("source-value-10"), + ZeroCopyUTF8String("dst") -> ZeroCopyUTF8String("value-")), + Map(ZeroCopyUTF8String("src") -> ZeroCopyUTF8String("source-value-20"), + ZeroCopyUTF8String("dst") -> ZeroCopyUTF8String("value-"))) + + val funcParams = Seq("dst", "value-$1", "nonexistent-src", ".*") + + val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, funcParams) + val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema) + val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) + val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) + + resultLabelValues.sameElements(expectedLabels) shouldEqual true + + testSample.map(_.rows.map(_.getDouble(1))).zip(resultRows).foreach { + case (ex, res) => { + ex.zip(res).foreach { + case (val1, val2) => + val1 shouldEqual val2 + } + } + } + } + + it("should not overwrite the destination label if the source label is not matched") { + + val expectedLabels = List(Map(ZeroCopyUTF8String("src") -> ZeroCopyUTF8String("source-value-10"), + ZeroCopyUTF8String("dst") -> ZeroCopyUTF8String("original-destination-value")), + Map(ZeroCopyUTF8String("src") -> ZeroCopyUTF8String("source-value-20"), + ZeroCopyUTF8String("dst") -> ZeroCopyUTF8String("original-destination-value"))) + + val funcParams = Seq("dst", "value-$1", "src", "dummy-regex") + + val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, funcParams) + val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema) + val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) + val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) + + resultLabelValues.sameElements(expectedLabels) shouldEqual true + + testSample.map(_.rows.map(_.getDouble(1))).zip(resultRows).foreach { + case (ex, res) => { + ex.zip(res).foreach { + case (val1, val2) => + val1 shouldEqual val2 + } + } + } + } + + it("should drop labels that are set to empty values") { + + val expectedLabels = List(Map(ZeroCopyUTF8String("src") -> ZeroCopyUTF8String("source-value-10")), + Map(ZeroCopyUTF8String("src") -> ZeroCopyUTF8String("source-value-20"))) + + val funcParams = Seq("dst", "", "dst", ".*") + + val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, funcParams) + val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema) + val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) + val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) + + resultLabelValues.sameElements(expectedLabels) shouldEqual true + + testSample.map(_.rows.map(_.getDouble(1))).zip(resultRows).foreach { + case (ex, res) => { + ex.zip(res).foreach { + case (val1, val2) => + val1 shouldEqual val2 + } + } + } + } + + it("should remove duplicated identical output label sets") { + + val expectedLabels = List(Map(ZeroCopyUTF8String("dst") -> ZeroCopyUTF8String("original-destination-value")), + Map(ZeroCopyUTF8String("dst") -> ZeroCopyUTF8String("original-destination-value"))) + + val funcParams = Seq("src", "", "", "") + + val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, funcParams) + val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema) + val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) + val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) + + resultLabelValues.sameElements(expectedLabels) shouldEqual true + + testSample.map(_.rows.map(_.getDouble(1))).zip(resultRows).foreach { + case (ex, res) => { + ex.zip(res).foreach { + case (val1, val2) => + val1 shouldEqual val2 + } + } + } + } + + +} \ No newline at end of file From 1625e8c2dc4ba041b3f75d62f89185a7c31e51d3 Mon Sep 17 00:00:00 2001 From: TanviBhavsar Date: Fri, 8 Mar 2019 11:25:21 -0800 Subject: [PATCH 08/41] Changed default window to queryConfig.staleSampleAfterMs + 1 (#272) --- .../query/exec/PeriodicSamplesMapper.scala | 4 +- .../exec/PeriodicSamplesMapperSpec.scala | 41 +++++++++++++++++++ 2 files changed, 44 insertions(+), 1 deletion(-) create mode 100644 query/src/test/scala/filodb/query/exec/PeriodicSamplesMapperSpec.scala diff --git a/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala b/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala index bb1a8b9351..9a382d1491 100644 --- a/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala +++ b/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala @@ -49,7 +49,9 @@ final case class PeriodicSamplesMapper(start: Long, // Generate one range function to check if it is chunked val sampleRangeFunc = rangeFuncGen() // Really, use the stale lookback window size, not 0 which doesn't make sense - val windowLength = window.getOrElse(if (functionId.isEmpty) queryConfig.staleSampleAfterMs else 0L) + // Default value for window should be queryConfig.staleSampleAfterMs + 1 for empty functionId, + // so that it returns value present at time - staleSampleAfterMs + val windowLength = window.getOrElse(if (functionId.isEmpty) queryConfig.staleSampleAfterMs + 1 else 0L) sampleRangeFunc match { case c: ChunkedRangeFunction[_] if valColType == ColumnType.HistogramColumn => diff --git a/query/src/test/scala/filodb/query/exec/PeriodicSamplesMapperSpec.scala b/query/src/test/scala/filodb/query/exec/PeriodicSamplesMapperSpec.scala new file mode 100644 index 0000000000..d412c544d0 --- /dev/null +++ b/query/src/test/scala/filodb/query/exec/PeriodicSamplesMapperSpec.scala @@ -0,0 +1,41 @@ +package filodb.query.exec + +import monix.execution.Scheduler.Implicits.global +import monix.reactive.Observable +import org.scalatest.{FunSpec, Matchers} +import org.scalatest.concurrent.ScalaFutures + +import filodb.core.MetricsTestData +import filodb.core.query._ +import filodb.query._ +import filodb.query.exec.rangefn.RawDataWindowingSpec + +class PeriodicSamplesMapperSpec extends FunSpec with Matchers with ScalaFutures with RawDataWindowingSpec { + + val resultSchema = ResultSchema(MetricsTestData.timeseriesDataset.infosFromIDs(0 to 1), 1) + + val samples = Seq( + 100000L -> 100d, + 153000L -> 160d, + 200000L -> 200d + ) + val expectedResults = List(100000L -> 100d, + 200000L -> 200d, + 300000L -> 200d, + 400000L -> 200d, + 500000L -> 200d + ) + val rv = timeValueRV(samples) + + it("should return value present at time - staleSampleAfterMs") { + + val periodicSamplesVectorFnMapper = exec.PeriodicSamplesMapper(100000L, 100000, 600000L, None, None) + val resultObs = periodicSamplesVectorFnMapper(Observable.fromIterable(Seq(rv)), queryConfig, 1000, resultSchema) + + val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map + (r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN)) + + resultRows.foreach(_.toList shouldEqual expectedResults) + + } +} From d477099f2ce7a0c8d8e8b4e7f0cc70b97c70b2b6 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Fri, 8 Mar 2019 17:14:49 -0800 Subject: [PATCH 09/41] feat(memory): Implementation of NibblePacking compression/storage scheme (#269) --- doc/compression.md | 2 +- .../filodb.memory/format/NibblePack.scala | 285 ++++++++++++++++++ .../filodb.memory/format/NibblePackTest.scala | 121 ++++++++ .../format/vectors/LongVectorTest.scala | 21 +- 4 files changed, 417 insertions(+), 12 deletions(-) create mode 100644 memory/src/main/scala/filodb.memory/format/NibblePack.scala create mode 100644 memory/src/test/scala/filodb.memory/format/NibblePackTest.scala diff --git a/doc/compression.md b/doc/compression.md index fe5f1f7bf1..3f9e4c6a3e 100644 --- a/doc/compression.md +++ b/doc/compression.md @@ -31,7 +31,7 @@ These all have the capability to redue the number of bits in successive values. ## Predictive NibblePacking -This is a storage scheme which takes the output of one of the compression algorithms (really, they are **prediction** algorithms) above, which is a stream of 64-bit numbers with hopefully many zero bits, and encodes them in an efficient way. This encoding scheme works for the output of any of the integer or floating point compression/prediction schemes. We make these observations: +This is a storage scheme which takes the output of one of the compression algorithms (really, they are **prediction** algorithms) above, which is a stream of 64-bit numbers with hopefully many zero bits, and encodes them in an efficient way. It is loosely based on [this article](https://medium.com/@vaclav.loffelmann/the-worlds-first-middle-out-compression-for-time-series-data-part-2-40c048632911). This encoding scheme works for the output of any of the integer or floating point compression/prediction schemes. We make these observations: 1. The 64-bit values may have both leading zeros and trailing zeros (def true for floating point XOR/DFCM output) 2. Most sequences of these values have very similar numbers of leading and trailing zeros. diff --git a/memory/src/main/scala/filodb.memory/format/NibblePack.scala b/memory/src/main/scala/filodb.memory/format/NibblePack.scala new file mode 100644 index 0000000000..0bbe391ce3 --- /dev/null +++ b/memory/src/main/scala/filodb.memory/format/NibblePack.scala @@ -0,0 +1,285 @@ +package filodb.memory.format + +import java.nio.ByteOrder.LITTLE_ENDIAN + +import org.agrona.{DirectBuffer, MutableDirectBuffer} +import scalaxy.loops._ + +/** + * An implementation of the NibblePack algorithm for efficient encoding, see [[doc/compression.md]] + * Works with a predictor that maximizes zero bits/words of floating point or integer data. + */ +object NibblePack { + /** + * Packs Long values directly using NibblePack. Internally uses pack8. Inputs are not transformed. + */ + final def packNonIncreasing(input: Array[Long], buf: MutableDirectBuffer, bufindex: Int): Int = { + val inputArray = tempArray + var i = 0 + var pos = bufindex + while (i < input.size) { + inputArray(i % 8) = input(i) + i += 1 + if (i % 8 == 0) { + pos = pack8(inputArray, buf, pos) + } + } + + // Flush remainder - if any left + if (i % 8 != 0) { + for { j <- (i % 8) until 8 optimized } { inputArray(j) = 0 } + pos = pack8(inputArray, buf, pos) + } + + pos + } + + /** + * Packs Long values which must be positive and increasing. If the next value is lower than the previous value + * then a 0 is packed -- negative deltas are not allowed. + */ + final def packDelta(input: Array[Long], buf: MutableDirectBuffer, bufindex: Int): Int = { + val inputArray = tempArray + var last = 0L + var i = 0 + var pos = bufindex + while (i < input.size) { + val delta = if (input(i) >= last) input(i) - last else 0L + last = input(i) + inputArray(i % 8) = delta + i += 1 + if (i % 8 == 0) { + pos = pack8(inputArray, buf, pos) + } + } + + // Flush remainder - if any left + if (i % 8 != 0) { + for { j <- (i % 8) until 8 optimized } { inputArray(j) = 0 } + pos = pack8(inputArray, buf, pos) + } + + pos + } + + /** + * Packs 8 input values into a buffer using NibblePacking. Returns ending buffer position. + * This is an internal method, usually one wants to use one of the other pack* methods. + * @param buf the MutableDirectBuffer into which to write. Recommended is to use ExpandableArrayBuffer or + * ExpandableDirectByteBuffer so that it can grow as needed. + * @param bufindex the starting index of the output buffer into which to write + * @return the ending MutableDirectBuffer position + */ + final def pack8(input: Array[Long], buf: MutableDirectBuffer, bufindex: Int): Int = { + var bufpos = bufindex + require(input.size >= 8) + + var bitmask = 0 + // Figure out which words are nonzero, pack bitmask + for { i <- 0 until 8 optimized } { + if (input(i) != 0) bitmask |= 1 << i + } + buf.putByte(bufpos, bitmask.toByte) + bufpos += 1 + + if (bitmask != 0) { + // figure out min # of nibbles to represent nonzero words + var minLeadingZeros = 64 + var minTrailingZeros = 64 + for { i <- 0 until 8 optimized } { + minLeadingZeros = Math.min(minLeadingZeros, java.lang.Long.numberOfLeadingZeros(input(i))) + minTrailingZeros = Math.min(minTrailingZeros, java.lang.Long.numberOfTrailingZeros(input(i))) + } + + val trailingNibbles = minTrailingZeros / 4 + val numNibbles = 16 - (minLeadingZeros / 4) - trailingNibbles + val nibbleWord = (((numNibbles - 1) << 4) | trailingNibbles) + buf.putByte(bufpos, nibbleWord.toByte) + bufpos += 1 + + // Decide which packer to use + bufpos = packUniversal(input, buf, bufpos, numNibbles, trailingNibbles) + } + + bufpos + } + + // Returns the final bufpos + private def packUniversal(inputs: Array[Long], buf: MutableDirectBuffer, bufindex: Int, + numNibbles: Int, trailingZeroNibbles: Int): Int = { + var bufpos = bufindex + val trailingShift = trailingZeroNibbles * 4 + val numBits = numNibbles * 4 + var outWord = 0L + var bitCursor = 0 + + for { i <- 0 until 8 optimized } { + val input = inputs(i) + if (input != 0) { + val remaining = 64 - bitCursor + val shiftedInput = input >>> trailingShift + + // This is least significant portion of input + outWord |= shiftedInput << bitCursor + + // Write out current word if we've used up all 64 bits + if (remaining <= numBits) { + buf.putLong(bufpos, outWord, LITTLE_ENDIAN) + bufpos += 8 + if (remaining < numBits) { + // Most significant portion left over from previous word + outWord = shiftedInput >>> remaining + } else { + outWord = 0 // reset for 64-bit input case + } + } + + bitCursor = (bitCursor + numBits) % 64 + } + } + + // Write remainder word if there are any bits remaining, and only advance buffer right # of bytes + if (bitCursor > 0) { + buf.putLong(bufpos, outWord, LITTLE_ENDIAN) + bufpos += ((bitCursor + 7) / 8) + } + + bufpos + } + + private val tlTempArray = new ThreadLocal[Array[Long]]() + def tempArray: Array[Long] = tlTempArray.get match { + case UnsafeUtils.ZeroPointer => val newArray = new Array[Long](8) + tlTempArray.set(newArray) + newArray + case a: Array[Long] => a + } + + sealed trait UnpackResult + case object Ok extends UnpackResult + case object InputTooShort extends UnpackResult + + val empty = Array.empty[Byte] + + trait Sink { + def process(data: Long): Unit + } + + final case class DeltaSink(outArray: Array[Long]) extends Sink { + private var current: Long = 0L + private var pos: Int = 0 + def process(data: Long): Unit = { + // It's necessary to ignore "extra" elements because NibblePack always unpacks in multiples of 8, but the + // user might not intuitively allocate output arrays in elements of 8. + if (pos < outArray.size) { + current += data + outArray(pos) = current + pos += 1 + } + } + } + + /** + * Generic unpack function which outputs values to a Sink which can process raw 64-bit values from unpack8. + * @param compressed a DirectBuffer wrapping the compressed bytes. Position 0 must be the beginning of the buffer + * to unpack. NOTE: the passed in DirectBuffer will be mutated to wrap the NEXT bytes that + * can be unpacked. + */ + final def unpackToSink(compressed: DirectBuffer, sink: Sink, numValues: Int): UnpackResult = { + val array = tempArray + var res: UnpackResult = Ok + var valuesLeft = numValues + while (valuesLeft > 0 && res == Ok && compressed.capacity > 0) { + res = unpack8(compressed, array) + if (res == Ok) { + for { i <- 0 until 8 optimized } { + sink.process(array(i)) + } + valuesLeft -= 8 + } + } + res + } + + /** + * Unpacks 8 words of input at a time. + * @param compressed a DirectBuffer wrapping the compressed bytes. Position 0 must be the beginning of the buffer + * to unpack. NOTE: the passed in DirectBuffer will be mutated to wrap the NEXT bytes that + * can be unpacked. + * @param outArray an output array with at least 8 slots for Long values + * @return an UnpackResult + */ + final def unpack8(compressed: DirectBuffer, outArray: Array[Long]): UnpackResult = { + val nonzeroMask = compressed.getByte(0) + if (nonzeroMask == 0) { + java.util.Arrays.fill(outArray, 0L) + subslice(compressed, 1) + Ok + } else { + val numNibblesU8 = compressed.getByte(1) & 0x00ff // Make sure this is unsigned 8 bits! + val numBits = ((numNibblesU8 >>> 4) + 1) * 4 + val trailingZeroes = (numNibblesU8 & 0x0f) * 4 + val totalBytes = 2 + (numBits * java.lang.Integer.bitCount(nonzeroMask & 0x0ff) + 7) / 8 + val mask = if (numBits >= 64) -1L else (1L << numBits) - 1 + var bufIndex = 2 + var bitCursor = 0 + + var inWord = readLong(compressed, bufIndex) + bufIndex += 8 + + for { bit <- 0 until 8 optimized } { + if ((nonzeroMask & (1 << bit)) != 0) { + val remaining = 64 - bitCursor + + // Shift and read in LSB + val shiftedIn = inWord >>> bitCursor + var outWord = shiftedIn & mask + + // If remaining bits are in next word, read next word, unless no more space + if (remaining <= numBits && bufIndex < totalBytes) { + if (bufIndex < compressed.capacity) { + inWord = readLong(compressed, bufIndex) + bufIndex += 8 + if (remaining < numBits) { + outWord |= (inWord << remaining) & mask + } + } else { + return InputTooShort + } + } + + outArray(bit) = outWord << trailingZeroes + bitCursor = (bitCursor + numBits) % 64 + } else { + outArray(bit) = 0 + } + } + + // Return the "remaining slice" for easy and clean chaining of nibble_unpack8 calls with no mutable state + subslice(compressed, totalBytes) + Ok + } + } + + private def subslice(buffer: DirectBuffer, start: Int): Unit = + if (buffer.capacity > start) { + buffer.wrap(buffer, start, buffer.capacity - start) + } else { + buffer.wrap(empty) + } + + // Method to read a Long but ensure we don't step out of bounds at the end if we don't have 8 bytes left + private def readLong(inbuf: DirectBuffer, index: Int): Long = { + if ((index + 8) <= inbuf.capacity) { + inbuf.getLong(index, LITTLE_ENDIAN) + } else { + var i = 0 + var outWord = 0L + while (index + i < inbuf.capacity) { + outWord |= (inbuf.getByte(index + i) & 0x00ffL) << (8 * i) + i += 1 + } + outWord + } + } +} \ No newline at end of file diff --git a/memory/src/test/scala/filodb.memory/format/NibblePackTest.scala b/memory/src/test/scala/filodb.memory/format/NibblePackTest.scala new file mode 100644 index 0000000000..7065b6f9be --- /dev/null +++ b/memory/src/test/scala/filodb.memory/format/NibblePackTest.scala @@ -0,0 +1,121 @@ +package filodb.memory.format + +import org.agrona.ExpandableArrayBuffer +import org.agrona.concurrent.UnsafeBuffer + +import org.scalatest._ +import org.scalatest.prop.PropertyChecks + +class NibblePackTest extends FunSpec with Matchers with PropertyChecks { + it("should NibblePack 8 words partial non-zero even nibbles") { + // All 8 are nonzero, even # nibbles + val buf = new ExpandableArrayBuffer() + val inputs = Array(0L, + 0x0000003322110000L, 0x0000004433220000L, + 0x0000005544330000L, 0x0000006655440000L, + 0L, 0L, 0L) + val outpos = NibblePack.pack8(inputs, buf, 0) + + // Expected result: + val expectedBuf = Array[Byte]( + 0x1e, // 0b0001_1110u8, // only some bits on + 0x54, // six nibbles wide, four zero nibbles trailing + 0x11, 0x22, 0x33, 0x22, 0x33, 0x44, + 0x33, 0x44, 0x55, 0x44, 0x55, 0x66) + + outpos shouldEqual expectedBuf.length + buf.byteArray.take(expectedBuf.length) shouldEqual expectedBuf + } + + it("should NibblePack 8 words partial non-zero odd nibbles") { + // All 8 are nonzero, even # nibbles + val buf = new ExpandableArrayBuffer() + val inputs = Array(0L, + 0x0000003322100000L, 0x0000004433200000L, + 0x0000005544300000L, 0x0000006655400000L, + 0x0000007654300000L, 0L, 0L) + val outpos = NibblePack.pack8(inputs, buf, 0) + + // Expected result: + val expectedBuf = Array[Byte]( + 0x3e, // 0b0011_1110u8, // only some bits on + 0x45, // five nibbles wide, five zero nibbles trailing + 0x21, 0x32, 0x23, 0x33, 0x44, // First two values + 0x43, 0x54, 0x45, 0x55, 0x66, + 0x43, 0x65, 0x07 + ) + outpos shouldEqual expectedBuf.length + buf.byteArray.take(expectedBuf.length) shouldEqual expectedBuf + } + + it("should correctly unpack partial 8 words odd nibbles") { + val compressed = Array[Byte]( + 0x3e, // 0b0011_1110u8, // only some bits on + 0x45, // five nibbles wide, five zero nibbles trailing + 0x21, 0x32, 0x23, 0x33, 0x44, // First two values + 0x43, 0x54, 0x45, 0x55, 0x66, + 0x43, 0x65, 0x07 + ) + + val expected = Array(0L, + 0x0000003322100000L, 0x0000004433200000L, + 0x0000005544300000L, 0x0000006655400000L, + 0x0000007654300000L, 0L, 0L) + + val inbuf = new UnsafeBuffer(compressed) + val outarray = new Array[Long](8) + val res = NibblePack.unpack8(inbuf, outarray) + res shouldEqual NibblePack.Ok + + inbuf.capacity shouldEqual 0 + outarray shouldEqual expected + } + + it("should pack and unpack delta values") { + val inputs = Array(0L, 1000, 1001, 1002, 1003, 2005, 2010, 3034, 4045, 5056, 6067, 7078) + val buf = new ExpandableArrayBuffer() + val bytesWritten = NibblePack.packDelta(inputs, buf, 0) + + val sink = NibblePack.DeltaSink(new Array[Long](inputs.size)) + val bufSlice = new UnsafeBuffer(buf, 0, bytesWritten) + val res = NibblePack.unpackToSink(bufSlice, sink, inputs.size) + + res shouldEqual NibblePack.Ok + sink.outArray shouldEqual inputs + + val inputs2 = Array(10000, 1032583228027L) + val written2 = NibblePack.packDelta(inputs2, buf, 0) + val sink2 = NibblePack.DeltaSink(new Array[Long](inputs2.size)) + bufSlice.wrap(buf, 0, written2) + val res2 = NibblePack.unpackToSink(bufSlice, sink2, inputs2.size) + + res2 shouldEqual NibblePack.Ok + sink2.outArray shouldEqual inputs2 + } + + import org.scalacheck._ + + // Generate a list of increasing integers, every time bound it slightly differently + // (to test different int compression techniques) + def increasingLongList: Gen[Seq[Long]] = + for { + maxVal <- Gen.oneOf(1000, 5000, 30000, Math.pow(2L, 40).toLong) + seqList <- Gen.containerOf[Seq, Long](Gen.choose(10, maxVal)) + } yield { seqList.scanLeft(10000L)(_ + Math.abs(_)) } + + it("should pack and unpack random list of increasing Longs via delta") { + val buf = new ExpandableArrayBuffer() + forAll(increasingLongList) { longs => + + val inputs = longs.toArray + val bytesWritten = NibblePack.packDelta(inputs, buf, 0) + + val sink = NibblePack.DeltaSink(new Array[Long](inputs.size)) + val bufSlice = new UnsafeBuffer(buf, 0, bytesWritten) + val res = NibblePack.unpackToSink(bufSlice, sink, inputs.size) + + res shouldEqual NibblePack.Ok + sink.outArray shouldEqual inputs + } + } +} \ No newline at end of file diff --git a/memory/src/test/scala/filodb.memory/format/vectors/LongVectorTest.scala b/memory/src/test/scala/filodb.memory/format/vectors/LongVectorTest.scala index 761aef97a4..fe3d02dc50 100644 --- a/memory/src/test/scala/filodb.memory/format/vectors/LongVectorTest.scala +++ b/memory/src/test/scala/filodb.memory/format/vectors/LongVectorTest.scala @@ -246,35 +246,34 @@ class LongVectorTest extends NativeVectorTest with PropertyChecks { import org.scalacheck._ - // Generate a list of bounded integers, every time bound it slightly differently + // Generate a list of increasing integers, every time bound it slightly differently // (to test different int compression techniques) - def boundedIntList: Gen[Seq[Int]] = + def increasingIntList: Gen[Seq[Long]] = for { maxVal <- Gen.oneOf(1000, 5000, 30000) // must be greater than 250ms so not within approximation seqList <- Gen.containerOf[Seq, Int](Gen.choose(10, maxVal)) - } yield { seqList } + } yield { seqList.scanLeft(10000L)(_ + Math.abs(_)) } it("should binarySearch Long/DDV vector correctly for random elements and searches") { try { - forAll(boundedIntList) { s => - val increasingTimes = s.scanLeft(10000L)(_ + Math.abs(_)) - val builder = LongBinaryVector.appendingVectorNoNA(memFactory, increasingTimes.length) - increasingTimes.foreach(builder.addData) + forAll(increasingIntList) { longs => + val builder = LongBinaryVector.appendingVectorNoNA(memFactory, longs.length) + longs.foreach(builder.addData) val optimized = builder.optimize(memFactory) val reader = LongBinaryVector(optimized) - forAll(Gen.choose(0, increasingTimes.last * 3)) { num => + forAll(Gen.choose(0, longs.last * 3)) { num => val out = reader.binarySearch(optimized, num) (out & 0x7fffffff) should be >= 0 - val posMatch = increasingTimes.indexWhere(_ >= num) + val posMatch = longs.indexWhere(_ >= num) (out & 0x7fffffff) should be < 100000 if (posMatch >= 0) { - if (increasingTimes(posMatch) == num) { // exact match, or within 250ms + if (longs(posMatch) == num) { // exact match, or within 250ms out shouldEqual posMatch } else { // not match, but # at pos is first # greater than num. So insert here. out shouldEqual (posMatch | 0x80000000) } } else { // our # is greater than all numbers; ie _ >= num never true. - out shouldEqual (0x80000000 | increasingTimes.length) + out shouldEqual (0x80000000 | longs.length) } } } From 1fada065405f7545ba64e9aa0501f515f89b3d8d Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Mon, 11 Mar 2019 11:02:42 -0700 Subject: [PATCH 10/41] fix(memory): fix minor Scaladoc formatting issue (#273) --- memory/src/main/scala/filodb.memory/format/NibblePack.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/memory/src/main/scala/filodb.memory/format/NibblePack.scala b/memory/src/main/scala/filodb.memory/format/NibblePack.scala index 0bbe391ce3..0ab9d06287 100644 --- a/memory/src/main/scala/filodb.memory/format/NibblePack.scala +++ b/memory/src/main/scala/filodb.memory/format/NibblePack.scala @@ -6,7 +6,7 @@ import org.agrona.{DirectBuffer, MutableDirectBuffer} import scalaxy.loops._ /** - * An implementation of the NibblePack algorithm for efficient encoding, see [[doc/compression.md]] + * An implementation of the NibblePack algorithm for efficient encoding, see [compression.md](doc/compression.md). * Works with a predictor that maximizes zero bits/words of floating point or integer data. */ object NibblePack { From 768ac82b6250d2428b32fd30aeda539abfb8559a Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Tue, 12 Mar 2019 10:13:47 -0700 Subject: [PATCH 11/41] feat(memory): Integrate NibblePack-compression into serialized Histograms (#267) --- .../binaryrecord2/RecordBuilder.scala | 4 +- .../binaryrecord2/RecordSchema.scala | 13 +- .../filodb/http/ClusterApiRouteSpec.scala | 4 + .../filodb.jmh/HistogramIngestBenchmark.scala | 43 +++++- .../scala/filodb.memory/BinaryRegion.scala | 3 +- .../main/scala/filodb.memory/format/Ptr.scala | 21 +++ .../filodb.memory/format/RowReader.scala | 5 +- .../filodb.memory/format/UnsafeUtils.scala | 4 +- .../format/vectors/Histogram.scala | 137 +++++++++-------- .../format/vectors/HistogramVector.scala | 141 +++++++++--------- .../format/vectors/HistogramTest.scala | 16 +- .../format/vectors/HistogramVectorTest.scala | 10 +- .../query/exec/HistogramQuantileMapper.scala | 4 +- 13 files changed, 248 insertions(+), 157 deletions(-) create mode 100644 memory/src/main/scala/filodb.memory/format/Ptr.scala diff --git a/core/src/main/scala/filodb.core/binaryrecord2/RecordBuilder.scala b/core/src/main/scala/filodb.core/binaryrecord2/RecordBuilder.scala index 6b3b826006..c6f037b950 100644 --- a/core/src/main/scala/filodb.core/binaryrecord2/RecordBuilder.scala +++ b/core/src/main/scala/filodb.core/binaryrecord2/RecordBuilder.scala @@ -1,7 +1,7 @@ package filodb.core.binaryrecord2 import com.typesafe.scalalogging.StrictLogging -import org.agrona.concurrent.UnsafeBuffer +import org.agrona.DirectBuffer import scalaxy.loops._ import filodb.core.metadata.{Column, Dataset} @@ -140,7 +140,7 @@ final class RecordBuilder(memFactory: MemFactory, // Adds a blob from another buffer which already has the length bytes as the first two bytes // For example: buffers created by BinaryHistograms. OR, a UTF8String medium. - final def addBlob(buf: UnsafeBuffer): Unit = { + final def addBlob(buf: DirectBuffer): Unit = { val numBytes = buf.getShort(0).toInt require(numBytes < buf.capacity) addBlob(buf.byteArray, buf.addressOffset + 2, numBytes) diff --git a/core/src/main/scala/filodb.core/binaryrecord2/RecordSchema.scala b/core/src/main/scala/filodb.core/binaryrecord2/RecordSchema.scala index ff7a01161f..c7519dc9bb 100644 --- a/core/src/main/scala/filodb.core/binaryrecord2/RecordSchema.scala +++ b/core/src/main/scala/filodb.core/binaryrecord2/RecordSchema.scala @@ -2,6 +2,7 @@ package filodb.core.binaryrecord2 import scala.collection.mutable.ArrayBuffer +import org.agrona.DirectBuffer import org.agrona.concurrent.UnsafeBuffer import filodb.core.metadata.{Column, Dataset} @@ -144,11 +145,11 @@ final class RecordSchema(val columns: Seq[ColumnInfo], UTF8StringMedium.numBytes(base, offset + UnsafeUtils.getInt(base, offset + offsets(index))) /** - * Sets an existing UnsafeBuffer to wrap around the given blob/UTF8/Histogram bytes, including the - * 2-byte length prefix. Since the UnsafeBuffer is already allocated, this results in no new allocations. + * Sets an existing DirectBuffer to wrap around the given blob/UTF8/Histogram bytes, including the + * 2-byte length prefix. Since the DirectBuffer is already allocated, this results in no new allocations. * Could be used to efficiently retrieve blobs or histograms again and again. */ - def blobAsBuffer(base: Any, offset: Long, index: Int, buf: UnsafeBuffer): Unit = base match { + def blobAsBuffer(base: Any, offset: Long, index: Int, buf: DirectBuffer): Unit = base match { case a: Array[Byte] => buf.wrap(a, utf8StringOffset(base, offset, index).toInt - UnsafeUtils.arayOffset, blobNumBytes(base, offset, index) + 2) @@ -157,7 +158,7 @@ final class RecordSchema(val columns: Seq[ColumnInfo], } // Same as above but allocates a new UnsafeBuffer wrapping the blob as a reference - def blobAsBuffer(base: Any, offset: Long, index: Int): UnsafeBuffer = { + def blobAsBuffer(base: Any, offset: Long, index: Int): DirectBuffer = { val newBuf = new UnsafeBuffer(Array.empty[Byte]) blobAsBuffer(base, offset, index, newBuf) newBuf @@ -433,8 +434,8 @@ final class BinaryRecordRowReader(schema: RecordSchema, val buf = new UnsafeBuffer(Array.empty[Byte]) // NOTE: this method reuses the same buffer to avoid allocations. - override def blobAsBuffer(columnNo: Int): UnsafeBuffer = { - UnsafeUtils.wrapUnsafeBuf(recordBase, schema.utf8StringOffset(recordBase, recordOffset, columnNo), + override def blobAsBuffer(columnNo: Int): DirectBuffer = { + UnsafeUtils.wrapDirectBuf(recordBase, schema.utf8StringOffset(recordBase, recordOffset, columnNo), getBlobNumBytes(columnNo) + 2, buf) buf } diff --git a/http/src/test/scala/filodb/http/ClusterApiRouteSpec.scala b/http/src/test/scala/filodb/http/ClusterApiRouteSpec.scala index 0c170b08e8..c88f36f6d0 100644 --- a/http/src/test/scala/filodb/http/ClusterApiRouteSpec.scala +++ b/http/src/test/scala/filodb/http/ClusterApiRouteSpec.scala @@ -94,6 +94,10 @@ class ClusterApiRouteSpec extends FunSpec with ScalatestRouteTest with AsyncTest } println(s"Current statuses = $statuses") info(s"Current statuses = $statuses") + if (statuses.exists(_ == ShardStatusError)) { + info(s"ERROR in status, breaking") + throw new RuntimeException(s"Got error in statuses $statuses") + } } while (statuses.take(2) != Seq(ShardStatusActive, ShardStatusActive)) Get(s"/api/v1/cluster/${dataset6.ref}/status") ~> clusterRoute ~> check { diff --git a/jmh/src/main/scala/filodb.jmh/HistogramIngestBenchmark.scala b/jmh/src/main/scala/filodb.jmh/HistogramIngestBenchmark.scala index df297b0a82..f8df8dc4e4 100644 --- a/jmh/src/main/scala/filodb.jmh/HistogramIngestBenchmark.scala +++ b/jmh/src/main/scala/filodb.jmh/HistogramIngestBenchmark.scala @@ -4,6 +4,8 @@ import java.util.concurrent.TimeUnit import ch.qos.logback.classic.{Level, Logger} import com.typesafe.config.ConfigFactory +import org.agrona.concurrent.UnsafeBuffer +import org.agrona.ExpandableArrayBuffer import org.openjdk.jmh.annotations.{Level => JMHLevel, _} import filodb.core.{MachineMetricsData, MetricsTestData, TestData} @@ -11,7 +13,7 @@ import filodb.core.binaryrecord2.RecordBuilder import filodb.core.memstore._ import filodb.core.store._ import filodb.memory.MemFactory -import filodb.memory.format.SeqRowReader +import filodb.memory.format.{NibblePack, SeqRowReader} //scalastyle:off regex /** @@ -117,4 +119,43 @@ class HistogramIngestBenchmark { pShard.ingest(promContainers(containerNo), 0) containerNo += 1 } + + def nonzeroLongInputs(numNonZeroes: Int): Array[Long] = { + val longs = new Array[Long](64) + (1 to numNonZeroes).foreach { i => + longs(i) = (Math.sin(i * Math.PI / numNonZeroes) * 1000.0).toLong + } + longs + } + + def increasingNonzeroes(numNonZeroes: Int): Array[Long] = { + val longs = nonzeroLongInputs(numNonZeroes) + for { i <- 1 until 64 } { + longs(i) = longs(i - 1) + longs(i) + } + longs + } + + val inputs = increasingNonzeroes(16) + val buf = new ExpandableArrayBuffer() + + @Benchmark + @BenchmarkMode(Array(Mode.Throughput)) + @OutputTimeUnit(TimeUnit.SECONDS) + def nibblePackDelta64(): Int = { + NibblePack.packDelta(inputs, buf, 0) + } + + val bytesWritten = NibblePack.packDelta(inputs, buf, 0) + val sink = NibblePack.DeltaSink(new Array[Long](inputs.size)) + val bufSlice = new UnsafeBuffer(buf, 0, bytesWritten) + + @Benchmark + @BenchmarkMode(Array(Mode.Throughput)) + @OutputTimeUnit(TimeUnit.SECONDS) + def nibbleUnpackDelta64(): Unit = { + bufSlice.wrap(buf, 0, bytesWritten) + val res = NibblePack.unpackToSink(bufSlice, sink, inputs.size) + require(res == NibblePack.Ok) + } } \ No newline at end of file diff --git a/memory/src/main/scala/filodb.memory/BinaryRegion.scala b/memory/src/main/scala/filodb.memory/BinaryRegion.scala index ee70c74c33..37d2a77335 100644 --- a/memory/src/main/scala/filodb.memory/BinaryRegion.scala +++ b/memory/src/main/scala/filodb.memory/BinaryRegion.scala @@ -47,7 +47,8 @@ object BinaryRegion { def copyArray(source: Array[Byte], dest: Array[Byte], destOffset: Int): Unit = System.arraycopy(source, 0, dest, destOffset, source.size) - // 64-bit pointer to native/offheap memory + // 64-bit pointer to native/offheap memory. NOTE: instead of using this, please use the Ptr* + // value classes as they are much more type safe type NativePointer = Long } diff --git a/memory/src/main/scala/filodb.memory/format/Ptr.scala b/memory/src/main/scala/filodb.memory/format/Ptr.scala new file mode 100644 index 0000000000..5c8c2c8354 --- /dev/null +++ b/memory/src/main/scala/filodb.memory/format/Ptr.scala @@ -0,0 +1,21 @@ +package filodb.memory.format + +/** + * Strongly typed native memory pointer types / value classes. + * Do not incur allocations as long as value class rules are followed. + * Separate mutable and immutable types. + * Helps prevent accidental mixing of Int/Long types and pointers, which lead to SEGVs. + * Modeled after the equivalent Ptr types in Rust. + */ +object Ptr { + // A read-only pointer to bytes. Equivalent to *const u8 in Rust/C. No length safety, but much better than + // type equivalency to Long's. Use it to prevent type errors where one accidentally substitutes ints or longs. + final case class U8(addr: Long) extends AnyVal { + // Simple pointer math, by # of U8 elements (bytes) + final def add(offset: Int): U8 = U8(addr + offset) + //scalastyle:off method.name + final def +(offset: Int): U8 = U8(addr + offset) + + final def get: Byte = UnsafeUtils.getByte(addr) + } +} \ No newline at end of file diff --git a/memory/src/main/scala/filodb.memory/format/RowReader.scala b/memory/src/main/scala/filodb.memory/format/RowReader.scala index 06d7be8bcf..b29270802f 100644 --- a/memory/src/main/scala/filodb.memory/format/RowReader.scala +++ b/memory/src/main/scala/filodb.memory/format/RowReader.scala @@ -5,6 +5,7 @@ import java.sql.Timestamp import scala.reflect.ClassTag +import org.agrona.DirectBuffer import org.agrona.concurrent.UnsafeBuffer import org.joda.time.DateTime import scalaxy.loops._ @@ -39,9 +40,9 @@ trait RowReader { * Smart implementations could reuse the same UnsafeBuffer to avoid allocations. * This default implementation simply allocates a new one. */ - def blobAsBuffer(columnNo: Int): UnsafeBuffer = { + def blobAsBuffer(columnNo: Int): DirectBuffer = { val buf = new UnsafeBuffer(Array.empty[Byte]) - UnsafeUtils.wrapUnsafeBuf(getBlobBase(columnNo), getBlobOffset(columnNo), getBlobNumBytes(columnNo), buf) + UnsafeUtils.wrapDirectBuf(getBlobBase(columnNo), getBlobOffset(columnNo), getBlobNumBytes(columnNo), buf) buf } diff --git a/memory/src/main/scala/filodb.memory/format/UnsafeUtils.scala b/memory/src/main/scala/filodb.memory/format/UnsafeUtils.scala index 3cc6d58c44..50e4ecc3dc 100644 --- a/memory/src/main/scala/filodb.memory/format/UnsafeUtils.scala +++ b/memory/src/main/scala/filodb.memory/format/UnsafeUtils.scala @@ -3,7 +3,7 @@ package filodb.memory.format import java.nio.ByteBuffer import com.kenai.jffi.MemoryIO -import org.agrona.concurrent.UnsafeBuffer +import org.agrona.DirectBuffer import scalaxy.loops._ // scalastyle:off number.of.methods @@ -47,7 +47,7 @@ object UnsafeUtils { MemoryIO.getCheckedInstance.newDirectByteBuffer(address, size) } - def wrapUnsafeBuf(base: Any, offset: Long, numBytes: Int, buf: UnsafeBuffer): Unit = { + def wrapDirectBuf(base: Any, offset: Long, numBytes: Int, buf: DirectBuffer): Unit = { if (base != UnsafeUtils.ZeroPointer) { buf.wrap(base.asInstanceOf[Array[Byte]], offset.toInt - arayOffset, numBytes) } else { 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 c6f7df9535..1e89212cb5 100644 --- a/memory/src/main/scala/filodb.memory/format/vectors/Histogram.scala +++ b/memory/src/main/scala/filodb.memory/format/vectors/Histogram.scala @@ -1,9 +1,10 @@ package filodb.memory.format.vectors -import org.agrona.concurrent.UnsafeBuffer +import java.nio.ByteOrder.LITTLE_ENDIAN + +import org.agrona.{DirectBuffer, MutableDirectBuffer} import scalaxy.loops._ -import filodb.memory.BinaryRegion import filodb.memory.format._ /** @@ -28,13 +29,13 @@ trait Histogram extends Ordered[Histogram] { def bucketValue(no: Int): Double /** - * Returns an UnsafeBuffer pointing to a serialized BinaryHistogram representation of this histogram. + * Returns an MutableDirectBuffer pointing to a serialized BinaryHistogram representation of this histogram. * @param buf if Some(buf) supplied, then that buf is either written into or re-used to wrap where the serialized * representation is. The supplied buffer must be large enough to hold serialized histogram. * if None is passed, then the thread-local buffer may be re-used, in which case be careful as that * buffer will be mutated with the next call to serialize() within the same thread. */ - def serialize(intoBuf: Option[UnsafeBuffer] = None): UnsafeBuffer + def serialize(intoBuf: Option[MutableDirectBuffer] = None): MutableDirectBuffer /** * Finds the first bucket number with a value greater than or equal to the given "rank" @@ -119,7 +120,7 @@ object Histogram { final def numBuckets: Int = 0 final def bucketTop(no: Int): Double = ??? final def bucketValue(no: Int): Double = ??? - final def serialize(intoBuf: Option[UnsafeBuffer] = None): UnsafeBuffer = { + final def serialize(intoBuf: Option[MutableDirectBuffer] = None): MutableDirectBuffer = { val buf = intoBuf.getOrElse(BinaryHistogram.histBuf) BinaryHistogram.writeNonIncreasing(HistogramBuckets.emptyBuckets, Array[Long](), buf) buf @@ -127,16 +128,41 @@ object Histogram { } } +trait HistogramWithBuckets extends Histogram { + def buckets: HistogramBuckets + final def numBuckets: Int = buckets.numBuckets + final def bucketTop(no: Int): Double = buckets.bucketTop(no) +} + +final case class LongHistogram(buckets: HistogramBuckets, values: Array[Long]) extends HistogramWithBuckets { + final def bucketValue(no: Int): Double = values(no).toDouble + final def serialize(intoBuf: Option[MutableDirectBuffer] = None): MutableDirectBuffer = { + val buf = intoBuf.getOrElse(BinaryHistogram.histBuf) + buckets match { + case g: GeometricBuckets => BinaryHistogram.writeDelta(g, values, buf) + } + buf + } +} + +object LongHistogram { + def fromPacked(bucketDef: HistogramBuckets, packedValues: DirectBuffer): Option[LongHistogram] = { + val values = new Array[Long](bucketDef.numBuckets) + val res = NibblePack.unpackToSink(packedValues, NibblePack.DeltaSink(values), bucketDef.numBuckets) + if (res == NibblePack.Ok) Some(LongHistogram(bucketDef, values)) else None + } +} + /** * A histogram class that can be used for aggregation and to represent intermediate values */ -final case class MutableHistogram(buckets: HistogramBuckets, values: Array[Double]) extends Histogram { - final def numBuckets: Int = buckets.numBuckets - final def bucketTop(no: Int): Double = buckets.bucketTop(no) +final case class MutableHistogram(buckets: HistogramBuckets, values: Array[Double]) extends HistogramWithBuckets { final def bucketValue(no: Int): Double = values(no) - final def serialize(intoBuf: Option[UnsafeBuffer] = None): UnsafeBuffer = { + final def serialize(intoBuf: Option[MutableDirectBuffer] = None): MutableDirectBuffer = { val buf = intoBuf.getOrElse(BinaryHistogram.histBuf) - BinaryHistogram.writeNonIncreasing(buckets, values.map(_.toLong), buf) + buckets match { + case g: GeometricBuckets => BinaryHistogram.writeDelta(g, values.map(_.toLong), buf) + } buf } @@ -149,10 +175,10 @@ final case class MutableHistogram(buckets: HistogramBuckets, values: Array[Doubl * Adds the values from another MutableHistogram having the same bucket schema. If it does not, then * an exception is thrown -- for now. Modifies itself. */ - final def add(other: MutableHistogram): Unit = + final def add(other: HistogramWithBuckets): Unit = if (buckets == other.buckets) { for { b <- 0 until numBuckets optimized } { - values(b) += other.values(b) + values(b) += other.bucketValue(b) } } else { throw new UnsupportedOperationException(s"Cannot add other with buckets ${other.buckets} to myself $buckets") @@ -178,11 +204,6 @@ sealed trait HistogramBuckets { */ def bucketTop(no: Int): Double - /** - * Serializes this bucket scheme to a byte array - */ - def toByteArray: Array[Byte] - /** * Materializes all bucket tops into an array. WARNING: Allocation. */ @@ -206,68 +227,62 @@ object HistogramBuckets { /** * Creates the right HistogramBuckets from a binary definition. NOTE: length field not included here * The first two bytes of any binary bucket schema definition are the number of buckets. + * @param buffer a DirectBuffer with index 0 pointing to the u16/Short length bytes */ - def apply(binaryBucketsDef: Array[Byte], formatCode: Byte): HistogramBuckets = formatCode match { - case HistFormat_Geometric_Delta => geometric(binaryBucketsDef, UnsafeUtils.arayOffset) - case HistFormat_Geometric1_Delta => geometric1(binaryBucketsDef, UnsafeUtils.arayOffset) + def apply(buffer: DirectBuffer, formatCode: Byte): HistogramBuckets = formatCode match { + case HistFormat_Geometric_Delta => geometric(buffer.byteArray, buffer.addressOffset + 2, false) + case HistFormat_Geometric1_Delta => geometric(buffer.byteArray, buffer.addressOffset + 2, true) case _ => emptyBuckets } - def apply(bucketsDef: BinaryRegion.NativePointer, formatCode: Byte): HistogramBuckets = formatCode match { - case HistFormat_Geometric_Delta => geometric(UnsafeUtils.ZeroArray, bucketsDef) - case HistFormat_Geometric1_Delta => geometric1(UnsafeUtils.ZeroArray, bucketsDef) + // NOTE: must point to u16/Short length prefix bytes + def apply(bucketsDef: Ptr.U8, formatCode: Byte): HistogramBuckets = formatCode match { + case HistFormat_Geometric_Delta => geometric(UnsafeUtils.ZeroArray, bucketsDef.add(2).addr, false) + case HistFormat_Geometric1_Delta => geometric(UnsafeUtils.ZeroArray, bucketsDef.add(2).addr, true) case _ => emptyBuckets } // Create geometric buckets definition - def geometric(bucketsDefBase: Array[Byte], bucketsDefOffset: Long): HistogramBuckets = + def geometric(bucketsDefBase: Array[Byte], bucketsDefOffset: Long, minusOne: Boolean): HistogramBuckets = GeometricBuckets(UnsafeUtils.getDouble(bucketsDefBase, bucketsDefOffset + OffsetBucketDetails), UnsafeUtils.getDouble(bucketsDefBase, bucketsDefOffset + OffsetBucketDetails + 8), - UnsafeUtils.getShort(bucketsDefBase, bucketsDefOffset + OffsetNumBuckets).toInt) - - def geometric1(bucketsDefBase: Array[Byte], bucketsDefOffset: Long): HistogramBuckets = - GeometricBuckets_1(UnsafeUtils.getDouble(bucketsDefBase, bucketsDefOffset + OffsetBucketDetails), - UnsafeUtils.getDouble(bucketsDefBase, bucketsDefOffset + OffsetBucketDetails + 8), - UnsafeUtils.getShort(bucketsDefBase, bucketsDefOffset + OffsetNumBuckets).toInt) - - /** - * Creates a binary bucket definition for a geometric series of histogram buckets. - * That means each bucket contains values <= (firstBucket) * (multipler) ^ (bucketNo) - * where bucketNo starts at 0 and goes till (numBuckets - 1) - * @param firstBucket initial bucket value - * @param multiplier the geometric multiplier between buckets - * @param numBuckets the total number of buckets - */ - final def geometricBucketDef(firstBucket: Double, - multiplier: Double, - numBuckets: Int): Array[Byte] = { - require(numBuckets < 65536, s"Too many buckets: $numBuckets") - val bytes = new Array[Byte](18) - UnsafeUtils.setShort(bytes, UnsafeUtils.arayOffset + OffsetNumBuckets, numBuckets.toShort) - UnsafeUtils.setDouble(bytes, UnsafeUtils.arayOffset + OffsetBucketDetails, firstBucket) - UnsafeUtils.setDouble(bytes, UnsafeUtils.arayOffset + OffsetBucketDetails + 8, multiplier) - bytes - } + UnsafeUtils.getShort(bucketsDefBase, bucketsDefOffset + OffsetNumBuckets).toInt, + minusOne) // A bucket definition for the bits of a long, ie from 2^0 to 2^63 // le's = [1, 3, 7, 15, 31, ....] - val binaryBuckets64 = GeometricBuckets_1(2.0d, 2.0d, 64) + val binaryBuckets64 = GeometricBuckets(2.0d, 2.0d, 64, minusOne = true) val emptyBuckets = GeometricBuckets(2.0d, 2.0d, 0) } /** * A geometric series bucketing scheme, where each successive bucket is a multiple of a previous one. + * That means each bucket contains values <= (firstBucket) * (multipler) ^ (bucketNo) + * where bucketNo starts at 0 and goes till (numBuckets - 1) + * @param minusOne if true, subtract 1 from the bucket top. Used for exclusive "not including" */ -final case class GeometricBuckets(firstBucket: Double, multiplier: Double, numBuckets: Int) extends HistogramBuckets { - final def bucketTop(no: Int): Double = firstBucket * Math.pow(multiplier, no) - def toByteArray: Array[Byte] = HistogramBuckets.geometricBucketDef(firstBucket, multiplier, numBuckets) -} - -/** - * A variation of a geometric series scheme where the bucket values are 1 less than the geoemtric series. - */ -final case class GeometricBuckets_1(firstBucket: Double, multiplier: Double, numBuckets: Int) extends HistogramBuckets { - final def bucketTop(no: Int): Double = (firstBucket * Math.pow(multiplier, no)) - 1 - def toByteArray: Array[Byte] = HistogramBuckets.geometricBucketDef(firstBucket, multiplier, numBuckets) +final case class GeometricBuckets(firstBucket: Double, + multiplier: Double, + numBuckets: Int, + minusOne: Boolean = false) extends HistogramBuckets { + private val adjustment = if (minusOne) -1 else 0 + final def bucketTop(no: Int): Double = (firstBucket * Math.pow(multiplier, no)) + adjustment + + import HistogramBuckets._ + /** + * Serializes this bucket definition to a mutable buffer, including writing length bytes + * @param buf the buffer to write to + * @param pos the position within buffer to write to + * @return the final position + */ + final def serialize(buf: MutableDirectBuffer, pos: Int): Int = { + require(numBuckets < 65536, s"Too many buckets: $numBuckets") + val numBucketsPos = pos + 2 + buf.putShort(pos, 2 + 8 + 8) + buf.putShort(numBucketsPos, numBuckets.toShort, LITTLE_ENDIAN) + buf.putDouble(numBucketsPos + OffsetBucketDetails, firstBucket, LITTLE_ENDIAN) + buf.putDouble(numBucketsPos + OffsetBucketDetails + 8, multiplier, LITTLE_ENDIAN) + pos + 2 + 2 + 8 + 8 + } } 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 b8db4de191..5e145e10ca 100644 --- a/memory/src/main/scala/filodb.memory/format/vectors/HistogramVector.scala +++ b/memory/src/main/scala/filodb.memory/format/vectors/HistogramVector.scala @@ -1,6 +1,8 @@ package filodb.memory.format.vectors +import com.typesafe.scalalogging.StrictLogging import debox.Buffer +import org.agrona.{DirectBuffer, ExpandableArrayBuffer, MutableDirectBuffer} import org.agrona.concurrent.UnsafeBuffer import scalaxy.loops._ @@ -16,8 +18,8 @@ import filodb.memory.format.Encodings._ * +0000 u16 2-byte total length of this BinaryHistogram (excluding this length) * +0002 u8 1-byte combined histogram buckets and values format code * 0x00 Empty/null histogram - * 0x01 geometric + NibblePacked delta Long values - * 0x02 geometric_1 + NibblePacked delta Long values (see [[HistogramBuckets]]) + * 0x03 geometric + NibblePacked delta Long values + * 0x04 geometric_1 + NibblePacked delta Long values (see [[HistogramBuckets]]) * * +0003 u16 2-byte length of Histogram bucket definition * +0005 [u8] Histogram bucket definition, see [[HistogramBuckets]] @@ -26,9 +28,9 @@ import filodb.memory.format.Encodings._ * * NOTE: most of the methods below actually expect a pointer to the +2 hist bucket definition, not the length field */ -object BinaryHistogram { +object BinaryHistogram extends StrictLogging { // Pass in a buffer which includes the length bytes. Value class - no allocations. - case class BinHistogram(buf: UnsafeBuffer) extends AnyVal { + case class BinHistogram(buf: DirectBuffer) extends AnyVal { def totalLength: Int = buf.getShort(0).toInt + 2 def numBuckets: Int = buf.getShort(5).toInt def formatCode: Byte = buf.getByte(2) @@ -36,8 +38,10 @@ object BinaryHistogram { def bucketDefOffset: Long = buf.addressOffset + 5 def valuesIndex: Int = 2 + 3 + bucketDefNumBytes // pointer to values bytes def valuesNumBytes: Int = totalLength - valuesIndex - def intoValuesBuf(destBuf: UnsafeBuffer): Unit = - UnsafeUtils.wrapUnsafeBuf(buf.byteArray, buf.addressOffset + valuesIndex, valuesNumBytes, destBuf) + def valuesByteSlice: DirectBuffer = { + UnsafeUtils.wrapDirectBuf(buf.byteArray, buf.addressOffset + valuesIndex, valuesNumBytes, valuesBuf) + valuesBuf + } override def toString: String = s"" /** @@ -48,82 +52,81 @@ object BinaryHistogram { */ def toHistogram: Histogram = formatCode match { case HistFormat_Geometric_Delta => - val bucketDef = HistogramBuckets.geometric(buf.byteArray, bucketDefOffset) - // TODO: flat buckets won't be supported anymore. Fix this - FlatBucketHistogram(bucketDef, this) + val bucketDef = HistogramBuckets.geometric(buf.byteArray, bucketDefOffset, false) + LongHistogram.fromPacked(bucketDef, valuesByteSlice).getOrElse(Histogram.empty) case HistFormat_Geometric1_Delta => - val bucketDef = HistogramBuckets.geometric1(buf.byteArray, bucketDefOffset) - // TODO: flat buckets won't be supported anymore. Fix this - FlatBucketHistogram(bucketDef, this) + val bucketDef = HistogramBuckets.geometric(buf.byteArray, bucketDefOffset, true) + LongHistogram.fromPacked(bucketDef, valuesByteSlice).getOrElse(Histogram.empty) + case x => + logger.debug(s"Unrecognizable histogram format code $x, returning empty histogram") + Histogram.empty } } - private val tlValuesBuf = new ThreadLocal[UnsafeBuffer]() - def valuesBuf: UnsafeBuffer = tlValuesBuf.get match { - case UnsafeUtils.ZeroPointer => val buf = new UnsafeBuffer(new Array[Byte](4096)) + // Thread local buffer used as read-only byte slice + private val tlValuesBuf = new ThreadLocal[DirectBuffer]() + def valuesBuf: DirectBuffer = tlValuesBuf.get match { + case UnsafeUtils.ZeroPointer => val buf = new UnsafeBuffer(Array.empty[Byte]) tlValuesBuf.set(buf) buf - case b: UnsafeBuffer => b + case b: DirectBuffer => b } // Thread local buffer used as temp buffer for writing binary histograms - private val tlHistBuf = new ThreadLocal[UnsafeBuffer]() - def histBuf: UnsafeBuffer = tlHistBuf.get match { - case UnsafeUtils.ZeroPointer => val buf = new UnsafeBuffer(new Array[Byte](8192)) + private val tlHistBuf = new ThreadLocal[MutableDirectBuffer]() + def histBuf: MutableDirectBuffer = tlHistBuf.get match { + case UnsafeUtils.ZeroPointer => val buf = new ExpandableArrayBuffer(4096) tlHistBuf.set(buf) buf - case b: UnsafeBuffer => b + case b: MutableDirectBuffer => b } val HistFormat_Null = 0x00.toByte - val HistFormat_Geometric_Delta = 0x01.toByte - val HistFormat_Geometric1_Delta = 0x02.toByte - - case class FlatBucketValues(buf: UnsafeBuffer) extends AnyVal { - def bucket(no: Int): Long = buf.getLong(no * 8) - } - - private case class FlatBucketHistogram(buckets: HistogramBuckets, binHist: BinHistogram) extends Histogram { - binHist.intoValuesBuf(valuesBuf) - final def numBuckets: Int = buckets.numBuckets - final def bucketTop(no: Int): Double = buckets.bucketTop(no) - final def bucketValue(no: Int): Double = FlatBucketValues(valuesBuf).bucket(no) - final def serialize(intoBuf: Option[UnsafeBuffer] = None): UnsafeBuffer = - intoBuf.map { x => x.wrap(binHist.buf); x }.getOrElse(binHist.buf) - } + val HistFormat_Geometric_Delta = 0x03.toByte + val HistFormat_Geometric1_Delta = 0x04.toByte /** * Writes binary histogram with geometric bucket definition and data which is non-increasing, but will be * decoded as increasing. Intended only for specific use cases when the source histogram are non increasing * buckets, ie each bucket has a count that is independent. - * Buckets are written as-is for now. + * @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 writeNonIncreasing(buckets: HistogramBuckets, values: Array[Long], buf: UnsafeBuffer): Int = { - val formatCode = buckets match { - case g: GeometricBuckets => HistFormat_Geometric_Delta - case g: GeometricBuckets_1 => HistFormat_Geometric1_Delta - case _ => ??? - } - val bucketDef = buckets.toByteArray - - val bytesNeeded = 2 + 1 + 2 + bucketDef.size + 8 * values.size - require(bytesNeeded < 65535, s"Histogram data is too large: $bytesNeeded bytes needed") - require(buf.capacity >= bytesNeeded, s"Buffer only has ${buf.capacity} bytes but we need $bytesNeeded") + def writeNonIncreasing(buckets: GeometricBuckets, values: Array[Long], buf: MutableDirectBuffer): Int = { + val formatCode = if (buckets.minusOne) HistFormat_Geometric1_Delta else HistFormat_Geometric_Delta - buf.putShort(0, (bytesNeeded - 2).toShort) buf.putByte(2, formatCode) - buf.putShort(3, bucketDef.size.toShort) - buf.putBytes(5, bucketDef) - val valuesIndex = 5 + bucketDef.size - for { b <- 0 until values.size optimized } { - buf.putLong(valuesIndex + b * 8, values(b)) - } - bytesNeeded + val valuesIndex = buckets.serialize(buf, 3) + val finalPos = NibblePack.packNonIncreasing(values, buf, valuesIndex) + + require(finalPos <= 65535, s"Histogram data is too large: $finalPos bytes needed") + buf.putShort(0, (finalPos - 2).toShort) + finalPos } - def writeNonIncreasing(buckets: HistogramBuckets, values: Array[Long]): Int = + def writeDelta(buckets: GeometricBuckets, values: Array[Long]): Int = writeNonIncreasing(buckets, values, histBuf) + + /** + * Encodes binary histogram with geometric bucket definition and data which is strictly increasing 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. + * so it can grow. + * @return the number of bytes written, including the length prefix + */ + def writeDelta(buckets: GeometricBuckets, values: Array[Long], buf: MutableDirectBuffer): Int = { + val formatCode = if (buckets.minusOne) HistFormat_Geometric1_Delta else HistFormat_Geometric_Delta + + buf.putByte(2, formatCode) + val valuesIndex = buckets.serialize(buf, 3) + val finalPos = NibblePack.packDelta(values, buf, valuesIndex) + + require(finalPos <= 65535, s"Histogram data is too large: $finalPos bytes needed") + buf.putShort(0, (finalPos - 2).toShort) + finalPos + } } object HistogramVector { @@ -145,10 +148,10 @@ object HistogramVector { UnsafeUtils.setShort(addr + OffsetNumHistograms, (getNumHistograms(addr) + 1).toShort) final def formatCode(addr: BinaryVectorPtr): Byte = UnsafeUtils.getByte(addr + OffsetFormatCode) - final def afterBucketDefAddr(addr: BinaryVectorPtr): BinaryRegion.NativePointer = - addr + OffsetBucketDef + bucketDefNumBytes(addr) + final def afterBucketDefAddr(addr: BinaryVectorPtr): Ptr.U8 = + Ptr.U8(addr) + OffsetBucketDef + bucketDefNumBytes(addr) final def bucketDefNumBytes(addr: BinaryVectorPtr): Int = UnsafeUtils.getShort(addr + OffsetBucketDefSize).toInt - final def bucketDefAddr(addr: BinaryVectorPtr): BinaryRegion.NativePointer = addr + OffsetBucketDef + final def bucketDefAddr(addr: BinaryVectorPtr): Ptr.U8 = Ptr.U8(addr) + OffsetBucketDef // Matches the bucket definition whose # bytes is at (base, offset) final def matchBucketDef(hist: BinaryHistogram.BinHistogram, addr: BinaryVectorPtr): Boolean = @@ -163,7 +166,7 @@ object HistogramVector { val bucketAddrPtr = afterBucketDefAddr(addr) val headerBytes = UnsafeUtils.getInt(addr) headerBytes + (0 until getNumBuckets(addr)).map { b => - val bucketVectorAddr = UnsafeUtils.getLong(bucketAddrPtr + 8*b) + val bucketVectorAddr = UnsafeUtils.getLong((bucketAddrPtr + 8*b).addr) UnsafeUtils.getInt(bucketVectorAddr) + 4 }.sum } @@ -226,7 +229,7 @@ class ColumnarAppendableHistogramVector(factory: MemFactory, if (numItems == 0) { // Copy the bucket definition and set the bucket def size UnsafeUtils.unsafe.copyMemory(buf.byteArray, h.bucketDefOffset, - UnsafeUtils.ZeroPointer, bucketDefAddr(addr), h.bucketDefNumBytes) + UnsafeUtils.ZeroPointer, bucketDefAddr(addr).addr, h.bucketDefNumBytes) UnsafeUtils.setShort(addr + OffsetBucketDefSize, h.bucketDefNumBytes.toShort) UnsafeUtils.setByte(addr + OffsetFormatCode, h.formatCode) @@ -240,11 +243,10 @@ class ColumnarAppendableHistogramVector(factory: MemFactory, } // Now, iterate through the counters and add them to each individual vector - h.intoValuesBuf(valueBuf) - val values = FlatBucketValues(valueBuf) + val hist = h.toHistogram bucketAppenders.foreach { appenders => for { b <- 0 until numBuckets optimized } { - val resp = appenders(b).addData(values.bucket(b)) + val resp = appenders(b).addData(hist.bucketValue(b).toLong) require(resp == Ack) } } @@ -255,7 +257,8 @@ class ColumnarAppendableHistogramVector(factory: MemFactory, final def addNA(): AddResponse = Ack // TODO: Add a 0 to every appender - def addFromReaderNoNA(reader: RowReader, col: Int): AddResponse = addData(reader.blobAsBuffer(col)) + def addFromReaderNoNA(reader: RowReader, col: Int): AddResponse = + addData(reader.blobAsBuffer(col).asInstanceOf[UnsafeBuffer]) def copyToBuffer: Buffer[UnsafeBuffer] = ??? def apply(index: Int): UnsafeBuffer = ??? @@ -285,7 +288,7 @@ class ColumnarAppendableHistogramVector(factory: MemFactory, val newHeaderAddr = memFactory.allocateOffheap(numBytes) // Copy headers including bucket def - val bucketPtrOffset = (afterBucketDefAddr(addr) - addr).toInt + val bucketPtrOffset = (afterBucketDefAddr(addr).addr - addr).toInt UnsafeUtils.copy(addr, newHeaderAddr, bucketPtrOffset) for { b <- 0 until optimizedBuckets.size optimized } { @@ -298,7 +301,7 @@ class ColumnarAppendableHistogramVector(factory: MemFactory, // NOTE: allocating vectors during ingestion is a REALLY BAD idea. For one if one runs out of memory then // it will fail but ingestion into other vectors might succeed, resulting in undefined switchBuffers behaviors. private def initBuckets(numBuckets: Int): Unit = { - val bucketPointersAddr = afterBucketDefAddr(addr) + val bucketPointersAddr = afterBucketDefAddr(addr).addr val appenders = (0 until numBuckets).map { b => val appender = LongBinaryVector.appendingVectorNoNA(factory, maxItems) UnsafeUtils.setLong(bucketPointersAddr + 8*b, appender.addr) @@ -323,14 +326,14 @@ class ColumnarHistogramReader(histVect: BinaryVectorPtr) extends HistogramReader final def length: Int = getNumHistograms(histVect) val numBuckets = if (length > 0) getNumBuckets(histVect) else 0 val bucketAddrs = if (length > 0) { - val bucketAddrBase = afterBucketDefAddr(histVect) + val bucketAddrBase = afterBucketDefAddr(histVect).addr (0 until numBuckets).map(b => UnsafeUtils.getLong(bucketAddrBase + 8 * b)).toArray } else { Array.empty[BinaryVectorPtr] } val readers = if (length > 0) bucketAddrs.map(LongBinaryVector.apply) else Array.empty[LongVectorDataReader] - val buckets = HistogramBuckets(bucketDefAddr(histVect), formatCode(histVect)) + val buckets = HistogramBuckets(bucketDefAddr(histVect).add(-2), formatCode(histVect)) val returnHist = MutableHistogram.empty(buckets) /** 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 ceebdec65d..dbe9ac83f5 100644 --- a/memory/src/test/scala/filodb.memory/format/vectors/HistogramTest.scala +++ b/memory/src/test/scala/filodb.memory/format/vectors/HistogramTest.scala @@ -1,6 +1,6 @@ package filodb.memory.format.vectors -import org.agrona.concurrent.UnsafeBuffer +import org.agrona.ExpandableArrayBuffer object HistogramTest { val bucketScheme = GeometricBuckets(1.0, 2.0, 8) @@ -21,21 +21,25 @@ object HistogramTest { import BinaryHistogram._ class HistogramTest extends NativeVectorTest { + val writeBuf = new ExpandableArrayBuffer() + describe("HistogramBuckets") { it("can list out bucket definition LE values properly for Geometric and Geometric_1") { val buckets1 = GeometricBuckets(5.0, 3.0, 4) buckets1.allBucketTops shouldEqual Array(5.0, 15.0, 45.0, 135.0) - val buckets2 = GeometricBuckets_1(2.0, 2.0, 8) + val buckets2 = GeometricBuckets(2.0, 2.0, 8, minusOne = true) buckets2.allBucketTops shouldEqual Array(1.0, 3.0, 7.0, 15.0, 31.0, 63.0, 127.0, 255.0) } it("can serialize and deserialize properly") { val buckets1 = GeometricBuckets(5.0, 2.0, 4) - HistogramBuckets(buckets1.toByteArray, HistFormat_Geometric_Delta) shouldEqual buckets1 + buckets1.serialize(writeBuf, 0) shouldEqual 2+2+8+8 + HistogramBuckets(writeBuf, HistFormat_Geometric_Delta) shouldEqual buckets1 - val buckets2 = GeometricBuckets_1(2.0, 2.0, 8) - HistogramBuckets(buckets2.toByteArray, HistFormat_Geometric1_Delta) shouldEqual buckets2 + val buckets2 = GeometricBuckets(2.0, 2.0, 8, minusOne = true) + buckets2.serialize(writeBuf, 0) shouldEqual 2+2+8+8 + HistogramBuckets(writeBuf, HistFormat_Geometric1_Delta) shouldEqual buckets2 } } @@ -52,7 +56,7 @@ class HistogramTest extends NativeVectorTest { it("should serialize to and from BinaryHistograms and compare correctly") { val binHistograms = mutableHistograms.map { h => - val buf = new UnsafeBuffer(new Array[Byte](2048)) + val buf = new ExpandableArrayBuffer() h.serialize(Some(buf)) } diff --git a/memory/src/test/scala/filodb.memory/format/vectors/HistogramVectorTest.scala b/memory/src/test/scala/filodb.memory/format/vectors/HistogramVectorTest.scala index 3ca12d372b..24a5655ea4 100644 --- a/memory/src/test/scala/filodb.memory/format/vectors/HistogramVectorTest.scala +++ b/memory/src/test/scala/filodb.memory/format/vectors/HistogramVectorTest.scala @@ -32,7 +32,7 @@ class HistogramVectorTest extends NativeVectorTest { it("should accept BinaryHistograms of the same schema and be able to query them") { val appender = HistogramVector.appendingColumnar(memFactory, 8, 50) rawLongBuckets.foreach { rawBuckets => - BinaryHistogram.writeNonIncreasing(bucketScheme, rawBuckets, buffer) + BinaryHistogram.writeDelta(bucketScheme, rawBuckets, buffer) appender.addData(buffer) shouldEqual Ack } @@ -53,7 +53,7 @@ class HistogramVectorTest extends NativeVectorTest { it("should optimize histograms and be able to query optimized vectors") { val appender = HistogramVector.appendingColumnar(memFactory, 8, 50) rawLongBuckets.foreach { rawBuckets => - BinaryHistogram.writeNonIncreasing(bucketScheme, rawBuckets, buffer) + BinaryHistogram.writeDelta(bucketScheme, rawBuckets, buffer) appender.addData(buffer) shouldEqual Ack } @@ -86,21 +86,21 @@ class HistogramVectorTest extends NativeVectorTest { it("should reject BinaryHistograms of schema different from first schema ingested") { val appender = HistogramVector.appendingColumnar(memFactory, 8, 50) rawLongBuckets.foreach { rawBuckets => - BinaryHistogram.writeNonIncreasing(bucketScheme, rawBuckets, buffer) + BinaryHistogram.writeDelta(bucketScheme, rawBuckets, buffer) appender.addData(buffer) shouldEqual Ack } appender.length shouldEqual rawHistBuckets.length // A record using a different schema - BinaryHistogram.writeNonIncreasing(HistogramBuckets.binaryBuckets64, Array[Long](0, 1, 2, 0), buffer) + BinaryHistogram.writeDelta(HistogramBuckets.binaryBuckets64, Array[Long](0, 1, 2, 0), buffer) appender.addData(buffer) shouldEqual BucketSchemaMismatch } it("should reject new adds when vector is full") { val appender = HistogramVector.appendingColumnar(memFactory, 8, 4) rawLongBuckets.foreach { rawBuckets => - BinaryHistogram.writeNonIncreasing(bucketScheme, rawBuckets, buffer) + BinaryHistogram.writeDelta(bucketScheme, rawBuckets, buffer) appender.addData(buffer) shouldEqual Ack } diff --git a/query/src/main/scala/filodb/query/exec/HistogramQuantileMapper.scala b/query/src/main/scala/filodb/query/exec/HistogramQuantileMapper.scala index 6a3d5d50fd..31275d15ed 100644 --- a/query/src/main/scala/filodb/query/exec/HistogramQuantileMapper.scala +++ b/query/src/main/scala/filodb/query/exec/HistogramQuantileMapper.scala @@ -1,7 +1,7 @@ package filodb.query.exec import monix.reactive.Observable -import org.agrona.concurrent.UnsafeBuffer +import org.agrona.MutableDirectBuffer import filodb.core.query._ import filodb.memory.format.{RowReader, ZeroCopyUTF8String} @@ -107,7 +107,7 @@ case class HistogramQuantileMapper(funcParams: Seq[Any]) extends RangeVectorTran final def numBuckets: Int = buckets.size final def bucketTop(no: Int): Double = buckets(no).le final def bucketValue(no: Int): Double = buckets(no).rate - final def serialize(intoBuf: Option[UnsafeBuffer] = None): UnsafeBuffer = ??? + final def serialize(intoBuf: Option[MutableDirectBuffer] = None): MutableDirectBuffer = ??? } /** From ea55bd5ea15e05b2d3533f346ae261b618e8e793 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Tue, 12 Mar 2019 10:33:34 -0700 Subject: [PATCH 12/41] feat/fix(core,query): Remove binaryrecord v1 and associated SEGVs (#275) --- .../client/FiloKryoSerializers.scala | 26 -- .../client/Serializer.scala | 23 +- .../queryengine2/QueryEngine.scala | 19 +- .../binaryrecord/BinaryRecord.scala | 349 ------------------ .../filodb.core/binaryrecord/FieldType.scala | 237 ------------ .../scala/filodb.core/binaryrecord/README.md | 29 -- .../binaryrecord/RecordSchema.scala | 64 ---- .../memstore/OnDemandPagingShard.scala | 7 +- .../memstore/TimeSeriesPartition.scala | 6 +- .../query/PartitionChunkIndex.scala | 173 --------- .../scala/filodb.core/query/RangeVector.scala | 2 - .../scala/filodb.core/query/ResultTypes.scala | 38 +- .../scala/filodb.core/store/ColumnStore.scala | 22 +- .../binaryrecord/BinaryRecordSpec.scala | 128 ------- .../memstore/TimeSeriesPartitionSpec.scala | 16 +- .../query/PartitionChunkIndexSpec.scala | 79 ---- .../filodb.core/store/ColumnStoreSpec.scala | 79 +--- .../scala/filodb/query/exec/RowKeyRange.scala | 12 - .../query/exec/SelectChunkInfosExec.scala | 11 +- .../query/exec/SelectRawPartitionsExec.scala | 11 +- .../exec/SelectRawPartitionsExecSpec.scala | 31 +- .../scala/filodb.spark/FiloRelation.scala | 10 +- 22 files changed, 50 insertions(+), 1322 deletions(-) delete mode 100644 core/src/main/scala/filodb.core/binaryrecord/BinaryRecord.scala delete mode 100644 core/src/main/scala/filodb.core/binaryrecord/FieldType.scala delete mode 100644 core/src/main/scala/filodb.core/binaryrecord/README.md delete mode 100644 core/src/main/scala/filodb.core/binaryrecord/RecordSchema.scala delete mode 100644 core/src/main/scala/filodb.core/query/PartitionChunkIndex.scala delete mode 100644 core/src/test/scala/filodb.core/binaryrecord/BinaryRecordSpec.scala delete mode 100644 core/src/test/scala/filodb.core/query/PartitionChunkIndexSpec.scala delete mode 100644 query/src/main/scala/filodb/query/exec/RowKeyRange.scala diff --git a/coordinator/src/main/scala/filodb.coordinator/client/FiloKryoSerializers.scala b/coordinator/src/main/scala/filodb.coordinator/client/FiloKryoSerializers.scala index bda17b296f..48cf5412f6 100644 --- a/coordinator/src/main/scala/filodb.coordinator/client/FiloKryoSerializers.scala +++ b/coordinator/src/main/scala/filodb.coordinator/client/FiloKryoSerializers.scala @@ -4,38 +4,12 @@ import com.esotericsoftware.kryo.{Serializer => KryoSerializer} import com.esotericsoftware.kryo.Kryo import com.typesafe.scalalogging.StrictLogging -import filodb.core.binaryrecord.{BinaryRecord, RecordSchema} import filodb.core.binaryrecord2.{RecordSchema => RecordSchema2} import filodb.core.query.{ColumnInfo, PartitionInfo, PartitionRangeVectorKey} import filodb.memory.format._ // NOTE: This file has to be in the kryo namespace so we can use the require() method -/** - * Serializer for BinaryRecords. One complication with BinaryRecords is that they require a schema. - * We don't want to instantiate a new RecordSchema with every single BR, that would be a huge waste of memory. - * However, it seems that Kryo remembers RecordSchema references, and if the same RecordSchema is used for multiple - * BinaryRecords in an object graph (say a VectorListResult or TupleListResult) then it will be stored by some - * reference ID. Thus it saves us cost and memory allocations on restore. :) - */ -class BinaryRecordSerializer extends KryoSerializer[BinaryRecord] with StrictLogging { - override def read(kryo: Kryo, input: Input, typ: Class[BinaryRecord]): BinaryRecord = { - val schema = kryo.readObject(input, classOf[RecordSchema]) - val bytes = input.readBytes(input.readInt) - BinaryRecord(schema, bytes) - } - - override def write(kryo: Kryo, output: Output, br: BinaryRecord): Unit = { - kryo.writeObject(output, br.schema) - output.writeInt(br.numBytes) - // It would be simpler if we simply took the bytes from ArrayBinaryRecord and wrote them, but - // BinaryRecords might go offheap. - output.require(br.numBytes) - br.copyTo(output.getBuffer, UnsafeUtils.arayOffset + output.position) - output.setPosition(output.position + br.numBytes) - } -} - object BinaryRegionUtils extends StrictLogging { def writeLargeRegion(base: Any, offset: Long, output: Output): Unit = { val numBytes = UnsafeUtils.getInt(base, offset) diff --git a/coordinator/src/main/scala/filodb.coordinator/client/Serializer.scala b/coordinator/src/main/scala/filodb.coordinator/client/Serializer.scala index 63b99053f8..07c10d4ec7 100644 --- a/coordinator/src/main/scala/filodb.coordinator/client/Serializer.scala +++ b/coordinator/src/main/scala/filodb.coordinator/client/Serializer.scala @@ -6,7 +6,6 @@ import com.esotericsoftware.kryo.io._ import de.javakaffee.kryoserializers.UnmodifiableCollectionsSerializer import filodb.core._ -import filodb.core.binaryrecord.{ArrayBinaryRecord, BinaryRecord, RecordSchema} import filodb.core.binaryrecord2.{RecordSchema => RecordSchema2} import filodb.core.metadata.Column import filodb.core.query.ColumnInfo @@ -35,10 +34,7 @@ class KryoInit { val colTypeSer = new ColumnTypeSerializer Column.ColumnType.values.zipWithIndex.foreach { case (ct, i) => kryo.register(ct.getClass, colTypeSer, 100 + i) } - kryo.addDefaultSerializer(classOf[RecordSchema], classOf[RecordSchemaSerializer]) kryo.addDefaultSerializer(classOf[RecordSchema2], classOf[RecordSchema2Serializer]) - kryo.addDefaultSerializer(classOf[BinaryRecord], classOf[BinaryRecordSerializer]) - kryo.addDefaultSerializer(classOf[ZeroCopyUTF8String], classOf[ZeroCopyUTF8StringSerializer]) initOtherFiloClasses(kryo) @@ -71,9 +67,6 @@ class KryoInit { def initOtherFiloClasses(kryo: Kryo): Unit = { // Initialize other commonly used FiloDB classes kryo.register(classOf[DatasetRef]) - kryo.register(classOf[BinaryRecord]) - kryo.register(classOf[ArrayBinaryRecord]) - kryo.register(classOf[RecordSchema]) kryo.register(classOf[RecordSchema2]) kryo.register(classOf[filodb.coordinator.ShardEvent]) kryo.register(classOf[filodb.coordinator.CurrentShardSnapshot]) @@ -82,17 +75,14 @@ class KryoInit { import filodb.core.query._ kryo.register(classOf[PartitionInfo], new PartitionInfoSerializer) - kryo.register(classOf[Tuple]) kryo.register(classOf[ColumnInfo]) - kryo.register(classOf[TupleResult]) - kryo.register(classOf[TupleListResult]) kryo.register(classOf[ColumnFilter]) import filodb.core.store._ kryo.register(classOf[ChunkSetInfo]) kryo.register(WriteBufferChunkScan.getClass) kryo.register(AllChunkScan.getClass) - kryo.register(classOf[RowKeyChunkScan]) + kryo.register(classOf[TimeRangeChunkScan]) kryo.register(classOf[FilteredPartitionScan]) kryo.register(classOf[ShardSplit]) @@ -111,17 +101,6 @@ class ColumnTypeSerializer extends KryoSerializer[Column.ColumnType] { override def write(kryo: Kryo, output: Output, colType: Column.ColumnType): Unit = {} } -class RecordSchemaSerializer extends KryoSerializer[RecordSchema] { - override def read(kryo: Kryo, input: Input, typ: Class[RecordSchema]): RecordSchema = { - val colTypesObj = kryo.readClassAndObject(input) - new RecordSchema(colTypesObj.asInstanceOf[Seq[Column.ColumnType]]) - } - - override def write(kryo: Kryo, output: Output, schema: RecordSchema): Unit = { - kryo.writeClassAndObject(output, schema.columnTypes) - } -} - class RecordSchema2Serializer extends KryoSerializer[RecordSchema2] { override def read(kryo: Kryo, input: Input, typ: Class[RecordSchema2]): RecordSchema2 = { val tuple = kryo.readClassAndObject(input) diff --git a/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala b/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala index 2f8f18499c..5b5ffa1a99 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala @@ -13,10 +13,10 @@ import monix.eval.Task import filodb.coordinator.ShardMapper import filodb.coordinator.client.QueryCommands.QueryOptions import filodb.core.Types -import filodb.core.binaryrecord.BinaryRecord import filodb.core.binaryrecord2.RecordBuilder import filodb.core.metadata.Dataset import filodb.core.query.{ColumnFilter, Filter} +import filodb.core.store._ import filodb.prometheus.ast.Vectors.PromMetricLabel import filodb.query.{exec, _} import filodb.query.exec._ @@ -239,7 +239,7 @@ class QueryEngine(dataset: Dataset, val execPlans = shardsFromFilters(renamedFilters, options).map { shard => val dispatcher = dispatcherForShard(shard) SelectRawPartitionsExec(queryId, submitTime, options.sampleLimit, dispatcher, dataset.ref, shard, - renamedFilters, toRowKeyRange(lp.rangeSelector), colIDs) + renamedFilters, toChunkScanMethod(lp.rangeSelector), colIDs) } PlanResult(execPlans, needsStitch) } @@ -302,7 +302,7 @@ class QueryEngine(dataset: Dataset, val metaExec = shardsFromFilters(renamedFilters, options).map { shard => val dispatcher = dispatcherForShard(shard) SelectChunkInfosExec(queryId, submitTime, options.sampleLimit, dispatcher, dataset.ref, shard, - renamedFilters, toRowKeyRange(lp.rangeSelector), colID) + renamedFilters, toChunkScanMethod(lp.rangeSelector), colID) } PlanResult(metaExec, false) } @@ -344,14 +344,13 @@ class QueryEngine(dataset: Dataset, else { dataset.rowKeyIDs ++ ids } } - private def toRowKeyRange(rangeSelector: RangeSelector): RowKeyRange = { + private def toChunkScanMethod(rangeSelector: RangeSelector): ChunkScanMethod = { rangeSelector match { - case IntervalSelector(from, to) => RowKeyInterval(BinaryRecord(dataset, Seq(from)), - BinaryRecord(dataset, Seq(to))) - case AllChunksSelector => AllChunks - case EncodedChunksSelector => EncodedChunks - case WriteBufferSelector => WriteBuffers - case InMemoryChunksSelector => InMemoryChunks + case IntervalSelector(from, to) => TimeRangeChunkScan(from, to) + case AllChunksSelector => AllChunkScan + case EncodedChunksSelector => ??? + case WriteBufferSelector => WriteBufferChunkScan + case InMemoryChunksSelector => InMemoryChunkScan case _ => ??? } } diff --git a/core/src/main/scala/filodb.core/binaryrecord/BinaryRecord.scala b/core/src/main/scala/filodb.core/binaryrecord/BinaryRecord.scala deleted file mode 100644 index ee5211f41e..0000000000 --- a/core/src/main/scala/filodb.core/binaryrecord/BinaryRecord.scala +++ /dev/null @@ -1,349 +0,0 @@ -package filodb.core.binaryrecord - -import java.nio.ByteBuffer - -import scala.language.postfixOps - -import org.boon.primitive.ByteBuf -import scalaxy.loops._ - -import filodb.core.Types._ -import filodb.core.metadata.{Column, Dataset} -import filodb.memory.format._ -import filodb.memory.format.RowReader.TypedFieldExtractor - -// scalastyle:off equals.hash.code -/** - * BinaryRecord is a record type that supports flexible schemas and reads/writes/usage - * with no serialization at all for extreme performance and low latency with minimal GC pressure. - * It will be used within FiloDB for very quick sorting of partition/segment/rowkeys and routing - * between nodes. - * - * It also supports a map type at the end of the record only. - * - * It also implements RowReader, so values can be extracted without another instantiation. - */ -class BinaryRecord private[binaryrecord](val schema: RecordSchema, - val base: Any, - val offset: Long, - val numBytes: Int) -extends ZeroCopyBinary with SchemaRowReader { - import BinaryRecord._ - - // private final compiles to a JVM bytecode field, cheaper to access (as opposed to a method) - private final val fields = schema.fields - - final def extractors: Array[TypedFieldExtractor[_]] = schema.extractors - - final def notNull(fieldNo: Int): Boolean = { - val word = fieldNo / 32 - ((UnsafeUtils.getInt(base, offset + word * 4) >> (fieldNo % 32)) & 1) == 0 - } - - final def noneNull: Boolean = { - for { field <- 0 until fields.size optimized } { - if (!notNull(field)) return false - } - true - } - - final def isEmpty: Boolean = length == 0 - - final def getBoolean(columnNo: Int): Boolean = fields(columnNo).get[Boolean](this) - final def getInt(columnNo: Int): Int = fields(columnNo).get[Int](this) - final def getLong(columnNo: Int): Long = fields(columnNo).get[Long](this) - final def getDouble(columnNo: Int): Double = fields(columnNo).get[Double](this) - final def getFloat(columnNo: Int): Float = ??? - final def getString(columnNo: Int): String = filoUTF8String(columnNo).asNewString - - final def getAny(columnNo: Int): Any = fields(columnNo).getAny(this) - - override final def filoUTF8String(columnNo: Int): ZeroCopyUTF8String = - fields(columnNo).get[ZeroCopyUTF8String](this) - - override def toString: String = - s"b[${(0 until fields.size).map(getAny).mkString(", ")}]" - - /** - * Does a field-by-field (semantic) comparison of this BinaryRecord against another BinaryRecord. - * It is assumed that the other BinaryRecord has the exact same schema, at least for all of the fields - * present in this BinaryRecord (other.schema.numFields >= this.schema.numFields) - * It is pretty fast as the field by field comparison involves no deserialization and uses intrinsics - * in many places. - * NOTE: if all fields in this BinaryRecord compare the same as the same fields in the other, then the - * comparison returns equal (0). This semantic is needed for row key range scans to work where only the - * first few fields may be compared. - */ - override final def compare(other: ZeroCopyBinary): Int = other match { - case rec2: BinaryRecord => - for { field <- 0 until fields.size optimized } { - val cmp = fields(field).cmpRecords(this, rec2) - if (cmp != 0) return cmp - } - 0 - case zcb: ZeroCopyBinary => - super.compare(zcb) - } - - // Don't implement an equals method. This is already done in SchemaRowReader. - - // scalastyle:off null - var mapObj: UTF8Map = null - // scalastyle:on null - - /** - * Returns an array of bytes which is sortable byte-wise for its contents (which is not the goal of - * BinaryRecord). Null fields will have default values read out. - * The produced bytes cannot be deserialized from or extracted, it is strictly for comparison. - */ - def toSortableBytes(numFields: Int = 2): Array[Byte] = { - val fieldsToWrite = Math.min(fields.size, numFields) - val buf = ByteBuf.create(SortableByteBufSize) - for { fieldNo <- 0 until fieldsToWrite optimized } { - fields(fieldNo).writeSortable(this, buf) - } - buf.toBytes - } - - override def getBlobBase(columnNo: ColumnId): Any = ??? - - override def getBlobOffset(columnNo: ColumnId): ChunkID = ??? - - override def getBlobNumBytes(columnNo: ColumnId): ColumnId = ??? -} - -class ArrayBinaryRecord(schema: RecordSchema, override val bytes: Array[Byte]) extends -BinaryRecord(schema, bytes, UnsafeUtils.arayOffset, bytes.size) - -object BinaryRecord { - val DefaultMaxRecordSize = 8192 - val MaxSmallOffset = 0x7fff - val MaxSmallLen = 0xffff - val SortableByteBufSize = 100 - - val empty = { - val emptyArray = Array[Byte]() - apply(RecordSchema.empty, emptyArray) - } - - def apply(schema: RecordSchema, bytes: Array[Byte]): BinaryRecord = - new ArrayBinaryRecord(schema, bytes) - - def apply(schema: RecordSchema, buffer: ByteBuffer): BinaryRecord = - if (buffer.hasArray) { apply(schema, buffer.array) } - else if (buffer.isDirect) { - val addr: Long = buffer.asInstanceOf[sun.nio.ch.DirectBuffer].address - // scalastyle:off null - new BinaryRecord(schema, null, addr, buffer.capacity) - // scalastyle:on null - } else { throw new IllegalArgumentException("Buffer is neither array or direct") } - - def apply(schema: RecordSchema, reader: RowReader, maxBytes: Int = DefaultMaxRecordSize): BinaryRecord = { - val builder = BinaryRecordBuilder(schema, maxBytes) - for { i <- 0 until schema.fields.size optimized } { - val field = schema.fields(i) - if (reader.notNull(i)) { - field.fieldType.addFromReader(builder, field, reader) - } else { - field.fieldType.addNull(builder, field) - } - } - // TODO: Someday, instead of allocating a new buffer every time, just use a giant chunk of offheap memory - // and keep allocating from that - builder.build(copy = true) - } - - // This is designed for creating partition keys. No nulls, custom extractors for computed columns - def apply(schema: RecordSchema, reader: RowReader, extractors: Array[TypedFieldExtractor[_]]): - BinaryRecord = { - val builder = BinaryRecordBuilder(schema, DefaultMaxRecordSize) - for { i <- 0 until schema.fields.size optimized } { - val field = schema.fields(i) - field.fieldType.addWithExtractor(builder, field, reader, extractors(i)) - } - builder.build(copy = true) - } - - def apply(dataset: Dataset, items: Seq[Any]): BinaryRecord = - apply(RecordSchema(dataset.rowKeyColumns.take(items.length)), SeqRowReader(items)) - - val timeSchema = RecordSchema(Column.ColumnType.LongColumn) - val TimestampRecordSize = 16 // 8 bytes for timestamp + NA bits - - def timestamp(time: Long): BinaryRecord = apply(timeSchema, SeqRowReader(Seq(time)), TimestampRecordSize) - - implicit val ordering = new Ordering[BinaryRecord] { - def compare(a: BinaryRecord, b: BinaryRecord): Int = a.compare(b) - } - - // Create the fixed-field int for variable length data blobs. If the result is negative (bit 31 set), - // then the offset and length are both packed in; otherwise, the fixed int is just an offset to a - // 4-byte int containing length, followed by the actual blob - final def blobFixedInt(offset: Int, blobLength: Int): Int = - if (offset <= MaxSmallOffset && blobLength <= MaxSmallLen) { - 0x80000000 | (offset << 16) | blobLength - } else { - offset - } - - final def getBlobOffsetLen(binRecord: BinaryRecord, fixedData: Int): (Long, Int) = { - if (fixedData < 0) { - (binRecord.offset + ((fixedData & 0x7fff0000) >> 16), fixedData & 0xffff) - } else { - (binRecord.offset + fixedData + 4, - UnsafeUtils.getInt(binRecord.base, binRecord.offset + fixedData)) - } - } - - final def getBlobOffsetLen(binRec: BinaryRecord, field: Field): (Long, Int) = { - val fixedData = UnsafeUtils.getInt(binRec.base, binRec.offset + field.fixedDataOffset) - getBlobOffsetLen(binRec, fixedData) - } -} - -/** - * Instead of trying to somehow make BinaryRecord itself Java-Serialization friendly, and supporting - * horrible mutable fields in a class that already uses Unsafe, we keep BinaryRecord itself with an - * immutable API, and delegate Java Serialization support to this wrapper class. NOTE: for high-volume - * BinaryRecord transfers, transfer the schema separately and just transmit the bytes from the BinaryRecord. - * This class is meant for low-volume use cases and always transfers the schema with every record. - */ -@SerialVersionUID(1009L) -case class BinaryRecordWrapper(var binRec: BinaryRecord) extends java.io.Externalizable { - // scalastyle:off null - def this() = this(null) - // scalastyle:on null - def writeExternal(out: java.io.ObjectOutput): Unit = { - out.writeUTF(binRec.schema.toString) - out.writeInt(binRec.length) - out.write(binRec.bytes) - } - def readExternal(in: java.io.ObjectInput): Unit = { - val schema = RecordSchema(in.readUTF()) - val recordBytes = new Array[Byte](in.readInt()) - in.readFully(recordBytes, 0, recordBytes.size) - binRec = BinaryRecord(schema, recordBytes) - } -} - -case class OutOfBytesException(needed: Int, max: Int) extends -Exception(s"BinaryRecordBuilder: needed $needed bytes, but only had $max.") - -class BinaryRecordBuilder(schema: RecordSchema, val base: Any, val offset: Long, maxBytes: Int) { - var numBytes = schema.variableDataStartOffset - var mapObj: Option[UTF8Map] = None - - // Initialize null words - assume every field is null until it is set - for { nullWordNo <- 0 until schema.nullBitWords } { - UnsafeUtils.setInt(base, offset + nullWordNo * 4, -1) - } - - // We don't appear to need to initialize null fields or the fixedData area to 0's, because both - // ByteBuffer allocate and allocateDirect seems to initialize memory for us to 0's already - - def setNull(fieldNo: Int): Unit = { - val wordOffset = offset + (fieldNo / 32) * 4 - UnsafeUtils.setInt(base, wordOffset, - UnsafeUtils.getInt(base, wordOffset) | (1 << (fieldNo % 32))) - } - - def setNotNull(fieldNo: Int): Unit = { - val wordOffset = offset + (fieldNo / 32) * 4 - val mask = ~(1 << (fieldNo % 32)) - UnsafeUtils.setInt(base, wordOffset, - UnsafeUtils.getInt(base, wordOffset) & mask) - } - - /** - * Reserves space from the variable length area at the end. Space will always be word-aligned. - * If it succeeds, the numBytes will be moved up at the end of the call. - * @param bytesToReserve the number of bytes to reserve. Will be rounded up to a word (4-byte) boundary. - * @return Some(origOffset) the original offset of the variable length space to write to, or None if there - * isn't room to write - */ - def reserveVarBytes(bytesToReserve: Int): Option[Long] = { - val roundedLen = (bytesToReserve + 3) & -4 - if (numBytes + roundedLen <= maxBytes) { - val offsetToWrite = offset + numBytes - numBytes += roundedLen - Some(offsetToWrite) - } else { - None - } - } - - /** - * Appends a variable length blob to the end, returning the 32-bit fixed length data field that either - * contains both offset and length or just the offset, in which case first 4 bytes in var section contains - * the length. Bytes will be copied from original blob. - * @throws OutOfBytesException if bytes for the blob cannot be allocated - */ - def appendBlob(blob: ZeroCopyBinary): Int = { - // First, get the fixed int which encodes offset and len and see if we need another 4 bytes for offset - val fixedData = BinaryRecord.blobFixedInt(numBytes, blob.length) - val neededBytes = blob.length + (if (fixedData < 0) 0 else 4) - - reserveVarBytes(neededBytes).map { destOffset => - if (fixedData < 0) { - blob.copyTo(base, destOffset) - } else { - UnsafeUtils.setInt(base, destOffset, blob.length) - blob.copyTo(base, destOffset + 4) - } - }.getOrElse(throw OutOfBytesException(neededBytes, maxBytes)) - fixedData - } - - /** - * Appends a UTF8Map to the end, returning the 32-bit offset to the map area. The format of the map - * is as follows: - * offset + 0 u32 number of key-value pairs - * offset + 4 u32 * numKV An int per KV pair: upper 16-bits: length of key, lower16: len value - * after that, all of the key value UTF8 blobs laid out back to back - */ - def appendMap(map: UTF8Map): Int = { - val neededBytes = 4 + 4 * map.size + map.map { case (k, v) => k.numBytes + v.numBytes }.sum - val fixedInt = numBytes - - reserveVarBytes(neededBytes).map { destOffset => - UnsafeUtils.setInt(base, destOffset, map.size) - var blobOffset = destOffset + 4 + 4 * map.size - var sizeOffset = destOffset + 4 - map.foreach { case (k, v) => - require(k.numBytes < 65536 && v.numBytes < 65536, s"Key/value sizes over 2^16!") - val kvLenInt = (k.numBytes << 16) | v.numBytes - UnsafeUtils.setInt(base, sizeOffset, kvLenInt) - sizeOffset += 4 - k.copyTo(base, blobOffset) - blobOffset += k.numBytes - v.copyTo(base, blobOffset) - blobOffset += v.numBytes - } - }.getOrElse(throw OutOfBytesException(neededBytes, maxBytes)) - fixedInt - } - - /** - * Builds a final BinaryRecord. - * @param copy if true, copies to a new Array[Byte]. Use if initially allocated giant buffer, otherwise - * the free space in that buffer cannot be reclaimed. - */ - def build(copy: Boolean = false): BinaryRecord = { - val newRecord = if (copy) { - val newAray = new Array[Byte](numBytes) - UnsafeUtils.unsafe.copyMemory(base, offset, newAray, UnsafeUtils.arayOffset, numBytes) - new ArrayBinaryRecord(schema, newAray) - } else { - new BinaryRecord(schema, base, offset, numBytes) - } - mapObj.foreach(newRecord.mapObj = _) - newRecord - } -} - -object BinaryRecordBuilder { - def apply(schema: RecordSchema, maxBytes: Int): BinaryRecordBuilder = { - val aray = new Array[Byte](maxBytes) - new BinaryRecordBuilder(schema, aray, UnsafeUtils.arayOffset, aray.size) - } -} \ No newline at end of file diff --git a/core/src/main/scala/filodb.core/binaryrecord/FieldType.scala b/core/src/main/scala/filodb.core/binaryrecord/FieldType.scala deleted file mode 100644 index b29b2ff959..0000000000 --- a/core/src/main/scala/filodb.core/binaryrecord/FieldType.scala +++ /dev/null @@ -1,237 +0,0 @@ -package filodb.core.binaryrecord - -import java.sql.Timestamp - -import com.typesafe.scalalogging.StrictLogging -import org.boon.primitive.ByteBuf - -import filodb.core.metadata.Column.ColumnType -import filodb.core.SingleKeyTypes._ -import filodb.core.Types._ -import filodb.memory.format.{RowReader, UnsafeUtils, ZeroCopyUTF8String} -import filodb.memory.format.RowReader._ - -trait FieldType[@specialized T] { - def numFixedBytes: Int = numFixedWords * 4 - def numFixedWords: Int - - def extract(data: BinaryRecord, field: Field): T - - // NOTE: the add method modifies the state of the BinaryRecordBuilder. Don't call more than once if - // you don't want to waste memory. - def add(builder: BinaryRecordBuilder, field: Field, data: T): Unit - - def addNull(builder: BinaryRecordBuilder, field: Field): Unit = {} - - def addFromReader(builder: BinaryRecordBuilder, - field: Field, - reader: RowReader): Unit - - def addWithExtractor(builder: BinaryRecordBuilder, - field: Field, - reader: RowReader, - extractor: TypedFieldExtractor[_]): Unit - - def writeSortable(data: BinaryRecord, field: Field, buf: ByteBuf): Unit - - def compare(rec1: BinaryRecord, rec2: BinaryRecord, field: Field): Int -} - -abstract class SimpleFieldType[@specialized T: TypedFieldExtractor] extends FieldType[T] with StrictLogging { - private final val extractor = implicitly[TypedFieldExtractor[T]] - final def addFromReader(builder: BinaryRecordBuilder, - field: Field, - reader: RowReader): Unit = - try { - add(builder, field, extractor.getField(reader, field.num)) - } catch { - case e: Exception => - logger.error(s"Could not extract from $reader using field ${field.num}", e) - throw e - } - - final def addWithExtractor(builder: BinaryRecordBuilder, - field: Field, - reader: RowReader, - customExtractor: TypedFieldExtractor[_]): Unit = - add(builder, field, customExtractor.asInstanceOf[TypedFieldExtractor[T]].getField(reader, field.num)) -} - -object FieldType { - import ColumnType._ - - val columnToField = Map[ColumnType, FieldType[_]]( - IntColumn -> IntFieldType, - LongColumn -> LongFieldType, - StringColumn -> UTF8StringFieldType, - DoubleColumn -> DoubleFieldType, - TimestampColumn -> LongFieldType, // default to long handling due to BinaryRecord - MapColumn -> UTF8MapFieldType - ) -} - -object IntFieldType extends SimpleFieldType[Int] { - val numFixedWords: Int = 1 - final def extract(data: BinaryRecord, field: Field): Int = - UnsafeUtils.getInt(data.base, data.offset + field.fixedDataOffset) - - final def add(builder: BinaryRecordBuilder, field: Field, data: Int): Unit = { - UnsafeUtils.setInt(builder.base, builder.offset + field.fixedDataOffset, data) - builder.setNotNull(field.num) - } - - final def writeSortable(data: BinaryRecord, field: Field, buf: ByteBuf): Unit = - buf.writeInt(extract(data, field) ^ Int32HighBit) - - final def compare(rec1: BinaryRecord, rec2: BinaryRecord, field: Field): Int = - UnsafeUtils.getInt(rec1.base, rec1.offset + field.fixedDataOffset) - - UnsafeUtils.getInt(rec2.base, rec2.offset + field.fixedDataOffset) -} - -object LongFieldType extends SimpleFieldType[Long] { - val numFixedWords: Int = 2 - final def extract(data: BinaryRecord, field: Field): Long = - UnsafeUtils.getLong(data.base, data.offset + field.fixedDataOffset) - - final def add(builder: BinaryRecordBuilder, field: Field, data: Long): Unit = { - UnsafeUtils.setLong(builder.base, builder.offset + field.fixedDataOffset, data) - builder.setNotNull(field.num) - } - - final def writeSortable(data: BinaryRecord, field: Field, buf: ByteBuf): Unit = - buf.writeLong(extract(data, field) ^ Long64HighBit) - - final def compare(rec1: BinaryRecord, rec2: BinaryRecord, field: Field): Int = - java.lang.Long.compare(UnsafeUtils.getLong(rec1.base, rec1.offset + field.fixedDataOffset), - UnsafeUtils.getLong(rec2.base, rec2.offset + field.fixedDataOffset)) -} - -object BooleanFieldType extends SimpleFieldType[Boolean] { - val numFixedWords: Int = 1 - final def extract(data: BinaryRecord, field: Field): Boolean = - UnsafeUtils.getInt(data.base, data.offset + field.fixedDataOffset) != 0 - - final def add(builder: BinaryRecordBuilder, field: Field, data: Boolean): Unit = - IntFieldType.add(builder, field, if (data) 1 else 0) - - final def writeSortable(data: BinaryRecord, field: Field, buf: ByteBuf): Unit = - buf.writeByte(if (extract(data, field)) 1 else 0) - - final def compare(rec1: BinaryRecord, rec2: BinaryRecord, field: Field): Int = - java.lang.Boolean.compare(extract(rec1, field), extract(rec2, field)) -} - -object DoubleFieldType extends SimpleFieldType[Double] { - val numFixedWords: Int = 2 - final def extract(data: BinaryRecord, field: Field): Double = - UnsafeUtils.getDouble(data.base, data.offset + field.fixedDataOffset) - - final def add(builder: BinaryRecordBuilder, field: Field, data: Double): Unit = { - UnsafeUtils.setDouble(builder.base, builder.offset + field.fixedDataOffset, data) - builder.setNotNull(field.num) - } - - final def writeSortable(data: BinaryRecord, field: Field, buf: ByteBuf): Unit = - buf.writeDouble(extract(data, field)) - - final def compare(rec1: BinaryRecord, rec2: BinaryRecord, field: Field): Int = - java.lang.Double.compare(extract(rec1, field), extract(rec2, field)) -} - -object TimestampFieldType extends SimpleFieldType[Timestamp] { - val numFixedWords: Int = 2 - final def extract(data: BinaryRecord, field: Field): Timestamp = - new Timestamp(LongFieldType.extract(data, field)) - - final def add(builder: BinaryRecordBuilder, field: Field, data: Timestamp): Unit = - LongFieldType.add(builder, field, data.getTime) - - final def writeSortable(data: BinaryRecord, field: Field, buf: ByteBuf): Unit = - LongFieldType.writeSortable(data, field, buf) - - final def compare(rec1: BinaryRecord, rec2: BinaryRecord, field: Field): Int = - java.lang.Long.compare(UnsafeUtils.getLong(rec1.base, rec1.offset + field.fixedDataOffset), - UnsafeUtils.getLong(rec2.base, rec2.offset + field.fixedDataOffset)) -} - -object UTF8StringFieldType extends SimpleFieldType[ZeroCopyUTF8String] { - val numFixedWords: Int = 1 - final def extract(data: BinaryRecord, field: Field): ZeroCopyUTF8String = { - val (offset, len) = BinaryRecord.getBlobOffsetLen(data, field) - new ZeroCopyUTF8String(data.base, offset, len) - } - - final def add(builder: BinaryRecordBuilder, field: Field, data: ZeroCopyUTF8String): Unit = { - UnsafeUtils.setInt(builder.base, builder.offset + field.fixedDataOffset, - builder.appendBlob(data)) - builder.setNotNull(field.num) - } - - override final def addNull(builder: BinaryRecordBuilder, field: Field): Unit = { - // A string with offset 0 and length 0 -> "" - UnsafeUtils.setInt(builder.base, builder.offset + field.fixedDataOffset, 0x80000000) - } - - // Only write first 8 bytes of string, padded to 8 with 0's if needed - final def writeSortable(data: BinaryRecord, field: Field, buf: ByteBuf): Unit = { - val first8bytes = new Array[Byte](8) - val utf8str = extract(data, field) - if (utf8str.length < 8) { - utf8str.copyTo(first8bytes, UnsafeUtils.arayOffset) - UnsafeUtils.unsafe.setMemory(first8bytes, UnsafeUtils.arayOffset + utf8str.length, - 8 - utf8str.length, 0) - } else { - utf8str.copyTo(first8bytes, UnsafeUtils.arayOffset, n=8) - } - buf.add(first8bytes) - } - - // Very efficient compare method does not even need to allocate new ZeroCopyUTF8String instances - final def compare(rec1: BinaryRecord, rec2: BinaryRecord, field: Field): Int = { - val (off1, len1) = BinaryRecord.getBlobOffsetLen(rec1, field) - val (off2, len2) = BinaryRecord.getBlobOffsetLen(rec2, field) - val wordCmp = UnsafeUtils.wordCompare(rec1.base, off1, rec2.base, off2, Math.min(len1, len2)) - if (wordCmp != 0) wordCmp else len1 - len2 - } -} - -object UTF8MapFieldType extends SimpleFieldType[UTF8Map] { - val numFixedWords: Int = 1 - // scalastyle:off null - final def extract(data: BinaryRecord, field: Field): UTF8Map = - if (data.mapObj != null) { data.mapObj } - else { - // read the data - val baseOffset = UnsafeUtils.getInt(data.base, data.offset + field.fixedDataOffset) + data.offset - val numPairs = UnsafeUtils.getInt(data.base, baseOffset) - var blobOffset = baseOffset + 4 + 4 * numPairs - val map = (0 until numPairs).map { i => - val sizeInt = UnsafeUtils.getInt(data.base, baseOffset + 4 + 4 * i) - val keyLen = sizeInt >> 16 - val valueLen = sizeInt & 0x0ffff - val key = new ZeroCopyUTF8String(data.base, blobOffset, keyLen) - blobOffset += keyLen - val value = new ZeroCopyUTF8String(data.base, blobOffset, valueLen) - blobOffset += valueLen - key -> value - }.toMap - data.mapObj = map - map - } - // scalastyle:on null - - final def add(builder: BinaryRecordBuilder, field: Field, data: UTF8Map): Unit = { - UnsafeUtils.setInt(builder.base, builder.offset + field.fixedDataOffset, - builder.appendMap(data)) - builder.mapObj = Some(data) - builder.setNotNull(field.num) - } - - override final def addNull(builder: BinaryRecordBuilder, field: Field): Unit = - UnsafeUtils.setInt(builder.base, builder.offset + field.fixedDataOffset, 0) - - final def writeSortable(data: BinaryRecord, field: Field, buf: ByteBuf): Unit = ??? - - final def compare(rec1: BinaryRecord, rec2: BinaryRecord, field: Field): Int = - extract(rec1, field).size - extract(rec2, field).size -} \ No newline at end of file diff --git a/core/src/main/scala/filodb.core/binaryrecord/README.md b/core/src/main/scala/filodb.core/binaryrecord/README.md deleted file mode 100644 index f47f7e2cb4..0000000000 --- a/core/src/main/scala/filodb.core/binaryrecord/README.md +++ /dev/null @@ -1,29 +0,0 @@ -# Overview of BinaryRecord - -BinaryRecord is designed to be an extremely high performance binary storage format for individual event records for the memtable and for sorting / transferring records between nodes. Characteristics that make it unique: - -* Tight integration with FiloDB Column types and RowReader -* Supports any FiloDB table schemas / flexible schemas -* Support for off heap memory/storage and memory mapped files (eg memtables) -* Support for null values -* Using RowReader, access to fields with no deserialization. This is unique and extremely important for high performance sorting, rowkey searching for row replacement/delete functionality, etc. - - Access to binary UTF8 string representation to avoid string encoding/decoding - -Essentially, BinaryRecord is a record type that supports flexible schemas and reads/writes/usage with no serialization at all for extreme performance and low latency with minimal GC pressure. - -We try to adhere to these principles for [low latency design](https://github.com/real-logic/simple-binary-encoding/wiki/Design-Principles). - -## Format - -* First n 32-bit words: used for representing null bit field (1=null) -* All other fields are word (32-bit) aligned -* Bool, int, float all take up 32 bits -* Double, Long 64 bits -* Strings and other blobs that don't fit in 64 bits: - - 32 bit fixed field that points to other storage - - If MSB (bit 31) HIGH, then bits 16-30 contain offset, bits 0-15 length - - If MSB (bit 31) LOW, then it contains byte offset into a 32-bit length field, with the blob following the length field - - The blob itself is stored word-aligned and padded with 00's at end if needed. This helps with efficient comparisons - - Note the offset is relative to the start of the binaryRecord - - diff --git a/core/src/main/scala/filodb.core/binaryrecord/RecordSchema.scala b/core/src/main/scala/filodb.core/binaryrecord/RecordSchema.scala deleted file mode 100644 index c3f4d34934..0000000000 --- a/core/src/main/scala/filodb.core/binaryrecord/RecordSchema.scala +++ /dev/null @@ -1,64 +0,0 @@ -package filodb.core.binaryrecord - -import scala.language.existentials - -import org.boon.primitive.ByteBuf - -import filodb.core.metadata.Column -import filodb.core.metadata.Column.ColumnType - -// scalastyle:off equals.hash.code -// Case classes already have equals, but we need to customize hash code -final case class Field(num: Int, colType: ColumnType, fixedDataOffset: Int, fieldType: FieldType[_]) { - final def get[T](record: BinaryRecord): T = fieldType.asInstanceOf[FieldType[T]].extract(record, this) - final def getAny(record: BinaryRecord): Any = fieldType.extract(record, this) - - // We need a hashCode that works across JVMs, so cannot hashCode something like fieldType - override def hashCode: Int = num + 100 * colType.hashCode + 10000 * fixedDataOffset - - final def writeSortable(record: BinaryRecord, buf: ByteBuf): Unit = - fieldType.writeSortable(record, this, buf) - final def cmpRecords(rec1: BinaryRecord, rec2: BinaryRecord): Int = fieldType.compare(rec1, rec2, this) -} - -/** - * Stores offsets and other information for a BinaryRecord for a given schema (seq of column types) - */ -final class RecordSchema(val columnTypes: Seq[ColumnType]) { - // Computes offsets for every field, where they would go etc - val numFields = columnTypes.length - - // Number of 32-bit words at beginning for null check - val nullBitWords = (numFields + 31) / 32 - val fixedDataStartOffset = nullBitWords * 4 - - // val fields - fixed data field section - var curOffset = fixedDataStartOffset - val fields = columnTypes.zipWithIndex.map { case (colType, no) => - if (colType == ColumnType.MapColumn) require(no == columnTypes.length - 1) - val field = Field(no, colType, curOffset, FieldType.columnToField(colType)) - curOffset += field.fieldType.numFixedBytes - field - }.toArray - - val extractors = columnTypes.map(_.keyType.extractor).toArray - - val variableDataStartOffset = curOffset - - override lazy val hashCode = fields.foldLeft(1)(_ * _.hashCode) - - override def toString: String = columnTypes.map(_.toString).mkString(":") -} - -object RecordSchema { - val empty = new RecordSchema(Nil) - - def apply(columns: Seq[Column]): RecordSchema = new RecordSchema(columns.map(_.columnType)) - - def apply(colType: ColumnType): RecordSchema = new RecordSchema(Seq(colType)) - - def apply(schemaStr: String): RecordSchema = { - val types = schemaStr.split(':').toSeq.map(s => ColumnType.withName(s)) - new RecordSchema(types) - } -} \ No newline at end of file diff --git a/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala b/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala index ca92b0c700..ea9b96e56a 100644 --- a/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala +++ b/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala @@ -11,7 +11,6 @@ import monix.execution.Scheduler import monix.reactive.{Observable, OverflowStrategy} import filodb.core.Types -import filodb.core.binaryrecord.BinaryRecord import filodb.core.downsample.{DownsampleConfig, DownsamplePublisher} import filodb.core.metadata.Dataset import filodb.core.store._ @@ -178,7 +177,7 @@ TimeSeriesShard(dataset, storeConfig, shardNum, rawStore, metastore, evictionPol minTime = Math.min(minTime, m.startTime) maxTime = Math.max(maxTime, m.endTime) } - RowKeyChunkScan(minTime, maxTime) + TimeRangeChunkScan(minTime, maxTime) } private def chunksToFetch(partition: ReadablePartition, @@ -192,11 +191,11 @@ TimeSeriesShard(dataset, storeConfig, shardNum, rawStore, metastore, evictionPol // Assume initial startKey of first chunk is the earliest - typically true unless we load in historical data // Compare desired time range with start key and see if in memory data covers desired range // Also assume we have in memory all data since first key. Just return the missing range of keys. - case r: RowKeyChunkScan => + case r: TimeRangeChunkScan => if (partition.numChunks > 0) { val memStartTime = partition.earliestTime val endQuery = memStartTime - 1 // do not include earliestTime, otherwise will pull in first chunk - if (r.startTime < memStartTime) { Some(RowKeyChunkScan(r.startkey, BinaryRecord.timestamp(endQuery))) } + if (r.startTime < memStartTime) { Some(TimeRangeChunkScan(r.startTime, endQuery)) } else { None } } else { Some(r) // if no chunks ingested yet, read everything from disk diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala index 41e4b0a0bf..b112d8f4e9 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala @@ -263,9 +263,9 @@ extends ChunkMap(memFactory, initMapSize) with ReadablePartition { def infos(method: ChunkScanMethod): ChunkInfoIterator = method match { case AllChunkScan => allInfos case InMemoryChunkScan => allInfos - case r: RowKeyChunkScan => allInfos.filter { ic => - ic.intersection(r.startTime, r.endTime).isDefined - } + case r: TimeRangeChunkScan => allInfos.filter { ic => + ic.intersection(r.startTime, r.endTime).isDefined + } case WriteBufferChunkScan => if (currentInfo == nullInfo) ChunkInfoIterator.empty else { // Return a single element iterator which holds a shared lock. diff --git a/core/src/main/scala/filodb.core/query/PartitionChunkIndex.scala b/core/src/main/scala/filodb.core/query/PartitionChunkIndex.scala deleted file mode 100644 index 0948e1351a..0000000000 --- a/core/src/main/scala/filodb.core/query/PartitionChunkIndex.scala +++ /dev/null @@ -1,173 +0,0 @@ -package filodb.core.query - -import org.jctools.maps.NonBlockingHashMapLong - -import filodb.core.binaryrecord.BinaryRecord -import filodb.core.metadata.Dataset -import filodb.core.store._ -import filodb.core.Types.ChunkID -import filodb.core.store.ChunkSetInfo._ - -/** - * An index providing facilities to search through the chunks of a partition in different ways - * NOTE: it focuses purely on the ChunkSetInfo, not on filters and other things - */ -trait PartitionChunkIndex { - def dataset: Dataset - def partKeyBase: Any - def partKeyOffset: Long - - def numChunks: Int - - /** - * Obtains a sequence of chunks where at least some of the rowkeys inside are within (startKey, endKey) - * The ordering of the chunks returned depends on implementation. - */ - def rowKeyRange(startKey: BinaryRecord, endKey: BinaryRecord): InfosSkipsIt - - /** - * Returns all chunks in some order which is implementation-specific - */ - def allChunks: InfosSkipsIt - - /** - * Returns the latest N chunk infos - */ - def latestN(n: Int): InfosSkipsIt - - /** - * Returns the ChunkSetInfo and skips for a single chunk with startKey and id. - * Depending on implementation, either only the id (which should be unique) or both may be used. - * @return an iterator with a single item (ChunkSetInfo, skipArray) or empty iterator if not found - */ - def singleChunk(startKey: BinaryRecord, id: ChunkID): InfosSkipsIt - - def findByMethod(method: ChunkScanMethod): InfosSkipsIt = - method match { - case AllChunkScan => allChunks - case RowKeyChunkScan(k1, k2) => rowKeyRange(k1.binRec, k2.binRec) - case WriteBufferChunkScan => latestN(1) - case InMemoryChunkScan => allChunks - } -} - -/** - * A PartitionChunkIndex that can incrementally update itself with new ChunkSetInfos. - * Designed to be added to incrementally and kept in memory - this way it can lower the overhead of - * repeated queries to the same partition. - */ -trait MutablePartitionChunkIndex extends PartitionChunkIndex { - /** - * Adds a ChunkSetInfo and the skips to the index - */ - def add(info: ChunkSetInfo, skips: Seq[ChunkRowSkipIndex]): Unit - - /** - * Removes a ChunkSetInfo. May need to regenerate indices. - */ - def remove(id: ChunkID): Unit -} - -object PartitionChunkIndex { - import ChunkSetInfo.SkipMap - def newSkip(key: java.lang.Long): SkipMap = new SkipMap() -} - -/** - * An implementation of PartitionChunkIndex which orders things by time - thus rowKeyRange and allChunks - * will return chunk infos in increasing time order - - * TODO: improve this implementation with binary bit indices such as JavaEWAH. - */ -class TimeBasedPartitionChunkIndex(val partKeyBase: Any, val partKeyOffset: Long, val dataset: Dataset) -extends MutablePartitionChunkIndex { - import collection.JavaConverters._ - - import filodb.core._ - import PartitionChunkIndex._ - import ChunkSetInfo._ - - val skipRows = new NonBlockingHashMapLong[SkipMap](64) - val infos = new java.util.TreeMap[(Long, ChunkID), ChunkSetInfo]( - Ordering[(Long, ChunkID)]) - - def add(info: ChunkSetInfo, skips: Seq[ChunkRowSkipIndex]): Unit = { - infos.put((info.startTime, info.id), info) - for { skip <- skips } { - skipRows.put(skip.id, skipRows.getOrElseUpdate(skip.id, newSkip).or(skip.overrides)) - } - } - - def numChunks: Int = infos.size - - def rowKeyRange(startKey: BinaryRecord, endKey: BinaryRecord): InfosSkipsIt = { - // Exclude chunks which start after the end search range - infos.headMap((endKey.getLong(0), Long.MaxValue)).values.iterator.asScala.collect { - case c: ChunkSetInfo if c.intersection(startKey.getLong(0), endKey.getLong(0)).isDefined => - (c, skipRows.get(c.id)) - } - } - - def allChunks: InfosSkipsIt = - infos.values.iterator.asScala.map { info => - (info, skipRows.get(info.id)) - } - - def singleChunk(startKey: BinaryRecord, id: ChunkID): InfosSkipsIt = - infos.subMap((startKey.getLong(0), id), true, (startKey.getLong(0), id), true).values.iterator.asScala.map { info => - (info, skipRows.get(info.id)) - } - - // NOTE: latestN does not make sense for the TimeBasedPartitionChunkIndex. - def latestN(n: Int): InfosSkipsIt = ??? - - def remove(id: ChunkID): Unit = { - infos.keySet.iterator.asScala - .find(_._2 == id) - .foreach(infos.remove) - skipRows.remove(id) - } -} - -/** - * A PartitionChunkIndex which is ordered by increasing ChunkID - */ -class ChunkIDPartitionChunkIndex(val partKeyBase: Any, val partKeyOffset: Long, val dataset: Dataset) -extends MutablePartitionChunkIndex { - import collection.JavaConverters._ - - val infosSkips = new java.util.TreeMap[ChunkID, (ChunkSetInfo, SkipMap)] - - def add(info: ChunkSetInfo, skips: Seq[ChunkRowSkipIndex]): Unit = { - infosSkips.put(info.id, (info, emptySkips)) - for { skip <- skips } { - Option(infosSkips.get(skip.id)) match { - case Some((origInfo, origSkips)) => - infosSkips.put(skip.id, (origInfo, origSkips.or(skip.overrides))) - case None => - } - } - } - - def numChunks: Int = infosSkips.size - - def rowKeyRange(startKey: BinaryRecord, endKey: BinaryRecord): InfosSkipsIt = { - // Linear search through all infos to find intersections - // TODO: use an interval tree to speed this up? - infosSkips.values.iterator.asScala.filter { - case (info, skips) => info.intersection(startKey.getLong(0), endKey.getLong(0)).isDefined - } - } - - def allChunks: InfosSkipsIt = infosSkips.values.iterator.asScala - - def singleChunk(startKey: BinaryRecord, id: ChunkID): InfosSkipsIt = - infosSkips.subMap(id, true, id, true).values.iterator.asScala - - def latestN(n: Int): InfosSkipsIt = - infosSkips.descendingMap.values.iterator.asScala.take(n).toBuffer.reverse.toIterator - - def remove(id: ChunkID): Unit = { - infosSkips.remove(id) - } -} \ No newline at end of file diff --git a/core/src/main/scala/filodb.core/query/RangeVector.scala b/core/src/main/scala/filodb.core/query/RangeVector.scala index 6c75046b4b..0024a1300b 100644 --- a/core/src/main/scala/filodb.core/query/RangeVector.scala +++ b/core/src/main/scala/filodb.core/query/RangeVector.scala @@ -4,7 +4,6 @@ import com.typesafe.scalalogging.StrictLogging import kamon.Kamon import org.joda.time.DateTime -import filodb.core.binaryrecord.BinaryRecord import filodb.core.binaryrecord2.{MapItemConsumer, RecordBuilder, RecordContainer, RecordSchema} import filodb.core.metadata.Column import filodb.core.metadata.Column.ColumnType._ @@ -143,7 +142,6 @@ final class SerializableRangeVector(val key: RangeVectorKey, val curTime = System.currentTimeMillis key.toString + "\n\t" + rows.map { - case br: BinaryRecord if br.isEmpty => "\t" case reader => val firstCol = if (formatTime && schema.isTimeSeries) { val timeStamp = reader.getLong(0) diff --git a/core/src/main/scala/filodb.core/query/ResultTypes.scala b/core/src/main/scala/filodb.core/query/ResultTypes.scala index 678b49219a..4663c8d8cd 100644 --- a/core/src/main/scala/filodb.core/query/ResultTypes.scala +++ b/core/src/main/scala/filodb.core/query/ResultTypes.scala @@ -4,10 +4,8 @@ import scala.reflect.runtime.universe._ import com.typesafe.scalalogging.StrictLogging import monix.eval.Task -import monix.reactive.Observable import org.joda.time.DateTime -import filodb.core.binaryrecord.BinaryRecord import filodb.core.binaryrecord2.RecordSchema import filodb.core.metadata.Column import filodb.core.store.ChunkScanMethod @@ -21,20 +19,13 @@ final case class PartitionInfo(schema: RecordSchema, base: Array[Byte], offset: override def toString: String = s"/shard:$shardNo/${schema.stringify(base, offset)}" } -/** - * A single element of data described by a fixed schema of types, corresponding to a single key or timestamp. - * Could be either raw or intermediate data. - * Ex: a (timestamp, value) tuple. Or a (Double) scalar. (count, total) for an average. - */ -final case class Tuple(info: Option[PartitionInfo], data: BinaryRecord) - /** * Describes column/field name and type */ final case class ColumnInfo(name: String, colType: Column.ColumnType) /** - * Describes the full schema of Vectors and Tuples, including how many initial columns are for row keys. + * Describes the full schema of result types, including how many initial columns are for row keys. * The first ColumnInfo in the schema describes the first vector in Vectors and first field in Tuples, etc. * @param brSchemas if any of the columns is a binary record, thsi */ @@ -72,7 +63,6 @@ sealed trait Result extends java.io.Serializable { toRowReaders.map { case (partInfoOpt, rowReaders) => partInfoOpt.map(_.toString).getOrElse("") + "\n\t" + rowReaders.take(partitionRowLimit).map { - case br: BinaryRecord if br.isEmpty => "\t" case reader => val firstCol = if (formatTime && schema.isTimeSeries) { val timeStamp = reader.getLong(0) @@ -86,16 +76,6 @@ sealed trait Result extends java.io.Serializable { } } -final case class TupleListResult(schema: ResultSchema, tuples: Seq[Tuple]) extends Result { - def toRowReaders: Iterator[(Option[PartitionInfo], Seq[RowReader])] = - tuples.toIterator.map { case Tuple(info, data) => (info, Seq(data)) } -} - -final case class TupleResult(schema: ResultSchema, tuple: Tuple) extends Result { - def toRowReaders: Iterator[(Option[PartitionInfo], Seq[RowReader])] = - Iterator.single((tuple.info, Seq(tuple.data))) -} - /** * Converts various types to result types * TODO: consider collapsing into Result @@ -130,20 +110,4 @@ object ResultMaker extends StrictLogging { limit: Int = 1000): Task[Result] = ??? def fromResult(res: Result): Unit = {} } - - implicit object TupleObservableMaker extends ResultMaker[Observable[Tuple]] { - def toResult(tuples: Observable[Tuple], - schema: ResultSchema, - chunkMethod: ChunkScanMethod, - limit: Int = 1000): Task[Result] = { - tuples.take(limit).toListL.map { tupleList => - TupleListResult(schema, tupleList) - } - } - - def fromResult(res: Result): Observable[Tuple] = res match { - case TupleListResult(_, tuples) => Observable.fromIterable(tuples) - case other: Result => throw new RuntimeException(s"Unexpected result $other... possible type/plan error") - } - } } diff --git a/core/src/main/scala/filodb.core/store/ColumnStore.scala b/core/src/main/scala/filodb.core/store/ColumnStore.scala index 7948ae2ae1..51be420e68 100644 --- a/core/src/main/scala/filodb.core/store/ColumnStore.scala +++ b/core/src/main/scala/filodb.core/store/ColumnStore.scala @@ -3,7 +3,6 @@ package filodb.core.store import com.typesafe.scalalogging.StrictLogging import monix.reactive.Observable -import filodb.core.binaryrecord.{BinaryRecord, BinaryRecordWrapper} import filodb.core.metadata.Dataset import filodb.core.query._ import filodb.memory.BinaryRegionLarge @@ -41,30 +40,11 @@ trait AllTimeScanMethod { } case object AllChunkScan extends AllTimeScanMethod with ChunkScanMethod -// NOTE: BinaryRecordWrapper must be used as this case class might be Java Serialized -final case class RowKeyChunkScan(firstBinKey: BinaryRecordWrapper, - lastBinKey: BinaryRecordWrapper) extends ChunkScanMethod { - def startkey: BinaryRecord = firstBinKey.binRec - def endkey: BinaryRecord = lastBinKey.binRec - def startTime: Long = startkey.getLong(0) - def endTime: Long = endkey.getLong(0) -} +final case class TimeRangeChunkScan(startTime: Long, endTime: Long) extends ChunkScanMethod case object WriteBufferChunkScan extends AllTimeScanMethod with ChunkScanMethod // Only read chunks which are in memory case object InMemoryChunkScan extends AllTimeScanMethod with ChunkScanMethod -object RowKeyChunkScan { - def apply(startKey: BinaryRecord, endKey: BinaryRecord): RowKeyChunkScan = - RowKeyChunkScan(BinaryRecordWrapper(startKey), BinaryRecordWrapper(endKey)) - - def apply(dataset: Dataset, startKey: Seq[Any], endKey: Seq[Any]): RowKeyChunkScan = - RowKeyChunkScan(BinaryRecord(dataset, startKey), BinaryRecord(dataset, endKey)) - - def apply(startTime: Long, endTime: Long): RowKeyChunkScan = - RowKeyChunkScan(BinaryRecord.timestamp(startTime), BinaryRecord.timestamp(endTime)) -} - - trait ScanSplit { // Should return a set of hostnames or IP addresses describing the preferred hosts for that scan split def hostnames: Set[String] diff --git a/core/src/test/scala/filodb.core/binaryrecord/BinaryRecordSpec.scala b/core/src/test/scala/filodb.core/binaryrecord/BinaryRecordSpec.scala deleted file mode 100644 index 1eb3754ec6..0000000000 --- a/core/src/test/scala/filodb.core/binaryrecord/BinaryRecordSpec.scala +++ /dev/null @@ -1,128 +0,0 @@ -package filodb.core.binaryrecord - -import java.io.{ByteArrayInputStream, ByteArrayOutputStream, ObjectInputStream, ObjectOutputStream} -import java.sql.Timestamp - -import org.scalatest.{FunSpec, Matchers} - -import filodb.core.Types._ -import filodb.memory.format.{SeqRowReader, TupleRowReader, ZeroCopyUTF8String} - -class BinaryRecordSpec extends FunSpec with Matchers { - import filodb.core.metadata.Column.ColumnType._ - import ZeroCopyUTF8String._ - - val schema1_i = new RecordSchema(Seq(IntColumn)) - val schema1_s = new RecordSchema(Seq(StringColumn)) - - val schema2_sl = new RecordSchema(Seq(StringColumn, LongColumn)) - val schema2_is = new RecordSchema(Seq(IntColumn, StringColumn)) - - val schema3_bdt = new RecordSchema(Seq(IntColumn, DoubleColumn, TimestampColumn)) - - val schema4 = new RecordSchema(Seq(StringColumn, MapColumn)) - - val reader1 = TupleRowReader((Some("data"), Some(-15L))) - val reader2 = TupleRowReader((Some(1234), Some("one-two-three"))) - val reader3 = TupleRowReader((Some(1), Some(5.7), Some(new Timestamp(1000000L)))) - - it("should create and extract individual fields and match when all fields present") { - BinaryRecord(schema1_i, reader2).getInt(0) should equal (1234) - BinaryRecord(schema1_s, reader1).getString(0) should equal ("data") - BinaryRecord(schema1_s, reader1).filoUTF8String(0) should equal (ZeroCopyUTF8String("data")) - - val binRec3 = BinaryRecord(schema2_sl, reader1) - binRec3.notNull(0) should equal (true) - binRec3.notNull(1) should equal (true) - binRec3.getString(0) should equal ("data") - binRec3.getLong(1) should equal (-15L) - binRec3.noneNull should equal (true) - intercept[ClassCastException] { - binRec3.getString(1) - } - - val binRec4 = BinaryRecord(schema2_is, reader2) - binRec4.getString(1) should equal ("one-two-three") - binRec4.getInt(0) should equal (1234) - binRec4.noneNull should equal (true) - - val binRec5 = BinaryRecord(schema3_bdt, reader3) - binRec5.getDouble(1) should equal (5.7) - binRec5.getLong(2) should equal (1000000L) - binRec5.noneNull should equal (true) - } - - it("should create and extract fields and check notNull correctly") { - val binRec1 = BinaryRecord(schema2_sl, TupleRowReader((None, Some(10L)))) - binRec1.notNull(0) should equal (false) - binRec1.notNull(1) should equal (true) - binRec1.noneNull should equal (false) - binRec1.getLong(1) should equal (10L) - } - - it("should get default values back for null fields") { - val binRec1 = BinaryRecord(schema2_sl, TupleRowReader((None, None))) - binRec1.notNull(0) should equal (false) - binRec1.notNull(1) should equal (false) - binRec1.noneNull should equal (false) - binRec1.getLong(1) should equal (0L) - binRec1.getString(0) should equal ("") - } - - it("should get bytes out and get back same BinaryRecord") { - val bytes = BinaryRecord(schema3_bdt, reader3).bytes - val binRec = BinaryRecord(schema3_bdt, bytes) - binRec.getDouble(1) should equal (5.7) - binRec.getLong(2) should equal (1000000L) - } - - it("should generate same hashcode for different instances of the same RecordSchema") { - val schema3_is = new RecordSchema(Seq(IntColumn, StringColumn)) - schema2_is.hashCode should equal (schema3_is.hashCode) - } - - it("should produce shorter BinaryRecords if smaller number of items fed") { - import filodb.core.GdeltTestData._ - - val shortBR1 = BinaryRecord(dataset2, Seq(30L)) - shortBR1.schema.numFields should equal (1) - } - - it("should semantically compare BinaryRecords field by field") { - import filodb.core.GdeltTestData._ - - // Should compare semantically rather than by binary. Int occurs first byte-wise, but 2nd semantically - val rec1 = BinaryRecord(dataset2, Seq(55L, "FRA")) - rec1 should be < (BinaryRecord(dataset2, Seq(60L, "CHL"))) - rec1 should equal (BinaryRecord(dataset2, Seq(55L, "FRA"))) - - // Should be able to compare shorter record with longer one - BinaryRecord(dataset2, Seq(55L)) should equal (rec1) - BinaryRecord(dataset2, Seq(56L)) should be > (rec1) - } - - it("should semantically compare BinaryRecord Int and Long fields correctly") (pending) - - it("should read, write, and compare Map fields") { - val map1 = Map("application".utf8 -> "FiloDB".utf8, "env".utf8 -> "staging".utf8) - val binRec1 = BinaryRecord(schema4, SeqRowReader(Seq("400s", map1))) - binRec1.as[UTF8Map](1) should equal (map1) - - val binRec2 = BinaryRecord(schema4, binRec1.bytes) - binRec2 should equal (binRec1) - binRec2.as[UTF8Map](1) should equal (map1) - } - - it("should serialize and deserialize RecordSchema and BinaryRecordWrapper") { - RecordSchema(schema3_bdt.toString).fields should equal (schema3_bdt.fields) - - val binRec1 = BinaryRecord(schema2_is, reader2) - val baos = new ByteArrayOutputStream - val oos = new ObjectOutputStream(baos) - oos.writeObject(BinaryRecordWrapper(binRec1)) - - val ois = new ObjectInputStream(new ByteArrayInputStream(baos.toByteArray)) - val readWrapper = ois.readObject().asInstanceOf[BinaryRecordWrapper] - readWrapper.binRec should equal (binRec1) - } -} \ No newline at end of file diff --git a/core/src/test/scala/filodb.core/memstore/TimeSeriesPartitionSpec.scala b/core/src/test/scala/filodb.core/memstore/TimeSeriesPartitionSpec.scala index b8834d88cf..0caed40822 100644 --- a/core/src/test/scala/filodb.core/memstore/TimeSeriesPartitionSpec.scala +++ b/core/src/test/scala/filodb.core/memstore/TimeSeriesPartitionSpec.scala @@ -156,7 +156,7 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { // Flushed chunk: initTS -> initTS + 9000 (1000 ms per tick) // Read from flushed chunk only - val readIt = part.timeRangeRows(RowKeyChunkScan(initTS, initTS + 9000), Array(0, 1)).map(_.getDouble(1)) + val readIt = part.timeRangeRows(TimeRangeChunkScan(initTS, initTS + 9000), Array(0, 1)).map(_.getDouble(1)) readIt.toBuffer shouldEqual minData.take(10) val infos1 = part.infos(initTS, initTS + 9000) @@ -165,27 +165,29 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { info1.numRows shouldEqual 10 info1.startTime shouldEqual initTS - val readIt2 = part.timeRangeRows(RowKeyChunkScan(initTS + 1000, initTS + 7000), Array(0, 1)).map(_.getDouble(1)) + val readIt2 = part.timeRangeRows(TimeRangeChunkScan(initTS + 1000, initTS + 7000), Array(0, 1)).map(_.getDouble(1)) readIt2.toBuffer shouldEqual minData.drop(1).take(7) // Read from appending chunk only: initTS + 10000 - val readIt3 = part.timeRangeRows(RowKeyChunkScan(appendingTS, appendingTS + 3000), Array(0, 1)).map(_.getDouble(1)) + val readIt3 = part.timeRangeRows(TimeRangeChunkScan(appendingTS, appendingTS + 3000), Array(0, 1)) + .map(_.getDouble(1)) readIt3.toBuffer shouldEqual minData.drop(10) // Try to read from before flushed to part of flushed chunk - val readIt4 = part.timeRangeRows(RowKeyChunkScan(initTS - 7000, initTS + 3000), Array(0, 1)).map(_.getDouble(1)) + val readIt4 = part.timeRangeRows(TimeRangeChunkScan(initTS - 7000, initTS + 3000), Array(0, 1)).map(_.getDouble(1)) readIt4.toBuffer shouldEqual minData.take(4) // both flushed and appending chunk - val readIt5 = part.timeRangeRows(RowKeyChunkScan(initTS + 7000, initTS + 14000), Array(0, 1)).map(_.getDouble(1)) + val readIt5 = part.timeRangeRows(TimeRangeChunkScan(initTS + 7000, initTS + 14000), Array(0, 1)).map(_.getDouble(1)) readIt5.toBuffer shouldEqual minData.drop(7) // No data: past appending chunk - val readIt6 = part.timeRangeRows(RowKeyChunkScan(initTS + 20000, initTS + 24000), Array(0, 1)).map(_.getDouble(1)) + val readIt6 = part.timeRangeRows(TimeRangeChunkScan(initTS + 20000, initTS + 24000), Array(0, 1)) + .map(_.getDouble(1)) readIt6.toBuffer shouldEqual Nil // No data: before initTS - val readIt7 = part.timeRangeRows(RowKeyChunkScan(initTS - 9000, initTS - 900), Array(0, 1)).map(_.getDouble(1)) + val readIt7 = part.timeRangeRows(TimeRangeChunkScan(initTS - 9000, initTS - 900), Array(0, 1)).map(_.getDouble(1)) readIt7.toBuffer shouldEqual Nil } diff --git a/core/src/test/scala/filodb.core/query/PartitionChunkIndexSpec.scala b/core/src/test/scala/filodb.core/query/PartitionChunkIndexSpec.scala deleted file mode 100644 index f0677b169b..0000000000 --- a/core/src/test/scala/filodb.core/query/PartitionChunkIndexSpec.scala +++ /dev/null @@ -1,79 +0,0 @@ -package filodb.core.query - -import filodb.core.binaryrecord.BinaryRecord -import filodb.core.store.{ChunkRowSkipIndex, ChunkSetInfo} -import filodb.memory.format.vectors.NativeVectorTest - -class PartitionChunkIndexSpec extends NativeVectorTest { - import collection.JavaConverters._ - - import filodb.core.NamesTestData._ - - val info1 = ChunkSetInfo(memFactory, dataset, 100L, 3, firstKey, lastKey) - val info2 = ChunkSetInfo(memFactory, dataset, 99L, 3, keyForName(1), lastKey) - - val firstRecord = BinaryRecord.timestamp(firstKey) - val lastRecord = BinaryRecord.timestamp(lastKey) - - describe("TimeBasedPartitionChunkIndex") { - it("should add out of order chunks and return in rowkey order") { - val newIndex = new TimeBasedPartitionChunkIndex(null, defaultPartKey, dataset) - - // Initial index should be empty - newIndex.numChunks should equal (0) - - // Add chunk info with no skips. Note that info1 has lower firstKey but higher id, on purpose - newIndex.add(info1, Nil) - newIndex.numChunks should equal (1) - - newIndex.add(info2, Nil) - newIndex.numChunks should equal (2) - - newIndex.allChunks.toList.map(_._1) should equal (List(info1, info2)) - - newIndex.rowKeyRange(firstRecord, firstRecord).toList.map(_._1) should equal (List(info1)) - } - - it("should return no chunks if rowKeyRange startKey is greater than endKey") { - val newIndex = new TimeBasedPartitionChunkIndex(null, defaultPartKey, dataset) - newIndex.add(info1, Nil) - newIndex.add(info2, Nil) - - newIndex.rowKeyRange(lastRecord, firstRecord).toList.map(_._1) should equal (Nil) - } - } - - describe("ChunkIDPartitionChunkIndex") { - it("should add out of order chunks and return in chunkID order") { - val newIndex = new ChunkIDPartitionChunkIndex(null, defaultPartKey, dataset) - - // Initial index should be empty - newIndex.numChunks should equal (0) - - // Add chunk info with no skips. Note that info1 has lower firstKey but higher id, on purpose - newIndex.add(info1, Nil) - newIndex.numChunks should equal (1) - - newIndex.add(info2, Nil) - newIndex.numChunks should equal (2) - - newIndex.allChunks.toList.map(_._1) should equal (List(info2, info1)) - - newIndex.rowKeyRange(firstRecord, firstRecord).toList.map(_._1) should equal (List(info1)) - } - - it("should handle skips") { - val newIndex = new ChunkIDPartitionChunkIndex(null, defaultPartKey, dataset) - val origInfo = ChunkSetInfo(memFactory, dataset, 9, info1.numRows, info1.startTime, info1.endTime) - val info2 = ChunkSetInfo(memFactory, dataset, 14, info1.numRows, info1.startTime, info1.endTime) - newIndex.add(origInfo, Nil) - newIndex.add(info1, Seq(ChunkRowSkipIndex(9, Array(2, 5)))) - newIndex.add(info2, Seq(ChunkRowSkipIndex(9, Array(3, 5, 8)))) - - val infosAndSkips = newIndex.allChunks.toList - infosAndSkips should have length (3) - infosAndSkips(0)._1.id should equal (9) - infosAndSkips(0)._2.toList.asScala should equal (Seq(2, 3, 5, 8)) - } - } -} \ No newline at end of file diff --git a/core/src/test/scala/filodb.core/store/ColumnStoreSpec.scala b/core/src/test/scala/filodb.core/store/ColumnStoreSpec.scala index 8c8fe2cb50..69dc2d4e8d 100644 --- a/core/src/test/scala/filodb.core/store/ColumnStoreSpec.scala +++ b/core/src/test/scala/filodb.core/store/ColumnStoreSpec.scala @@ -7,10 +7,8 @@ import org.scalatest.concurrent.ScalaFutures import org.scalatest.time.{Millis, Seconds, Span} import filodb.core._ -import filodb.core.binaryrecord.BinaryRecord import filodb.core.memstore.{FixedMaxPartitionsEvictionPolicy, FlushStream, TimeSeriesMemStore} import filodb.core.query.{ColumnFilter, Filter} -import filodb.memory.format.ZeroCopyUTF8String._ // TODO: figure out what to do with this.. most of the tests are really irrelevant trait ColumnStoreSpec extends FlatSpec with Matchers @@ -80,8 +78,7 @@ with BeforeAndAfter with BeforeAndAfterAll with ScalaFutures { colStore.write(dataset, chunkSetStream()).futureValue should equal (Success) // partition exists but no chunks in range > 1000, this should find nothing - val noChunkScan = RowKeyChunkScan(BinaryRecord(dataset, Seq(1000L)), - BinaryRecord(dataset, Seq(2000L))) + val noChunkScan = TimeRangeChunkScan(1000L, 2000L) val parts = colStore.readRawPartitions(dataset, Seq(0, 1, 2), partScan, noChunkScan).toListL.runAsync.futureValue parts should have length (1) parts.head.chunkSets should have length (0) @@ -154,80 +151,6 @@ with BeforeAndAfter with BeforeAndAfterAll with ScalaFutures { rowIt.map(_.getInt(0)).sum should equal (22) } - // TODO: redo this test for several reasons. First, having to sort by a non-time row key is not realistic anymore. - // Second switch to new BR2 format. - ignore should "range scan by row keys and filter rows with single partition key" in { - import GdeltTestData._ - // Requirement: Must ingest 50 rows per chunk - // val partsRows = getRowsByPartKey(dataset2) - // partsRows.foreach { case (part, rows) => - // rows.grouped(50).foreach { rowset => - // val sorted = rowset.sortBy(r => (r.getString(3), r.getInt(0))) - // val stream = toChunkSetStream(dataset2, part, sorted, 50) - // colStore.write(dataset2, stream).futureValue should equal (Success) - // } - // } - - val paramSet = colStore.getScanSplits(dataset.ref, 1) - paramSet should have length (1) - - // First 50 rows have no blank Actor2Code's. We range scan on ("", 50) to ("", 99), which - // does exist in second 50 rows in both partitions 197901 and 197902. - // TODO: switch this to EventID, Actor2Code scan. - val method1 = FilteredPartitionScan(paramSet.head) - val rowRange = RowKeyChunkScan(BinaryRecord(dataset2, Seq("", 50)), - BinaryRecord(dataset2, Seq("", 99))) - val rows = memStore.scanRows(dataset2, dataset2.colIDs("GLOBALEVENTID", "MonthYear").get, - method1, rowRange) - .map(r => (r.getInt(0), r.getInt(1))).toList - rows.length should equal (49) - rows.map(_._2).toSet should equal (Set(197901, 197902)) - rows.map(_._1).min should equal (50) - - // Ask for only partition 197902 and row keys that don't exist, there should be no rows - val filter2 = ColumnFilter("MonthYear", Filter.Equals(197902)) - val method2 = FilteredPartitionScan(paramSet.head, Seq(filter2)) - val rowRange2 = RowKeyChunkScan(BinaryRecord(dataset2, Seq("", 0)), - BinaryRecord(dataset2, Seq("", 2))) - val rowIter2 = memStore.scanRows(dataset2, Seq(0), method2, rowRange2) - rowIter2.toSeq.length should equal (0) - - // Should be able to filter chunks by just the first rowkey column. First V is id=51 - val rowRange3 = RowKeyChunkScan(BinaryRecord(dataset2, Seq("V")), - BinaryRecord(dataset2, Seq("Z"))) - val rowIter3 = memStore.scanRows(dataset2, Seq(0), method1, rowRange3) - rowIter3.length should equal (41) - } - - // TODO: FilteredPartitionScan() for ColumnStores does not work without an index right now - ignore should "filter rows written with multiple column partition keys" in { - import GdeltTestData._ - memStore.setup(dataset3, 0, TestData.storeConf) - val stream = Observable.now(records(dataset3)) - // Force flush of all groups at end - memStore.ingestStream(dataset3.ref, 0, stream ++ FlushStream.allGroups(4), s, 86400).futureValue - - val paramSet = colStore.getScanSplits(dataset.ref, 1) - paramSet should have length (1) - - // Test 1: IN query on first column only - val filter1 = ColumnFilter("Actor2Code", Filter.In(Set("JPN".utf8, "KHM".utf8))) - val method1 = FilteredPartitionScan(paramSet.head, Seq(filter1)) - - val readSegs1 = colStore.stats.readPartitions - val rows = memStore.scanRows(dataset3, dataset3.colIDs("NumArticles", "Actor2Code").get, method1).toSeq - rows.map(_.getInt(0)).sum should equal (30) - rows.map(_.filoUTF8String(1)).toSet should equal (Set("JPN".utf8, "KHM".utf8)) - (colStore.stats.readPartitions - readSegs1) should equal (2) - - // Test 2: = filter on both partition columns - val filters = Seq(ColumnFilter("Actor2Code", Filter.Equals("JPN".utf8)), - ColumnFilter("Year", Filter.Equals(1979))) - val method2 = FilteredPartitionScan(paramSet.head, filters) - val rowIt = memStore.scanRows(dataset3, dataset3.colIDs("NumArticles").get, method2) - rowIt.map(_.getInt(0)).sum should equal (10) - } - // "rangeVectors api" should "return Range Vectors for given filter and read all rows" in { ignore should "return Range Vectors for given filter and read all rows" in { import GdeltTestData._ diff --git a/query/src/main/scala/filodb/query/exec/RowKeyRange.scala b/query/src/main/scala/filodb/query/exec/RowKeyRange.scala deleted file mode 100644 index 0047c9f442..0000000000 --- a/query/src/main/scala/filodb/query/exec/RowKeyRange.scala +++ /dev/null @@ -1,12 +0,0 @@ -package filodb.query.exec - -import filodb.core.binaryrecord.BinaryRecord - -sealed trait RowKeyRange - -case class RowKeyInterval(from: BinaryRecord, to: BinaryRecord) extends RowKeyRange -case object AllChunks extends RowKeyRange -case object WriteBuffers extends RowKeyRange -case object InMemoryChunks extends RowKeyRange -case object EncodedChunks extends RowKeyRange - diff --git a/query/src/main/scala/filodb/query/exec/SelectChunkInfosExec.scala b/query/src/main/scala/filodb/query/exec/SelectChunkInfosExec.scala index 3474b637b6..551dd26f43 100644 --- a/query/src/main/scala/filodb/query/exec/SelectChunkInfosExec.scala +++ b/query/src/main/scala/filodb/query/exec/SelectChunkInfosExec.scala @@ -39,7 +39,7 @@ final case class SelectChunkInfosExec(id: String, dataset: DatasetRef, shard: Int, filters: Seq[ColumnFilter], - rowKeyRange: RowKeyRange, + chunkMethod: ChunkScanMethod, column: Types.ColumnId) extends LeafExecPlan { import SelectChunkInfosExec._ @@ -51,13 +51,6 @@ final case class SelectChunkInfosExec(id: String, (implicit sched: Scheduler, timeout: FiniteDuration): Observable[RangeVector] = { val dataColumn = dataset.dataColumns(column) - val chunkMethod = rowKeyRange match { - case RowKeyInterval(from, to) => RowKeyChunkScan(from, to) - case AllChunks => AllChunkScan - case WriteBuffers => WriteBufferChunkScan - case InMemoryChunks => InMemoryChunkScan - case EncodedChunks => ??? - } val partMethod = FilteredPartitionScan(ShardSplit(shard), filters) val partCols = dataset.infosFromIDs(dataset.partitionColumns.map(_.id)) val numGroups = source.groupsInDataset(dataset) @@ -73,6 +66,6 @@ final case class SelectChunkInfosExec(id: String, } } - protected def args: String = s"shard=$shard, rowKeyRange=$rowKeyRange, filters=$filters, col=$column" + protected def args: String = s"shard=$shard, chunkMethod=$chunkMethod, filters=$filters, col=$column" } diff --git a/query/src/main/scala/filodb/query/exec/SelectRawPartitionsExec.scala b/query/src/main/scala/filodb/query/exec/SelectRawPartitionsExec.scala index 216d5e8014..a6aba2798a 100644 --- a/query/src/main/scala/filodb/query/exec/SelectRawPartitionsExec.scala +++ b/query/src/main/scala/filodb/query/exec/SelectRawPartitionsExec.scala @@ -23,7 +23,7 @@ final case class SelectRawPartitionsExec(id: String, dataset: DatasetRef, shard: Int, filters: Seq[ColumnFilter], - rowKeyRange: RowKeyRange, + chunkMethod: ChunkScanMethod, colIds: Seq[Types.ColumnId]) extends LeafExecPlan { require(colIds.nonEmpty) @@ -38,17 +38,10 @@ final case class SelectRawPartitionsExec(id: String, timeout: FiniteDuration): Observable[RangeVector] = { require(colIds.indexOfSlice(dataset.rowKeyIDs) == 0) - val chunkMethod = rowKeyRange match { - case RowKeyInterval(from, to) => RowKeyChunkScan(from, to) - case AllChunks => AllChunkScan - case WriteBuffers => WriteBufferChunkScan - case InMemoryChunks => InMemoryChunkScan - case EncodedChunks => ??? - } val partMethod = FilteredPartitionScan(ShardSplit(shard), filters) source.rangeVectors(dataset, colIds, partMethod, chunkMethod) } - protected def args: String = s"shard=$shard, rowKeyRange=$rowKeyRange, filters=$filters, colIDs=$colIds" + protected def args: String = s"shard=$shard, chunkMethod=$chunkMethod, filters=$filters, colIDs=$colIds" } diff --git a/query/src/test/scala/filodb/query/exec/SelectRawPartitionsExecSpec.scala b/query/src/test/scala/filodb/query/exec/SelectRawPartitionsExecSpec.scala index a4148ed647..f4b9779c4d 100644 --- a/query/src/test/scala/filodb/query/exec/SelectRawPartitionsExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/SelectRawPartitionsExecSpec.scala @@ -12,12 +12,11 @@ import org.scalatest.time.{Millis, Seconds, Span} import filodb.core.MetricsTestData._ import filodb.core.{TestData, Types} -import filodb.core.binaryrecord.BinaryRecord import filodb.core.binaryrecord2.RecordBuilder import filodb.core.memstore.{FixedMaxPartitionsEvictionPolicy, SomeData, TimeSeriesMemStore} import filodb.core.metadata.Column.ColumnType.{DoubleColumn, HistogramColumn, TimestampColumn} import filodb.core.query.{ColumnFilter, Filter} -import filodb.core.store.{InMemoryMetaStore, NullColumnStore} +import filodb.core.store.{AllChunkScan, InMemoryMetaStore, NullColumnStore, TimeRangeChunkScan} import filodb.memory.MemFactory import filodb.memory.format.{SeqRowReader, ZeroCopyUTF8String} import filodb.query._ @@ -82,7 +81,7 @@ class SelectRawPartitionsExecSpec extends FunSpec with Matchers with ScalaFuture val filters = Seq (ColumnFilter("__name__", Filter.Equals("http_req_total".utf8)), ColumnFilter("job", Filter.Equals("myCoolService".utf8))) val execPlan = SelectRawPartitionsExec("someQueryId", now, numRawSamples, dummyDispatcher, - timeseriesDataset.ref, 0, filters, AllChunks, Seq(0, 1)) + timeseriesDataset.ref, 0, filters, AllChunkScan, Seq(0, 1)) val resp = execPlan.execute(memStore, timeseriesDataset, queryConfig).runAsync.futureValue val result = resp.asInstanceOf[QueryResult] @@ -98,11 +97,11 @@ class SelectRawPartitionsExecSpec extends FunSpec with Matchers with ScalaFuture val filters = Seq (ColumnFilter("__name__", Filter.Equals("http_req_total".utf8)), ColumnFilter("job", Filter.Equals("myCoolService".utf8))) // read from an interval of 100000ms, resulting in 11 samples - val start: BinaryRecord = BinaryRecord(timeseriesDataset, Seq(now - numRawSamples * reportingInterval)) - val end: BinaryRecord = BinaryRecord(timeseriesDataset, Seq(now - (numRawSamples-10) * reportingInterval)) + val startTime = now - numRawSamples * reportingInterval + val endTime = now - (numRawSamples-10) * reportingInterval val execPlan = SelectRawPartitionsExec("someQueryId", now, numRawSamples, dummyDispatcher, timeseriesDataset.ref, 0, - filters, RowKeyInterval(start, end), Seq(0, 1)) + filters, TimeRangeChunkScan(startTime, endTime), Seq(0, 1)) val resp = execPlan.execute(memStore, timeseriesDataset, queryConfig).runAsync.futureValue val result = resp.asInstanceOf[QueryResult] @@ -116,12 +115,10 @@ class SelectRawPartitionsExecSpec extends FunSpec with Matchers with ScalaFuture it ("should read raw Long samples from Memstore using IntervalSelector") { import ZeroCopyUTF8String._ val filters = Seq(ColumnFilter("series", Filter.Equals("Series 1".utf8))) - // read from an interval of 100000ms, resulting in 11 samples - val start: BinaryRecord = BinaryRecord(MMD.dataset1, Seq(100000L)) - val end: BinaryRecord = BinaryRecord(MMD.dataset1, Seq(150000L)) + // read from an interval of 100000ms, resulting in 11 samples val execPlan = SelectRawPartitionsExec("someQueryId", now, numRawSamples, dummyDispatcher, MMD.dataset1.ref, 0, - filters, RowKeyInterval(start, end), Seq(0, 4)) + filters, TimeRangeChunkScan(100000L, 150000L), Seq(0, 4)) val resp = execPlan.execute(memStore, MMD.dataset1, queryConfig).runAsync.futureValue val result = resp.asInstanceOf[QueryResult] @@ -132,12 +129,10 @@ class SelectRawPartitionsExecSpec extends FunSpec with Matchers with ScalaFuture it ("should read raw Histogram samples from Memstore using IntervalSelector") { import ZeroCopyUTF8String._ - val start: BinaryRecord = BinaryRecord(MMD.histDataset, Seq(100000L)) - val end: BinaryRecord = BinaryRecord(MMD.histDataset, Seq(150000L)) val filters = Seq(ColumnFilter("dc", Filter.Equals("0".utf8))) val execPlan = SelectRawPartitionsExec("id1", now, numRawSamples, dummyDispatcher, MMD.histDataset.ref, 0, - filters, RowKeyInterval(start, end), Seq(0, 3)) + filters, TimeRangeChunkScan(100000L, 150000L), Seq(0, 3)) val resp = execPlan.execute(memStore, MMD.histDataset, queryConfig).runAsync.futureValue val result = resp.asInstanceOf[QueryResult] @@ -153,7 +148,7 @@ class SelectRawPartitionsExecSpec extends FunSpec with Matchers with ScalaFuture val filters = Seq (ColumnFilter("__name__", Filter.Equals("http_req_total".utf8)), ColumnFilter("job", Filter.Equals("myCoolService".utf8))) val execPlan = SelectRawPartitionsExec("someQueryId", now, numRawSamples, dummyDispatcher, timeseriesDataset.ref, 0, - filters, AllChunks, Seq(0, 1)) + filters, AllChunkScan, Seq(0, 1)) val start = now - numRawSamples * reportingInterval - 100 // reduce by 100 to not coincide with reporting intervals val step = 20000 val end = now - (numRawSamples-100) * reportingInterval @@ -184,7 +179,7 @@ class SelectRawPartitionsExecSpec extends FunSpec with Matchers with ScalaFuture import ZeroCopyUTF8String._ val filters = Seq(ColumnFilter("series", Filter.Equals("Series 1".utf8))) val execPlan = SelectRawPartitionsExec("someQueryId", now, numRawSamples, dummyDispatcher, MMD.dataset1.ref, 0, - filters, AllChunks, Seq(0, 4)) + filters, AllChunkScan, Seq(0, 4)) // Raw data like 101000, 111000, .... val start = 105000L @@ -204,7 +199,7 @@ class SelectRawPartitionsExecSpec extends FunSpec with Matchers with ScalaFuture import ZeroCopyUTF8String._ val filters = Seq(ColumnFilter("dc", Filter.Equals("0".utf8))) val execPlan = SelectRawPartitionsExec("id1", now, numRawSamples, dummyDispatcher, MMD.histDataset.ref, 0, - filters, AllChunks, Seq(0, 3)) + filters, AllChunkScan, Seq(0, 3)) val start = 105000L val step = 20000L @@ -227,7 +222,7 @@ class SelectRawPartitionsExecSpec extends FunSpec with Matchers with ScalaFuture val filters = Seq (ColumnFilter("__name__", Filter.Equals("http_req_total".utf8)), ColumnFilter("job", Filter.Equals("myCoolService".utf8))) val execPlan = SelectRawPartitionsExec("someQueryId", now, numRawSamples, dummyDispatcher, timeseriesDataset.ref, 0, - filters, AllChunks, Seq(0, 1)) + filters, AllChunkScan, Seq(0, 1)) val resultSchema = execPlan.schema(timeseriesDataset) resultSchema.isTimeSeries shouldEqual true resultSchema.numRowKeyColumns shouldEqual 1 @@ -240,7 +235,7 @@ class SelectRawPartitionsExecSpec extends FunSpec with Matchers with ScalaFuture val filters = Seq (ColumnFilter("__name__", Filter.Equals("http_req_total".utf8)), ColumnFilter("job", Filter.Equals("myCoolService".utf8))) val execPlan = SelectChunkInfosExec("someQueryId", now, numRawSamples, dummyDispatcher, - timeseriesDataset.ref, 0, filters, AllChunks, 0) + timeseriesDataset.ref, 0, filters, AllChunkScan, 0) val resp = execPlan.execute(memStore, timeseriesDataset, queryConfig).runAsync.futureValue info(s"resp = $resp") val result = resp.asInstanceOf[QueryResult] diff --git a/spark/src/main/scala/filodb.spark/FiloRelation.scala b/spark/src/main/scala/filodb.spark/FiloRelation.scala index 951ac7aa53..16a1e9f953 100644 --- a/spark/src/main/scala/filodb.spark/FiloRelation.scala +++ b/spark/src/main/scala/filodb.spark/FiloRelation.scala @@ -13,7 +13,6 @@ import org.apache.spark.unsafe.types.UTF8String import filodb.coordinator.client.Client.parse import filodb.core._ -import filodb.core.binaryrecord.BinaryRecord import filodb.core.metadata.{Column, Dataset} import filodb.core.query.{ColumnFilter, Filter => FF, KeyFilter} import filodb.core.store._ @@ -149,10 +148,11 @@ object FiloRelation extends StrictLogging { } AllChunkScan } else { - val firstKey = BinaryRecord(dataset, ranges.map(_.get._1)) - val lastKey = BinaryRecord(dataset, ranges.map(_.get._2)) - logger.info(s"Pushdown of rowkey scan ($firstKey, $lastKey)") - RowKeyChunkScan(firstKey, lastKey) + // WARNING: we take the first value from the keys and assume it is the timestamp. + val startTime = ranges.head.get._1.asInstanceOf[Long] + val endTime = ranges.head.get._2.asInstanceOf[Long] + logger.info(s"Pushdown of time range ($startTime, $endTime)") + TimeRangeChunkScan(startTime, endTime) } } else { logger.info(s"Filters $groupedFilters skipped some row key columns, must be from row key columns 0..n") From c5d39146107312c816693475ecc871cb5a7dc684 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Tue, 12 Mar 2019 10:34:46 -0700 Subject: [PATCH 13/41] feat(query): Include partId and shardNums in query result on verbose option (#276) --- .../client/FiloKryoSerializers.scala | 3 ++- .../client/SerializationSpec.scala | 2 +- .../filodb.core/memstore/TimeSeriesShard.scala | 1 + .../main/scala/filodb.core/query/RangeVector.scala | 9 +++++++-- .../main/scala/filodb.core/store/ChunkSource.scala | 3 ++- .../scala/filodb.core/query/RangeVectorSpec.scala | 3 ++- .../main/scala/filodb/http/PrometheusApiRoute.scala | 13 +++++++------ .../filodb/prometheus/query/PrometheusModel.scala | 13 +++++++++---- .../filodb/query/exec/SelectChunkInfosExec.scala | 2 +- 9 files changed, 32 insertions(+), 17 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/client/FiloKryoSerializers.scala b/coordinator/src/main/scala/filodb.coordinator/client/FiloKryoSerializers.scala index 48cf5412f6..8b23d24ecd 100644 --- a/coordinator/src/main/scala/filodb.coordinator/client/FiloKryoSerializers.scala +++ b/coordinator/src/main/scala/filodb.coordinator/client/FiloKryoSerializers.scala @@ -37,7 +37,7 @@ class PartitionRangeVectorKeySerializer extends KryoSerializer[PartitionRangeVec val schema = kryo.readObject(input, classOf[RecordSchema2]) val keyCols = kryo.readClassAndObject(input) PartitionRangeVectorKey(partBytes, UnsafeUtils.arayOffset, - schema, keyCols.asInstanceOf[Seq[ColumnInfo]], input.readInt, input.readInt) + schema, keyCols.asInstanceOf[Seq[ColumnInfo]], input.readInt, input.readInt, input.readInt) } override def write(kryo: Kryo, output: Output, key: PartitionRangeVectorKey): Unit = { @@ -46,6 +46,7 @@ class PartitionRangeVectorKeySerializer extends KryoSerializer[PartitionRangeVec kryo.writeClassAndObject(output, key.partKeyCols) output.writeInt(key.sourceShard) output.writeInt(key.groupNum) + output.writeInt(key.partId) } } diff --git a/coordinator/src/test/scala/filodb.coordinator/client/SerializationSpec.scala b/coordinator/src/test/scala/filodb.coordinator/client/SerializationSpec.scala index 86eb14a8c6..482b1bf206 100644 --- a/coordinator/src/test/scala/filodb.coordinator/client/SerializationSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/client/SerializationSpec.scala @@ -136,7 +136,7 @@ class SerializationSpec extends ActorTest(SerializationSpecConfig.getNewSystem) // scalastyle:off null val rvKey = new PartitionRangeVectorKey(null, defaultPartKey, dataset1.partKeySchema, - Seq(ColumnInfo("string", ColumnType.StringColumn)), 1, 5) + Seq(ColumnInfo("string", ColumnType.StringColumn)), 1, 5, 100) val rowbuf = tuples.map { t => new SeqRowReader(Seq[Any](t._1, t._2)) diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala index fab4a6a294..a67c5c3bba 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala @@ -1063,6 +1063,7 @@ class TimeSeriesShard(val dataset: Dataset, 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=${dataset.ref} shard=$shardNum") removePartition(partitionObj) partsRemoved += 1 maxEndTime = Math.max(maxEndTime, endTime) diff --git a/core/src/main/scala/filodb.core/query/RangeVector.scala b/core/src/main/scala/filodb.core/query/RangeVector.scala index 0024a1300b..01c5219e8a 100644 --- a/core/src/main/scala/filodb.core/query/RangeVector.scala +++ b/core/src/main/scala/filodb.core/query/RangeVector.scala @@ -20,6 +20,7 @@ import filodb.memory.format.{RowReader, ZeroCopyUTF8String => UTF8Str} trait RangeVectorKey extends java.io.Serializable { def labelValues: Map[UTF8Str, UTF8Str] def sourceShards: Seq[Int] + def partIds: Seq[Int] override def toString: String = s"/shard:${sourceShards.mkString(",")}/$labelValues" } @@ -40,8 +41,10 @@ final case class PartitionRangeVectorKey(partBase: Array[Byte], partSchema: RecordSchema, partKeyCols: Seq[ColumnInfo], sourceShard: Int, - groupNum: Int) extends RangeVectorKey { + groupNum: Int, + partId: Int) extends RangeVectorKey { override def sourceShards: Seq[Int] = Seq(sourceShard) + override def partIds: Seq[Int] = Seq(partId) def labelValues: Map[UTF8Str, UTF8Str] = { partKeyCols.zipWithIndex.flatMap { case (c, pos) => c.colType match { @@ -58,7 +61,9 @@ final case class PartitionRangeVectorKey(partBase: Array[Byte], override def toString: String = s"/shard:$sourceShard/${partSchema.stringify(partBase, partOffset)} [grp$groupNum]" } -final case class CustomRangeVectorKey(labelValues: Map[UTF8Str, UTF8Str], sourceShards: Seq[Int] = Nil) +final case class CustomRangeVectorKey(labelValues: Map[UTF8Str, UTF8Str], + sourceShards: Seq[Int] = Nil, + partIds: Seq[Int] = Nil) extends RangeVectorKey { } diff --git a/core/src/main/scala/filodb.core/store/ChunkSource.scala b/core/src/main/scala/filodb.core/store/ChunkSource.scala index 9957adc434..c83a4d30ba 100644 --- a/core/src/main/scala/filodb.core/store/ChunkSource.scala +++ b/core/src/main/scala/filodb.core/store/ChunkSource.scala @@ -109,7 +109,8 @@ trait ChunkSource extends RawChunkSource { val subgroup = TimeSeriesShard.partKeyGroup(dataset.partKeySchema, partition.partKeyBase, partition.partKeyOffset, numGroups) val key = new PartitionRangeVectorKey(partition.partKeyBase, partition.partKeyOffset, - dataset.partKeySchema, partCols, partition.shard, subgroup) + dataset.partKeySchema, partCols, partition.shard, + subgroup, partition.partID) RawDataRangeVector(key, partition, chunkMethod, ids) } } diff --git a/core/src/test/scala/filodb.core/query/RangeVectorSpec.scala b/core/src/test/scala/filodb.core/query/RangeVectorSpec.scala index 6c8d04e85a..353dac1e9f 100644 --- a/core/src/test/scala/filodb.core/query/RangeVectorSpec.scala +++ b/core/src/test/scala/filodb.core/query/RangeVectorSpec.scala @@ -18,7 +18,8 @@ class RangeVectorSpec extends FunSpec with Matchers { }.iterator override def key: RangeVectorKey = new RangeVectorKey { def labelValues: Map[ZeroCopyUTF8String, ZeroCopyUTF8String] = Map.empty - def sourceShards: Seq[Int] = Seq(0) + def sourceShards: Seq[Int] = Nil + def partIds: Seq[Int] = Nil } } diff --git a/http/src/main/scala/filodb/http/PrometheusApiRoute.scala b/http/src/main/scala/filodb/http/PrometheusApiRoute.scala index 37cb4427ca..e5e3346f6b 100644 --- a/http/src/main/scala/filodb/http/PrometheusApiRoute.scala +++ b/http/src/main/scala/filodb/http/PrometheusApiRoute.scala @@ -39,9 +39,10 @@ class PrometheusApiRoute(nodeCoord: ActorRef, settings: HttpSettings)(implicit a // [Range Queries](https://prometheus.io/docs/prometheus/latest/querying/api/#range-queries) path( "api" / "v1" / "query_range") { get { - parameter('query.as[String], 'start.as[Double], 'end.as[Double], 'step.as[Int]) { (query, start, end, step) => + parameter('query.as[String], 'start.as[Double], 'end.as[Double], + 'step.as[Int], 'verbose.as[Boolean].?) { (query, start, end, step, verbose) => val logicalPlan = Parser.queryRangeToLogicalPlan(query, TimeStepParams(start.toLong, step, end.toLong)) - askQueryAndRespond(dataset, logicalPlan) + askQueryAndRespond(dataset, logicalPlan, verbose.getOrElse(false)) } } } ~ @@ -51,9 +52,9 @@ class PrometheusApiRoute(nodeCoord: ActorRef, settings: HttpSettings)(implicit a // [Instant Queries](https://prometheus.io/docs/prometheus/latest/querying/api/#instant-queries) path( "api" / "v1" / "query") { get { - parameter('query.as[String], 'time.as[Double]) { (query, time) => + parameter('query.as[String], 'time.as[Double], 'verbose.as[Boolean].?) { (query, time, verbose) => val logicalPlan = Parser.queryToLogicalPlan(query, time.toLong) - askQueryAndRespond(dataset, logicalPlan) + askQueryAndRespond(dataset, logicalPlan, verbose.getOrElse(false)) } } } ~ @@ -99,10 +100,10 @@ class PrometheusApiRoute(nodeCoord: ActorRef, settings: HttpSettings)(implicit a } } - private def askQueryAndRespond(dataset: String, logicalPlan: LogicalPlan) = { + private def askQueryAndRespond(dataset: String, logicalPlan: LogicalPlan, verbose: Boolean) = { val command = LogicalPlan2Query(DatasetRef.fromDotString(dataset), logicalPlan, queryOptions) onSuccess(asyncAsk(nodeCoord, command)) { - case qr: QueryResult => complete(toPromSuccessResponse(qr)) + case qr: QueryResult => complete(toPromSuccessResponse(qr, verbose)) case qr: QueryError => complete(toPromErrorResponse(qr)) case UnknownDataset => complete(Codes.NotFound -> ErrorResponse("badQuery", s"Dataset $dataset is not registered")) diff --git a/prometheus/src/main/scala/filodb/prometheus/query/PrometheusModel.scala b/prometheus/src/main/scala/filodb/prometheus/query/PrometheusModel.scala index 99be888bba..3953317a71 100644 --- a/prometheus/src/main/scala/filodb/prometheus/query/PrometheusModel.scala +++ b/prometheus/src/main/scala/filodb/prometheus/query/PrometheusModel.scala @@ -76,8 +76,8 @@ object PrometheusModel { b.build() } - def toPromSuccessResponse(qr: filodb.query.QueryResult): SuccessResponse = { - SuccessResponse(Data(toPromResultType(qr.resultType), qr.result.map(toPromResult(_)))) + def toPromSuccessResponse(qr: filodb.query.QueryResult, verbose: Boolean): SuccessResponse = { + SuccessResponse(Data(toPromResultType(qr.resultType), qr.result.map(toPromResult(_, verbose)))) } def toPromResultType(r: QueryResultType): String = { @@ -91,8 +91,13 @@ object PrometheusModel { /** * Used to send out HTTP response */ - def toPromResult(srv: SerializableRangeVector): Result = { - Result(srv.key.labelValues.map { case (k, v) => (k.toString, v.toString)}, + def toPromResult(srv: SerializableRangeVector, verbose: Boolean): Result = { + val tags = srv.key.labelValues.map { case (k, v) => (k.toString, v.toString)} ++ + (if (verbose) Map("_shards_" -> srv.key.sourceShards.mkString(","), + "_partIds_" -> srv.key.partIds.mkString(",")) + else Map.empty) + + Result(tags, // remove NaN in HTTP results // Known Issue: Until we support NA in our vectors, we may not be able to return NaN as an end-of-time-series // in HTTP raw query results. diff --git a/query/src/main/scala/filodb/query/exec/SelectChunkInfosExec.scala b/query/src/main/scala/filodb/query/exec/SelectChunkInfosExec.scala index 551dd26f43..3971283a03 100644 --- a/query/src/main/scala/filodb/query/exec/SelectChunkInfosExec.scala +++ b/query/src/main/scala/filodb/query/exec/SelectChunkInfosExec.scala @@ -61,7 +61,7 @@ final case class SelectChunkInfosExec(id: String, val subgroup = TimeSeriesShard.partKeyGroup(dataset.partKeySchema, partition.partKeyBase, partition.partKeyOffset, numGroups) val key = new PartitionRangeVectorKey(partition.partKeyBase, partition.partKeyOffset, - dataset.partKeySchema, partCols, shard, subgroup) + dataset.partKeySchema, partCols, shard, subgroup, partition.partID) ChunkInfoRangeVector(key, partition, chunkMethod, dataColumn) } } From 6e4d058e17394457684f5c8f8bea4e7dcf1db12d Mon Sep 17 00:00:00 2001 From: whizkido Date: Tue, 12 Mar 2019 14:33:41 -0700 Subject: [PATCH 14/41] feat(query):Support for comparison operators (#274) * Support for Comparison Operators. I plan to change to channge the AST to Logical Plan conversion sometime in the future. * Support for comparison operator * Review comments are taken care of * Turning off scala style --- .../filodb/prometheus/ast/Operators.scala | 12 ++-- .../main/scala/filodb/query/PlanEnums.scala | 12 ++++ .../binaryOp/BinaryOperatorFunction.scala | 38 ++++++----- .../exec/rangefn/BinaryOperatorSpec.scala | 66 +++++++++++++++---- 4 files changed, 93 insertions(+), 35 deletions(-) diff --git a/prometheus/src/main/scala/filodb/prometheus/ast/Operators.scala b/prometheus/src/main/scala/filodb/prometheus/ast/Operators.scala index f1e9d2d96e..94381fa99d 100644 --- a/prometheus/src/main/scala/filodb/prometheus/ast/Operators.scala +++ b/prometheus/src/main/scala/filodb/prometheus/ast/Operators.scala @@ -45,27 +45,27 @@ trait Operators { } case class NotEqual(isBool: Boolean) extends Comparision { - override def getPlanOperator: BinaryOperator = BinaryOperator.NEQ + override def getPlanOperator: BinaryOperator = if (!isBool) BinaryOperator.NEQ else BinaryOperator.NEQ_BOOL } case class Eq(isBool: Boolean) extends Comparision { - override def getPlanOperator: BinaryOperator = BinaryOperator.EQL + override def getPlanOperator: BinaryOperator = if (!isBool) BinaryOperator.EQL else BinaryOperator.EQL_BOOL } case class Gt(isBool: Boolean) extends Comparision { - override def getPlanOperator: BinaryOperator = BinaryOperator.GTR + override def getPlanOperator: BinaryOperator = if (!isBool) BinaryOperator.GTR else BinaryOperator.GTR_BOOL } case class Gte(isBool: Boolean) extends Comparision { - override def getPlanOperator: BinaryOperator = BinaryOperator.GTE + override def getPlanOperator: BinaryOperator = if (!isBool) BinaryOperator.GTE else BinaryOperator.GTE_BOOL } case class Lt(isBool: Boolean) extends Comparision { - override def getPlanOperator: BinaryOperator = BinaryOperator.LSS + override def getPlanOperator: BinaryOperator = if (!isBool) BinaryOperator.LSS else BinaryOperator.LSS_BOOL } case class Lte(isBool: Boolean) extends Comparision { - override def getPlanOperator: BinaryOperator = BinaryOperator.LTE + override def getPlanOperator: BinaryOperator = if (!isBool) BinaryOperator.LTE else BinaryOperator.LTE_BOOL } case class LabelMatch(label: String, labelMatchOp: Operator, value: String) extends PromToken diff --git a/query/src/main/scala/filodb/query/PlanEnums.scala b/query/src/main/scala/filodb/query/PlanEnums.scala index 976f678d66..948f54ff89 100644 --- a/query/src/main/scala/filodb/query/PlanEnums.scala +++ b/query/src/main/scala/filodb/query/PlanEnums.scala @@ -158,6 +158,18 @@ object BinaryOperator extends Enum[BinaryOperator] { case object GTR extends ComparisonOperator + case object EQL_BOOL extends ComparisonOperator + + case object NEQ_BOOL extends ComparisonOperator + + case object LTE_BOOL extends ComparisonOperator + + case object LSS_BOOL extends ComparisonOperator + + case object GTE_BOOL extends ComparisonOperator + + case object GTR_BOOL extends ComparisonOperator + case object EQLRegex extends BinaryOperator // FIXME when implemented case object NEQRegex extends BinaryOperator // FIXME when implemented diff --git a/query/src/main/scala/filodb/query/exec/binaryOp/BinaryOperatorFunction.scala b/query/src/main/scala/filodb/query/exec/binaryOp/BinaryOperatorFunction.scala index 1afc07f374..988bbb4adf 100644 --- a/query/src/main/scala/filodb/query/exec/binaryOp/BinaryOperatorFunction.scala +++ b/query/src/main/scala/filodb/query/exec/binaryOp/BinaryOperatorFunction.scala @@ -15,27 +15,29 @@ object BinaryOperatorFunction { * @param function to be invoked * @return the function */ + + // scalastyle:off def factoryMethod(function: BinaryOperator): ScalarFunction = { function match { - case SUB => new ScalarFunction { - override def calculate(lhs: Double, rhs: Double): Double = lhs - rhs - } - case ADD => new ScalarFunction { - override def calculate(lhs: Double, rhs: Double): Double = lhs + rhs - } - case MUL => new ScalarFunction { - override def calculate(lhs: Double, rhs: Double): Double = lhs * rhs - } - case MOD => new ScalarFunction { - override def calculate(lhs: Double, rhs: Double): Double = lhs % rhs - } - case DIV => new ScalarFunction { - override def calculate(lhs: Double, rhs: Double): Double = lhs / rhs - } - case POW => new ScalarFunction { - override def calculate(lhs: Double, rhs: Double): Double = math.pow(lhs, rhs) - } + case SUB => new ScalarFunction { override def calculate(lhs: Double, rhs: Double): Double = lhs - rhs } + case ADD => new ScalarFunction { override def calculate(lhs: Double, rhs: Double): Double = lhs + rhs } + case MUL => new ScalarFunction { override def calculate(lhs: Double, rhs: Double): Double = lhs * rhs } + case MOD => new ScalarFunction { override def calculate(lhs: Double, rhs: Double): Double = lhs % rhs } + case DIV => new ScalarFunction { override def calculate(lhs: Double, rhs: Double): Double = lhs / rhs } + case POW => new ScalarFunction { override def calculate(lhs: Double, rhs: Double): Double = math.pow(lhs, rhs) } + case LSS => new ScalarFunction { override def calculate(lhs: Double, rhs: Double): Double = if (lhs < rhs) lhs else Double.NaN } + case LTE => new ScalarFunction { override def calculate(lhs: Double, rhs: Double): Double = if (lhs <= rhs) lhs else Double.NaN } + case GTR => new ScalarFunction { override def calculate(lhs: Double, rhs: Double): Double = if (lhs > rhs) lhs else Double.NaN } + case GTE => new ScalarFunction { override def calculate(lhs: Double, rhs: Double): Double = if (lhs >= rhs) lhs else Double.NaN } + case EQL => new ScalarFunction { override def calculate(lhs: Double, rhs: Double): Double = if (lhs == rhs) lhs else Double.NaN } + case NEQ => new ScalarFunction { override def calculate(lhs: Double, rhs: Double): Double = if (lhs != rhs) lhs else Double.NaN } + case LSS_BOOL => new ScalarFunction { override def calculate(lhs: Double, rhs: Double): Double = if (lhs < rhs) 1.0 else 0.0 } + case LTE_BOOL => new ScalarFunction { override def calculate(lhs: Double, rhs: Double): Double = if (lhs <= rhs) 1.0 else 0.0 } + case GTR_BOOL => new ScalarFunction { override def calculate(lhs: Double, rhs: Double): Double = if (lhs > rhs) 1.0 else 0.0 } + case GTE_BOOL => new ScalarFunction { override def calculate(lhs: Double, rhs: Double): Double = if (lhs >= rhs) 1.0 else 0.0 } + case EQL_BOOL => new ScalarFunction { override def calculate(lhs: Double, rhs: Double): Double = if (lhs == rhs) 1.0 else 0.0 } + case NEQ_BOOL => new ScalarFunction { override def calculate(lhs: Double, rhs: Double): Double = if (lhs != rhs) 1.0 else 0.0 } case _ => throw new UnsupportedOperationException(s"$function not supported.") } } diff --git a/query/src/test/scala/filodb/query/exec/rangefn/BinaryOperatorSpec.scala b/query/src/test/scala/filodb/query/exec/rangefn/BinaryOperatorSpec.scala index fdde02eaa1..b7ac115508 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/BinaryOperatorSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/BinaryOperatorSpec.scala @@ -52,6 +52,8 @@ class BinaryOperatorSpec extends FunSpec with Matchers with ScalaFutures { override def rows: Iterator[RowReader] = data.iterator }) fireBinaryOperatorTests(samples) + fireComparatorOperatorTests(samples) + } it ("should handle NaN") { @@ -79,6 +81,8 @@ class BinaryOperatorSpec extends FunSpec with Matchers with ScalaFutures { } ) fireBinaryOperatorTests(samples) + fireComparatorOperatorTests(samples) + } it ("should handle special cases") { @@ -100,6 +104,7 @@ class BinaryOperatorSpec extends FunSpec with Matchers with ScalaFutures { } ) fireBinaryOperatorTests(samples) + fireComparatorOperatorTests(samples) } private def fireBinaryOperatorTests(samples: Array[RangeVector]): Unit = { @@ -151,19 +156,58 @@ class BinaryOperatorSpec extends FunSpec with Matchers with ScalaFutures { // power - suffix val expectedPow2 = samples.map(_.rows.map(v => math.pow(v.getDouble(1), scalar))) applyBinaryOperationAndAssertResult(samples, expectedPow2, BinaryOperator.POW, scalar, false) + } - it ("should handle unknown functions") { - // sort_desc - the[UnsupportedOperationException] thrownBy { - val binaryOpMapper = exec.ScalarOperationMapper(BinaryOperator.EQL, 10, true) - binaryOpMapper(Observable.fromIterable(sampleBase), queryConfig, 1000, resultSchema) - } should have message "EQL not supported." - - the[UnsupportedOperationException] thrownBy { - val binaryOpMapper = exec.ScalarOperationMapper(BinaryOperator.GTE, 10, false) - binaryOpMapper(Observable.fromIterable(sampleBase), queryConfig, 1000, resultSchema) - } should have message "GTE not supported." + private def fireComparatorOperatorTests(samples: Array[RangeVector]): Unit = { + + // GTE - prefix + val expectedGTE = samples.map(_.rows.map(v => if (scalar >= v.getDouble(1)) scalar else Double.NaN)) + applyBinaryOperationAndAssertResult(samples, expectedGTE, BinaryOperator.GTE, scalar, true) + + // GTR - prefix + val expectedGTR = samples.map(_.rows.map(v => if (scalar > v.getDouble(1)) scalar else Double.NaN)) + applyBinaryOperationAndAssertResult(samples, expectedGTR, BinaryOperator.GTR, scalar, true) + + // LTE - prefix + val expectedLTE = samples.map(_.rows.map(v => if (scalar <= v.getDouble(1)) scalar else Double.NaN)) + applyBinaryOperationAndAssertResult(samples, expectedLTE, BinaryOperator.LTE, scalar, true) + + // LTR - prefix + val expectedLTR = samples.map(_.rows.map(v => if (scalar < v.getDouble(1)) scalar else Double.NaN)) + applyBinaryOperationAndAssertResult(samples, expectedLTR, BinaryOperator.LSS, scalar, true) + + // EQL - prefix + val expectedEQL = samples.map(_.rows.map(v => if (scalar == v.getDouble(1)) scalar else Double.NaN)) + applyBinaryOperationAndAssertResult(samples, expectedEQL, BinaryOperator.EQL, scalar, true) + + // NEQ - prefix + val expectedNEQ = samples.map(_.rows.map(v => if (scalar != v.getDouble(1)) scalar else Double.NaN)) + applyBinaryOperationAndAssertResult(samples, expectedNEQ, BinaryOperator.NEQ, scalar, true) + + // GTE_BOOL - prefix + val expectedGTE_BOOL = samples.map(_.rows.map(v => if (scalar >= v.getDouble(1)) 1.0 else 0.0)) + applyBinaryOperationAndAssertResult(samples, expectedGTE_BOOL, BinaryOperator.GTE_BOOL, scalar, true) + + // GTR_BOOL - prefix + val expectedGTR_BOOL = samples.map(_.rows.map(v => if (scalar > v.getDouble(1)) 1.0 else 0.0)) + applyBinaryOperationAndAssertResult(samples, expectedGTR_BOOL, BinaryOperator.GTR_BOOL, scalar, true) + + // LTE_BOOL - prefix + val expectedLTE_BOOL = samples.map(_.rows.map(v => if (scalar <= v.getDouble(1)) 1.0 else 0.0)) + applyBinaryOperationAndAssertResult(samples, expectedLTE_BOOL, BinaryOperator.LTE_BOOL, scalar, true) + + // LTR_BOOL - prefix + val expectedLTR_BOOL = samples.map(_.rows.map(v => if (scalar < v.getDouble(1)) 1.0 else 0.0)) + applyBinaryOperationAndAssertResult(samples, expectedLTR_BOOL, BinaryOperator.LSS_BOOL, scalar, true) + + // EQL_BOOL - prefix + val expectedEQL_BOOL = samples.map(_.rows.map(v => if (scalar == v.getDouble(1)) 1.0 else 0.0)) + applyBinaryOperationAndAssertResult(samples, expectedEQL_BOOL, BinaryOperator.EQL_BOOL, scalar, true) + + // NEQ_BOOL - prefix + val expectedNEQ_BOOL = samples.map(_.rows.map(v => if (scalar != v.getDouble(1)) 1.0 else 0.0)) + applyBinaryOperationAndAssertResult(samples, expectedNEQ_BOOL, BinaryOperator.NEQ_BOOL, scalar, true) } it ("should fail with wrong calculation") { From d3c4c65fdf61b75c20dc6133243e13a8ee5c820d Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Wed, 13 Mar 2019 11:18:21 -0700 Subject: [PATCH 15/41] feat(memory, core): Row based, NibblePacked HistogramColumn (#265) --- conf/timeseries-dev-source.conf | 4 + .../NodeCoordinatorActor.scala | 11 + .../binaryrecord2/RecordSchema.scala | 15 +- .../scala/filodb.core/memstore/MemStore.scala | 12 +- .../memstore/TimeSeriesShard.scala | 3 +- .../memstore/WriteBufferPool.scala | 20 +- .../filodb.core/store/IngestionConfig.scala | 7 +- .../src/test/scala/filodb.core/TestData.scala | 1 + .../downsample/ShardDownsamplerSpec.scala | 5 +- .../memstore/PartitionSetSpec.scala | 3 +- .../memstore/TimeSeriesPartitionSpec.scala | 4 +- doc/compression.md | 8 +- .../filodb/http/ClusterApiRouteSpec.scala | 3 +- .../filodb.memory/format/BinaryVector.scala | 4 +- .../main/scala/filodb.memory/format/Ptr.scala | 36 ++- .../scala/filodb.memory/format/Section.scala | 91 ++++++ .../filodb.memory/format/WireFormat.scala | 12 +- .../format/vectors/Histogram.scala | 6 +- .../format/vectors/HistogramCompressor.scala | 70 +++-- .../format/vectors/HistogramVector.scala | 280 ++++++++++-------- .../format/vectors/HistogramVectorTest.scala | 59 +++- .../rangefn/AggrOverTimeFunctionsSpec.scala | 6 +- 22 files changed, 443 insertions(+), 217 deletions(-) create mode 100644 memory/src/main/scala/filodb.memory/format/Section.scala diff --git a/conf/timeseries-dev-source.conf b/conf/timeseries-dev-source.conf index d72f52b981..f8ce6b1baa 100644 --- a/conf/timeseries-dev-source.conf +++ b/conf/timeseries-dev-source.conf @@ -29,6 +29,10 @@ max-chunks-size = 400 + # Write buffer size, in bytes, for blob columns (histograms, UTF8Strings). Since these are variable data types, + # we need a maximum size, not a maximum number of items. + max-blob-buffer-size = 15000 + # Number of bytes of offheap mem to allocate to chunk storage in each shard. Ex. 1000MB, 1G, 2GB # Assume 5 bytes per sample, should be roughly equal to (# samples per time series) * (# time series) shard-mem-size = 512MB diff --git a/coordinator/src/main/scala/filodb.coordinator/NodeCoordinatorActor.scala b/coordinator/src/main/scala/filodb.coordinator/NodeCoordinatorActor.scala index 03da3ae950..d96e20fe65 100644 --- a/coordinator/src/main/scala/filodb.coordinator/NodeCoordinatorActor.scala +++ b/coordinator/src/main/scala/filodb.coordinator/NodeCoordinatorActor.scala @@ -226,12 +226,23 @@ private[filodb] final class NodeCoordinatorActor(metaStore: MetaStore, } } + private def aliveIngesters: Seq[(DatasetRef, ActorRef)] = { + val kids = context.children.toBuffer + ingesters.toSeq.filter { case (dsRef, actorRef) => kids contains actorRef } + } + private def reset(origin: ActorRef): Unit = { ingesters.values.foreach(_ ! PoisonPill) queryActors.values.foreach(_ ! PoisonPill) ingesters.clear() queryActors.clear() memStore.reset() + + // Wait for all ingestor children to die + while (aliveIngesters.nonEmpty) { + logger.info(s"In reset, waiting for children to die.... ingesters=$ingesters children=${context.children}") + Thread sleep 250 + } origin ! NodeProtocol.StateReset } diff --git a/core/src/main/scala/filodb.core/binaryrecord2/RecordSchema.scala b/core/src/main/scala/filodb.core/binaryrecord2/RecordSchema.scala index c7519dc9bb..5524ec0adf 100644 --- a/core/src/main/scala/filodb.core/binaryrecord2/RecordSchema.scala +++ b/core/src/main/scala/filodb.core/binaryrecord2/RecordSchema.scala @@ -149,12 +149,15 @@ final class RecordSchema(val columns: Seq[ColumnInfo], * 2-byte length prefix. Since the DirectBuffer is already allocated, this results in no new allocations. * Could be used to efficiently retrieve blobs or histograms again and again. */ - def blobAsBuffer(base: Any, offset: Long, index: Int, buf: DirectBuffer): Unit = base match { - case a: Array[Byte] => - buf.wrap(a, utf8StringOffset(base, offset, index).toInt - UnsafeUtils.arayOffset, - blobNumBytes(base, offset, index) + 2) - case UnsafeUtils.ZeroPointer => - buf.wrap(utf8StringOffset(base, offset, index), blobNumBytes(base, offset, index) + 2) + def blobAsBuffer(base: Any, offset: Long, index: Int, buf: DirectBuffer): Unit = { + // Number of bytes to give out should not be beyond range of record + val blobLen = Math.min(numBytes(base, offset), blobNumBytes(base, offset, index) + 2) + base match { + case a: Array[Byte] => + buf.wrap(a, utf8StringOffset(base, offset, index).toInt - UnsafeUtils.arayOffset, blobLen) + case UnsafeUtils.ZeroPointer => + buf.wrap(utf8StringOffset(base, offset, index), blobLen) + } } // Same as above but allocates a new UnsafeBuffer wrapping the blob as a reference diff --git a/core/src/main/scala/filodb.core/memstore/MemStore.scala b/core/src/main/scala/filodb.core/memstore/MemStore.scala index fc8e9abb70..29261d29c8 100644 --- a/core/src/main/scala/filodb.core/memstore/MemStore.scala +++ b/core/src/main/scala/filodb.core/memstore/MemStore.scala @@ -205,17 +205,14 @@ trait MemStore extends ChunkSource { } object MemStore { - // TODO: make the max string vector size configurable. - val MaxUTF8VectorSize = 8192 - val MaxHistogramBuckets = 64 - /** * Figures out the AppendableVectors for each column, depending on type and whether it is a static/ * constant column for each partition. */ def getAppendables(memFactory: MemFactory, dataset: Dataset, - maxElements: Int): Array[BinaryAppendableVector[_]] = + config: StoreConfig): Array[BinaryAppendableVector[_]] = { + val maxElements = config.maxChunksSize dataset.dataColumns.zipWithIndex.map { case (col, index) => col.columnType match { // Time series data doesn't really need the NA/null functionality, so use more optimal vectors @@ -224,9 +221,10 @@ object MemStore { case LongColumn => bv.LongBinaryVector.appendingVectorNoNA(memFactory, maxElements) case DoubleColumn => bv.DoubleVector.appendingVectorNoNA(memFactory, maxElements) case TimestampColumn => bv.LongBinaryVector.timestampVector(memFactory, maxElements) - case StringColumn => bv.UTF8Vector.appendingVector(memFactory, maxElements, MaxUTF8VectorSize) - case HistogramColumn => bv.HistogramVector.appendingColumnar(memFactory, MaxHistogramBuckets, maxElements) + case StringColumn => bv.UTF8Vector.appendingVector(memFactory, maxElements, config.maxBlobBufferSize) + case HistogramColumn => bv.HistogramVector.appending(memFactory, config.maxBlobBufferSize) case other: Column.ColumnType => ??? } }.toArray + } } \ No newline at end of file diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala index a67c5c3bba..c50005d42f 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala @@ -261,8 +261,7 @@ class TimeSeriesShard(val dataset: Dataset, private val partKeyBuilder = new RecordBuilder(MemFactory.onHeapFactory, dataset.partKeySchema, reuseOneContainer = true) private val partKeyArray = partKeyBuilder.allContainers.head.base.asInstanceOf[Array[Byte]] - private val bufferPool = new WriteBufferPool(bufferMemoryManager, dataset, storeConfig.maxChunksSize, - storeConfig.allocStepSize) + private val bufferPool = new WriteBufferPool(bufferMemoryManager, dataset, storeConfig) private final val partitionGroups = Array.fill(numGroups)(new EWAHCompressedBitmap) private final val activelyIngesting = new EWAHCompressedBitmap diff --git a/core/src/main/scala/filodb.core/memstore/WriteBufferPool.scala b/core/src/main/scala/filodb.core/memstore/WriteBufferPool.scala index 68178d7df7..67ce93b065 100644 --- a/core/src/main/scala/filodb.core/memstore/WriteBufferPool.scala +++ b/core/src/main/scala/filodb.core/memstore/WriteBufferPool.scala @@ -4,15 +4,10 @@ import com.typesafe.scalalogging.StrictLogging import scalaxy.loops._ import filodb.core.metadata.Dataset -import filodb.core.store.ChunkSetInfo +import filodb.core.store.{ChunkSetInfo, StoreConfig} import filodb.memory.BinaryRegion.NativePointer import filodb.memory.MemFactory -object WriteBufferPool { - // The number of partition write buffers to allocate at one time - val DefaultAllocStepSize = 1000 -} - /** * A WriteBufferPool pre-allocates/creates a pool of WriteBuffers for sharing amongst many MemStore Partitions. * For efficiency it creates a whole set of BinaryAppendableVectors for all columns, so that @@ -25,25 +20,22 @@ object WriteBufferPool { * 1. Partition gets data - obtains new set of initial buffers * 2. End of flush() - original buffers, now encoded, are released, reset, and can be made available to others * - * @param maxChunkSize the max size of the write buffer in elements. - * @param allocationStepSize the number of partition write buffers to allocate at a time. - * Smaller=better use of memory; Bigger=more efficient allocation + * @param storeConf the StoreConfig containing parameters for configuring write buffers, etc. * * TODO: Use MemoryManager etc. and allocate memory from a fixed block instead of specifying max # partitions */ class WriteBufferPool(memFactory: MemFactory, val dataset: Dataset, - maxChunkSize: Int, - allocationStepSize: Int = WriteBufferPool.DefaultAllocStepSize) extends StrictLogging { + storeConf: StoreConfig) extends StrictLogging { import TimeSeriesPartition._ val queue = new collection.mutable.Queue[(NativePointer, AppenderArray)] private def allocateBuffers(): Unit = { - logger.debug(s"Allocating $allocationStepSize WriteBuffers....") + logger.debug(s"Allocating ${storeConf.allocStepSize} WriteBuffers....") // Fill queue up - (0 until allocationStepSize).foreach { n => - val builders = MemStore.getAppendables(memFactory, dataset, maxChunkSize) + (0 until storeConf.allocStepSize).foreach { n => + val builders = MemStore.getAppendables(memFactory, dataset, storeConf) val info = ChunkSetInfo(memFactory, dataset, 0, 0, Long.MinValue, Long.MaxValue) // Point vectors in chunkset metadata to builders addresses for { colNo <- 0 until dataset.numDataColumns optimized } { diff --git a/core/src/main/scala/filodb.core/store/IngestionConfig.scala b/core/src/main/scala/filodb.core/store/IngestionConfig.scala index d7983dfb34..dac1a3a7cd 100644 --- a/core/src/main/scala/filodb.core/store/IngestionConfig.scala +++ b/core/src/main/scala/filodb.core/store/IngestionConfig.scala @@ -13,6 +13,8 @@ final case class StoreConfig(flushInterval: FiniteDuration, diskTTLSeconds: Int, demandPagedRetentionPeriod: FiniteDuration, maxChunksSize: Int, + // Max write buffer size for Histograms, UTF8Strings, other blobs + maxBlobBufferSize: Int, // Number of bytes to allocate to chunk storage in each shard shardMemSize: Long, // Number of bytes to allocate to ingestion write buffers per shard @@ -36,6 +38,7 @@ final case class StoreConfig(flushInterval: FiniteDuration, "disk-time-to-live" -> (diskTTLSeconds + "s"), "demand-paged-chunk-retention-period" -> (demandPagedRetentionPeriod.toSeconds + "s"), "max-chunks-size" -> maxChunksSize, + "max-blob-buffer-size" -> maxBlobBufferSize, "shard-mem-size" -> shardMemSize, "ingestion-buffer-mem-size" -> ingestionBufferMemSize, "buffer-alloc-step-size" -> allocStepSize, @@ -60,7 +63,8 @@ object StoreConfig { val defaults = ConfigFactory.parseString(""" |disk-time-to-live = 3 days |demand-paged-chunk-retention-period = 72 hours - |max-chunks-size = 500 + |max-chunks-size = 400 + |max-blob-buffer-size = 15000 |ingestion-buffer-mem-size = 10M |buffer-alloc-step-size = 1000 |num-partitions-to-evict = 1000 @@ -81,6 +85,7 @@ object StoreConfig { config.as[FiniteDuration]("disk-time-to-live").toSeconds.toInt, config.as[FiniteDuration]("demand-paged-chunk-retention-period"), config.getInt("max-chunks-size"), + config.getInt("max-blob-buffer-size"), config.getMemorySize("shard-mem-size").toBytes, config.getMemorySize("ingestion-buffer-mem-size").toBytes, config.getInt("buffer-alloc-step-size"), diff --git a/core/src/test/scala/filodb.core/TestData.scala b/core/src/test/scala/filodb.core/TestData.scala index bd8f4a25c7..a4771eba1d 100644 --- a/core/src/test/scala/filodb.core/TestData.scala +++ b/core/src/test/scala/filodb.core/TestData.scala @@ -38,6 +38,7 @@ object TestData { val sourceConf = ConfigFactory.parseString(""" store { max-chunks-size = 100 + buffer-alloc-step-size = 50 demand-paged-chunk-retention-period = 10 hours shard-mem-size = 50MB groups-per-shard = 4 diff --git a/core/src/test/scala/filodb.core/downsample/ShardDownsamplerSpec.scala b/core/src/test/scala/filodb.core/downsample/ShardDownsamplerSpec.scala index 20f8dedfae..885f6c4b74 100644 --- a/core/src/test/scala/filodb.core/downsample/ShardDownsamplerSpec.scala +++ b/core/src/test/scala/filodb.core/downsample/ShardDownsamplerSpec.scala @@ -17,8 +17,6 @@ import filodb.memory.format.{TupleRowReader, ZeroCopyUTF8String} // scalastyle:off null class ShardDownsamplerSpec extends FunSpec with Matchers with BeforeAndAfterAll { - val maxChunkSize = 200 - val promDataset = Dataset.make("custom1", Seq("someStr:string", "tags:map"), Seq("timestamp:ts", "value:double"), @@ -38,7 +36,8 @@ class ShardDownsamplerSpec extends FunSpec with Matchers with BeforeAndAfterAll protected val ingestBlockHolder = new BlockMemFactory(blockStore, None, promDataset.blockMetaSize, true) - protected val tsBufferPool = new WriteBufferPool(TestData.nativeMem, promDataset, maxChunkSize, 100) + val storeConf = TestData.storeConf.copy(maxChunksSize = 200) + protected val tsBufferPool = new WriteBufferPool(TestData.nativeMem, promDataset, storeConf) override def afterAll(): Unit = { blockStore.releaseBlocks() diff --git a/core/src/test/scala/filodb.core/memstore/PartitionSetSpec.scala b/core/src/test/scala/filodb.core/memstore/PartitionSetSpec.scala index f214889456..66ee3b5a06 100644 --- a/core/src/test/scala/filodb.core/memstore/PartitionSetSpec.scala +++ b/core/src/test/scala/filodb.core/memstore/PartitionSetSpec.scala @@ -31,8 +31,7 @@ class PartitionSetSpec extends MemFactoryCleanupTest with ScalaFutures { private val blockStore = new PageAlignedBlockManager(100 * 1024 * 1024, new MemoryStats(Map("test"-> "test")), reclaimer, 1) - val maxChunkSize = 100 - protected val bufferPool = new WriteBufferPool(memFactory, dataset2, maxChunkSize, 50) + protected val bufferPool = new WriteBufferPool(memFactory, dataset2, TestData.storeConf) private val ingestBlockHolder = new BlockMemFactory(blockStore, None, dataset2.blockMetaSize, true) val builder = new RecordBuilder(memFactory, dataset2.ingestionSchema) diff --git a/core/src/test/scala/filodb.core/memstore/TimeSeriesPartitionSpec.scala b/core/src/test/scala/filodb.core/memstore/TimeSeriesPartitionSpec.scala index 0caed40822..951b5a4e0f 100644 --- a/core/src/test/scala/filodb.core/memstore/TimeSeriesPartitionSpec.scala +++ b/core/src/test/scala/filodb.core/memstore/TimeSeriesPartitionSpec.scala @@ -19,8 +19,8 @@ object TimeSeriesPartitionSpec { val memFactory = new NativeMemoryManager(10 * 1024 * 1024) - val maxChunkSize = 100 - protected val myBufferPool = new WriteBufferPool(memFactory, dataset1, maxChunkSize, 50) + val maxChunkSize = TestData.storeConf.maxChunksSize + protected val myBufferPool = new WriteBufferPool(memFactory, dataset1, TestData.storeConf) def makePart(partNo: Int, dataset: Dataset, partKey: NativePointer = defaultPartKey, diff --git a/doc/compression.md b/doc/compression.md index 3f9e4c6a3e..e4468a21e8 100644 --- a/doc/compression.md +++ b/doc/compression.md @@ -90,7 +90,13 @@ Or, if the above was viewed in a little-endian system as a 32-bit int, then the ## Histograms -FiloDB supports first class histograms as HistogramColumns in schemas. This means histograms are ingested as single entities and kept together as a single time series. Compressed histograms (using NibblePacking) are stored in write buffers and may be compressed further. +FiloDB supports first class histograms as HistogramColumns in schemas. This means histograms are ingested as single entities and kept together as a single time series. Histograms are required to have increasing bucket values; that is, the value in each bucket represents the total count of all buckets below that bucket as well -- the buckets are cumulative. This is based on the histogram bucket scheme used in Prometheus. + +Currently, incoming histograms which are cumulative are encoded on the wire using delta encoding of their integer values using the NibblePacking scheme described above. This leads to pretty efficient encoding of both busy/active histograms and inactive ones. +- The delta-encoded NibblePack'ed histograms save save in the BinaryRecord ingestion format. For 64 buckets, for example, this format saves 50x space compared to the traditional Prometheus data model. +- The encoded histograms are stored as is in write buffers. This allows us to save a huge amount of space there as well. +- Since encoded histograms are variable-length, we facilitate fast lookup of histograms within a HistogramVector by grouping histograms into "sections". Sections have headers to make it easy to skip over entire sections. +- Currently, the delta-encoded histogram format also serves as the compressed vector format, ie there is no further compression after arrival. Please see [BinaryHistogram](../memory/src/main/scala/filodb.memory/format/vectors/HistogramVector.scala) for more details about the on-the-wire / BinaryRecord format used for histograms. diff --git a/http/src/test/scala/filodb/http/ClusterApiRouteSpec.scala b/http/src/test/scala/filodb/http/ClusterApiRouteSpec.scala index c88f36f6d0..f0e054b0e7 100644 --- a/http/src/test/scala/filodb/http/ClusterApiRouteSpec.scala +++ b/http/src/test/scala/filodb/http/ClusterApiRouteSpec.scala @@ -43,12 +43,11 @@ class ClusterApiRouteSpec extends FunSpec with ScalatestRouteTest with AsyncTest before { probe.send(cluster.coordinatorActor, NodeProtocol.ResetState) probe.expectMsg(NodeProtocol.StateReset) + // Note: at this point all ingestor actors are shut down cluster.metaStore.clearAllData().futureValue cluster.metaStore.newDataset(dataset6).futureValue shouldEqual Success probe.send(clusterProxy, NodeProtocol.ResetState) probe.expectMsg(NodeProtocol.StateReset) - // Give enough time for old ingestor/query actors to die - Thread sleep 500 } describe("get datasets route") { diff --git a/memory/src/main/scala/filodb.memory/format/BinaryVector.scala b/memory/src/main/scala/filodb.memory/format/BinaryVector.scala index bde32c90be..69f5ef4cb8 100644 --- a/memory/src/main/scala/filodb.memory/format/BinaryVector.scala +++ b/memory/src/main/scala/filodb.memory/format/BinaryVector.scala @@ -45,6 +45,7 @@ object BinaryVector { case Classes.Long => (b => vectors.LongBinaryVector(b)) case Classes.Double => (b => vectors.DoubleVector(b)) case Classes.UTF8 => (b => vectors.UTF8Vector(b)) + case Classes.Histogram => (b => vectors.HistogramVector(b)) } type PtrToDataReader = PartialFunction[Class[_], BinaryVectorPtr => VectorDataReader] @@ -170,7 +171,8 @@ sealed trait AddResponse case object Ack extends AddResponse final case class VectorTooSmall(bytesNeeded: Int, bytesHave: Int) extends AddResponse case object ItemTooLarge extends AddResponse -case object BucketSchemaMismatch extends AddResponse +case object BucketSchemaMismatch extends AddResponse // Trying to add a histogram with nonmatching schema +case object InvalidHistogram extends AddResponse /** * A BinaryVector that you can append to. Has some notion of a maximum size (max # of items or bytes) diff --git a/memory/src/main/scala/filodb.memory/format/Ptr.scala b/memory/src/main/scala/filodb.memory/format/Ptr.scala index 5c8c2c8354..52e7fd40b1 100644 --- a/memory/src/main/scala/filodb.memory/format/Ptr.scala +++ b/memory/src/main/scala/filodb.memory/format/Ptr.scala @@ -16,6 +16,40 @@ object Ptr { //scalastyle:off method.name final def +(offset: Int): U8 = U8(addr + offset) - final def get: Byte = UnsafeUtils.getByte(addr) + final def getU8: Int = UnsafeUtils.getByte(addr) & 0x00ff + + final def asU16: U16 = U16(addr) + final def asI32: I32 = I32(addr) + final def asMut: MutU8 = MutU8(addr) + } + + final case class MutU8(addr: Long) extends AnyVal { + final def set(num: Int): Unit = UnsafeUtils.setByte(addr, num.toByte) + } + + final case class U16(addr: Long) extends AnyVal { + final def add(offset: Int): U16 = U16(addr + offset * 2) + final def +(offset: Int): U16 = U16(addr + offset * 2) + + final def getU16: Int = UnsafeUtils.getShort(addr) & 0x00ffff + + final def asMut: MutU16 = MutU16(addr) + } + + final case class MutU16(addr: Long) extends AnyVal { + final def set(num: Int): Unit = UnsafeUtils.setShort(addr, num.toShort) + } + + final case class I32(addr: Long) extends AnyVal { + final def add(offset: Int): I32 = I32(addr + offset * 4) + final def +(offset: Int): I32 = I32(addr + offset * 4) + + final def getI32: Int = UnsafeUtils.getInt(addr) + + final def asMut: MutI32 = MutI32(addr) + } + + final case class MutI32(addr: Long) extends AnyVal { + final def set(num: Int): Unit = UnsafeUtils.setInt(addr, num) } } \ No newline at end of file diff --git a/memory/src/main/scala/filodb.memory/format/Section.scala b/memory/src/main/scala/filodb.memory/format/Section.scala new file mode 100644 index 0000000000..6145b01c8b --- /dev/null +++ b/memory/src/main/scala/filodb.memory/format/Section.scala @@ -0,0 +1,91 @@ +package filodb.memory.format + +/** + * A Section is a subdivision of a BinaryVector, typically used for variable-length data. It contains a + * fixed header with length and # elements, which lets cursors skip quickly over sections. + * + * The design of the data is such that a single writer updates fields but multiple readers can read. + * Thus we have to be careful about the order in which updates are written. + * Basically, we want the data in all fields to be consistent. + */ +final case class Section private(addr: Long) extends AnyVal { + // not including length bytes + final def sectionNumBytes: Int = Ptr.U16(addr).getU16 + + final def numElements: Int = Ptr.U8(addr).add(2).getU8 + + // Ptr to first record of section + final def firstElem: Ptr.U8 = Ptr.U8(addr) + 4 + + // The address at the end of this section's elements, based on current num bytes + final def endAddr: Ptr.U8 = Ptr.U8(addr).add(4).add(sectionNumBytes) + + final def isComplete: Boolean = numElements > 0 + + /** + * Updates the number of bytes and elements atomically. + * Checks that new values make sense. + */ + final def update(addedBytes: Int, addedElements: Int): Unit = { + require(addedBytes > 0 && addedElements > 0) + val newNumElements = numElements + addedElements + val newNumBytes = sectionNumBytes + addedBytes + require(newNumElements <= 255 && newNumBytes <= 65535) + Ptr.I32(addr).asMut.set(newNumBytes | (newNumElements << 16)) + } + + final def setNumElements(num: Int): Unit = { + require(num >= 0 && num <= 255) + Ptr.U8(addr).add(2).asMut.set(num) + } +} + +object Section { + def fromPtr(addr: Ptr.U8): Section = Section(addr.addr) + + def init(sectionAddr: Ptr.U8): Section = { + val newSect = Section(sectionAddr.addr) + newSect.setNumElements(0) + sectionAddr.asU16.asMut.set(0) + newSect + } +} + +/** + * A writer which manages sections as blobs and elements are added to it, rolling over to a new section as needed. + */ +trait SectionWriter { + // Max # of elements per section, should be no more than 255. Usually 64? + def maxElementsPerSection: Int + + // Call to initialize the section writer with the address of the first section and how many bytes left + def initSectionWriter(firstSectionAddr: Ptr.U8, remainingBytes: Int): Unit = { + curSection = Section.init(firstSectionAddr) + bytesLeft = remainingBytes - 4 // account for initial section header bytes + } + + var curSection: Section = _ + var bytesLeft: Int = 0 + + // Appends a blob, writing a 2-byte length prefix before it. + protected def appendBlob(base: Any, offset: Long, numBytes: Int): AddResponse = { + // Check remaining length/space. A section must be less than 2^16 bytes long. Create new section if needed + val newNumBytes = curSection.sectionNumBytes + numBytes + if (curSection.numElements >= maxElementsPerSection || newNumBytes >= 65536) { + if (bytesLeft >= (4 + numBytes)) { + curSection = Section.init(curSection.endAddr) + bytesLeft -= 4 + } else return VectorTooSmall(4 + numBytes, bytesLeft) + } + + // Copy bytes to end address, update variables + if (bytesLeft >= (numBytes + 2)) { + val writeAddr = curSection.endAddr + writeAddr.asU16.asMut.set(numBytes) + UnsafeUtils.unsafe.copyMemory(base, offset, UnsafeUtils.ZeroPointer, (writeAddr + 2).addr, numBytes) + bytesLeft -= (numBytes + 2) + curSection.update(numBytes + 2, 1) + Ack + } else VectorTooSmall(numBytes + 2, bytesLeft) + } +} \ No newline at end of file diff --git a/memory/src/main/scala/filodb.memory/format/WireFormat.scala b/memory/src/main/scala/filodb.memory/format/WireFormat.scala index 88ec43d10a..e56ea10cc9 100644 --- a/memory/src/main/scala/filodb.memory/format/WireFormat.scala +++ b/memory/src/main/scala/filodb.memory/format/WireFormat.scala @@ -6,13 +6,15 @@ package filodb.memory.format */ object WireFormat { val VECTORTYPE_EMPTY = 0x01 - val VECTORTYPE_SIMPLE = 0x02 - val VECTORTYPE_DICT = 0x03 - val VECTORTYPE_CONST = 0x04 - val VECTORTYPE_DIFF = 0x05 + // Deprecated vector types + // val VECTORTYPE_SIMPLE = 0x02 + // val VECTORTYPE_DICT = 0x03 + // val VECTORTYPE_CONST = 0x04 + // val VECTORTYPE_DIFF = 0x05 val VECTORTYPE_BINSIMPLE = 0x06 val VECTORTYPE_BINDICT = 0x07 val VECTORTYPE_DELTA2 = 0x08 // Delta-delta encoded + val VECTORTYPE_HISTOGRAM = 0x09 def majorVectorType(headerBytes: Int): Int = headerBytes & 0x00ff def emptyVectorLen(headerBytes: Int): Int = { @@ -30,6 +32,8 @@ object WireFormat { val SUBTYPE_INT = 0x07 // Int gets special type because Longs and Doubles may be encoded as Int val SUBTYPE_INT_NOMASK = 0x08 + val SUBTYPE_H_SIMPLE = 0x10 // Histograms, stored as is + def vectorSubType(headerBytes: Int): Int = (headerBytes & 0x00ff00) >> 8 def majorAndSubType(headerBytes: Int): Int = headerBytes & 0x0ffff 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 1e89212cb5..07d92c8b8e 100644 --- a/memory/src/main/scala/filodb.memory/format/vectors/Histogram.scala +++ b/memory/src/main/scala/filodb.memory/format/vectors/Histogram.scala @@ -177,6 +177,8 @@ final case class MutableHistogram(buckets: HistogramBuckets, values: Array[Doubl */ final def add(other: HistogramWithBuckets): Unit = if (buckets == other.buckets) { + // If it was NaN before, reset to 0 to sum another hist + if (values(0).isNaN) java.util.Arrays.fill(values, 0.0) for { b <- 0 until numBuckets optimized } { values(b) += other.bucketValue(b) } @@ -187,7 +189,7 @@ final case class MutableHistogram(buckets: HistogramBuckets, values: Array[Doubl object MutableHistogram { def empty(buckets: HistogramBuckets): MutableHistogram = - MutableHistogram(buckets, new Array[Double](buckets.numBuckets)) + MutableHistogram(buckets, Array.fill(buckets.numBuckets)(Double.NaN)) } /** @@ -279,7 +281,7 @@ final case class GeometricBuckets(firstBucket: Double, final def serialize(buf: MutableDirectBuffer, pos: Int): Int = { require(numBuckets < 65536, s"Too many buckets: $numBuckets") val numBucketsPos = pos + 2 - buf.putShort(pos, 2 + 8 + 8) + buf.putShort(pos, (2 + 8 + 8).toShort) buf.putShort(numBucketsPos, numBuckets.toShort, LITTLE_ENDIAN) buf.putDouble(numBucketsPos + OffsetBucketDetails, firstBucket, LITTLE_ENDIAN) buf.putDouble(numBucketsPos + OffsetBucketDetails + 8, multiplier, LITTLE_ENDIAN) diff --git a/memory/src/main/scala/filodb.memory/format/vectors/HistogramCompressor.scala b/memory/src/main/scala/filodb.memory/format/vectors/HistogramCompressor.scala index 9c9bd7570b..e916733b20 100644 --- a/memory/src/main/scala/filodb.memory/format/vectors/HistogramCompressor.scala +++ b/memory/src/main/scala/filodb.memory/format/vectors/HistogramCompressor.scala @@ -2,27 +2,29 @@ package filodb.memory.format.vectors import scala.io.Source -import org.agrona.concurrent.UnsafeBuffer +import org.agrona.ExpandableArrayBuffer import filodb.memory.NativeMemoryManager +import filodb.memory.format._ //scalastyle:off // Input is a file with one line per histogram, bucket values are comma separated // This app is designed to measure histogram compression ratios based on real world histogram data object HistogramCompressor extends App { if (args.length < 1) { - println("Usage: sbt memory/run ") + println("Usage: sbt memory/run ") println("Tests chunk compression, not runtime or performance, against real-world increasing bucket histogram files") sys.exit(0) } val inputFile = args(0) - val chunkLength = if (args.length > 1) args(1).toInt else 300 - val numChunks = 20 + var numChunks = 0 + val numSamples = if (args.length > 1) args(1).toInt else 6000 val memFactory = new NativeMemoryManager(500 * 1024 * 1024) - val inputBuffer = new UnsafeBuffer(new Array[Byte](8192)) + val inputBuffer = new ExpandableArrayBuffer(4096) - val appender = HistogramVector.appendingColumnar(memFactory, 64, chunkLength) + val maxBytes = 60 * 300 // Maximum allowable histogram writebuffer size + val appender = HistogramVector.appending(memFactory, maxBytes) val bucketDef = HistogramBuckets.binaryBuckets64 var binHistBytesSum = 0 @@ -33,33 +35,51 @@ object HistogramCompressor extends App { var writeBufferTotal = 0 Source.fromFile(inputFile).getLines - .take(numChunks * chunkLength) - .grouped(chunkLength).foreach { chunkLines => + .take(numSamples) + .foreach { line => // Ingest each histogram, parse and create a BinaryHistogram, then ingest into our histogram column - chunkLines.foreach { line => - val buckets = line.split(",").map(_.trim.toLong) - val histSize = BinaryHistogram.writeNonIncreasing(bucketDef, buckets, inputBuffer) - numRecords += 1 - binHistBytesMax = Math.max(binHistBytesMax, histSize) - binHistBytesSum += histSize + val buckets = line.split(",").map(_.trim.toLong) + val histSize = BinaryHistogram.writeDelta(bucketDef, buckets, inputBuffer) + numRecords += 1 + binHistBytesMax = Math.max(binHistBytesMax, histSize) + binHistBytesSum += histSize - appender.addData(inputBuffer) - } + appender.addData(inputBuffer) match { + case Ack => // data added, no problem + case VectorTooSmall(_, _) => // not enough space. Encode, aggregate, and add to a new appender + // Optimize and get optimized size, dump out, aggregate + val writeBufSize = appender.numBytes + val optimized = appender.optimize(memFactory) + val encodedSize = BinaryVector.totalBytes(optimized) + println(s" WriteBuffer size: ${writeBufSize}\t\tEncoded size: $encodedSize") + encodedTotal += encodedSize + writeBufferTotal += writeBufSize + numChunks += 1 - // Optimize and get optimized size, dump out, aggregate - val writeBufSize = HistogramVector.columnarTotalSize(appender.addr) - val optimized = appender.optimize(memFactory) - val encodedSize = HistogramVector.columnarTotalSize(optimized) - println(s" WriteBuffer size: ${writeBufSize}\t\tEncoded size: $encodedSize") - encodedTotal += encodedSize - writeBufferTotal += writeBufSize + appender.reset() + // Add back the input that did not fit into appender again + appender.addData(inputBuffer) + case other => + println(s"Warning: response $other from appender.addData") + } } + val samplesEncoded = numSamples - appender.length + + // Encode final chunk? Or just forget it, because it will mess up size statistics? + // Dump out overall aggregates val avgEncoded = encodedTotal.toDouble / numChunks val avgWriteBuf = writeBufferTotal.toDouble / numChunks - println(s"Average encoded chunk size: $avgEncoded") - println(s"Average write buffer size: $avgWriteBuf") + val histPerChunk = samplesEncoded.toDouble / numChunks + println(s"Total number of chunks: $numChunks") + println(s"Samples encoded: $samplesEncoded") + println(s"Histograms per chunk: $histPerChunk") + + val normAvgEncoded = avgEncoded * 300 / histPerChunk + val normAvgWriteBuf = avgWriteBuf * 300 / histPerChunk + println(s"Average encoded chunk size: $avgEncoded (normalized to 300 histograms: $normAvgEncoded)") + println(s"Average write buffer size: $avgWriteBuf (normalized to 300 histograms: $normAvgWriteBuf)") println(s"Compression ratio: ${avgWriteBuf / avgEncoded}") println(s"Average binHistogram size: ${binHistBytesSum / numRecords.toDouble}") println(s"Max binHistogram size: $binHistBytesMax bytes") 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 5e145e10ca..de12fcd7c9 100644 --- a/memory/src/main/scala/filodb.memory/format/vectors/HistogramVector.scala +++ b/memory/src/main/scala/filodb.memory/format/vectors/HistogramVector.scala @@ -1,5 +1,7 @@ package filodb.memory.format.vectors +import java.nio.ByteBuffer + import com.typesafe.scalalogging.StrictLogging import debox.Buffer import org.agrona.{DirectBuffer, ExpandableArrayBuffer, MutableDirectBuffer} @@ -9,7 +11,6 @@ import scalaxy.loops._ import filodb.memory.{BinaryRegion, MemFactory} import filodb.memory.format._ import filodb.memory.format.BinaryVector.BinaryVectorPtr -import filodb.memory.format.Encodings._ /** * BinaryHistogram is the binary format for a histogram binary blob included in BinaryRecords and sent over the wire. @@ -85,6 +86,9 @@ object BinaryHistogram extends StrictLogging { val HistFormat_Geometric_Delta = 0x03.toByte val HistFormat_Geometric1_Delta = 0x04.toByte + def isValidFormatCode(code: Byte): Boolean = + (code == HistFormat_Null) || (code == HistFormat_Geometric1_Delta) || (code == HistFormat_Geometric_Delta) + /** * Writes binary histogram with geometric bucket definition and data which is non-increasing, but will be * decoded as increasing. Intended only for specific use cases when the source histogram are non increasing @@ -94,6 +98,7 @@ object BinaryHistogram extends StrictLogging { * @return the number of bytes written, including the length prefix */ def writeNonIncreasing(buckets: GeometricBuckets, values: Array[Long], buf: MutableDirectBuffer): Int = { + require(buckets.numBuckets == values.size, s"Values array size of ${values.size} != ${buckets.numBuckets}") val formatCode = if (buckets.minusOne) HistFormat_Geometric1_Delta else HistFormat_Geometric_Delta buf.putByte(2, formatCode) @@ -117,6 +122,7 @@ object BinaryHistogram extends StrictLogging { * @return the number of bytes written, including the length prefix */ def writeDelta(buckets: GeometricBuckets, values: Array[Long], buf: MutableDirectBuffer): Int = { + require(buckets.numBuckets == values.size, s"Values array size of ${values.size} != ${buckets.numBuckets}") val formatCode = if (buckets.minusOne) HistFormat_Geometric1_Delta else HistFormat_Geometric_Delta buf.putByte(2, formatCode) @@ -139,179 +145,136 @@ object HistogramVector { val OffsetNumBuckets = 11 // After the bucket area are regions for storing the counter values or pointers to them - final def getNumBuckets(addr: BinaryVectorPtr): Int = UnsafeUtils.getShort(addr + OffsetNumBuckets).toInt + final def getNumBuckets(addr: Ptr.U8): Int = addr.add(OffsetNumBuckets).asU16.getU16 - final def getNumHistograms(addr: BinaryVectorPtr): Int = UnsafeUtils.getShort(addr + OffsetNumHistograms).toInt - final def resetNumHistograms(addr: BinaryVectorPtr): Unit = - UnsafeUtils.setShort(addr + OffsetNumHistograms, 0) - final def incrNumHistograms(addr: BinaryVectorPtr): Unit = - UnsafeUtils.setShort(addr + OffsetNumHistograms, (getNumHistograms(addr) + 1).toShort) + final def getNumHistograms(addr: Ptr.U8): Int = addr.add(OffsetNumHistograms).asU16.getU16 + final def resetNumHistograms(addr: Ptr.U8): Unit = addr.add(OffsetNumHistograms).asU16.asMut.set(0) + final def incrNumHistograms(addr: Ptr.U8): Unit = + addr.add(OffsetNumHistograms).asU16.asMut.set(getNumHistograms(addr) + 1) - final def formatCode(addr: BinaryVectorPtr): Byte = UnsafeUtils.getByte(addr + OffsetFormatCode) - final def afterBucketDefAddr(addr: BinaryVectorPtr): Ptr.U8 = - Ptr.U8(addr) + OffsetBucketDef + bucketDefNumBytes(addr) - final def bucketDefNumBytes(addr: BinaryVectorPtr): Int = UnsafeUtils.getShort(addr + OffsetBucketDefSize).toInt - final def bucketDefAddr(addr: BinaryVectorPtr): Ptr.U8 = Ptr.U8(addr) + OffsetBucketDef + // Note: the format code defines bucket definition format + format of each individual compressed histogram + final def formatCode(addr: Ptr.U8): Byte = addr.add(OffsetFormatCode).getU8.toByte + final def afterBucketDefAddr(addr: Ptr.U8): Ptr.U8 = addr + OffsetBucketDef + bucketDefNumBytes(addr) + final def bucketDefNumBytes(addr: Ptr.U8): Int = addr.add(OffsetBucketDefSize).asU16.getU16 + final def bucketDefAddr(addr: Ptr.U8): Ptr.U8 = addr + OffsetBucketDef // Matches the bucket definition whose # bytes is at (base, offset) - final def matchBucketDef(hist: BinaryHistogram.BinHistogram, addr: BinaryVectorPtr): Boolean = + final def matchBucketDef(hist: BinaryHistogram.BinHistogram, addr: Ptr.U8): Boolean = (hist.formatCode == formatCode(addr)) && (hist.bucketDefNumBytes == bucketDefNumBytes(addr)) && { - UnsafeUtils.equate(UnsafeUtils.ZeroPointer, addr + OffsetBucketDef, hist.buf.byteArray, hist.bucketDefOffset, + UnsafeUtils.equate(UnsafeUtils.ZeroPointer, bucketDefAddr(addr).addr, hist.buf.byteArray, hist.bucketDefOffset, hist.bucketDefNumBytes) } - // Columnar HistogramVectors composed of multiple vectors, this calculates total used size - def columnarTotalSize(addr: BinaryVectorPtr): Int = { - val bucketAddrPtr = afterBucketDefAddr(addr) - val headerBytes = UnsafeUtils.getInt(addr) - headerBytes + (0 until getNumBuckets(addr)).map { b => - val bucketVectorAddr = UnsafeUtils.getLong((bucketAddrPtr + 8*b).addr) - UnsafeUtils.getInt(bucketVectorAddr) + 4 - }.sum + def appending(factory: MemFactory, maxBytes: Int): AppendableHistogramVector = { + val addr = factory.allocateOffheap(maxBytes) + new AppendableHistogramVector(factory, Ptr.U8(addr), maxBytes) } - val ReservedBucketDefSize = 256 - def appendingColumnar(factory: MemFactory, numBuckets: Int, maxItems: Int): ColumnarAppendableHistogramVector = { - // Really just an estimate. TODO: if we really go with columnar, make it more accurate - val neededBytes = OffsetBucketDef + ReservedBucketDefSize + 8 * numBuckets - val addr = factory.allocateOffheap(neededBytes) - new ColumnarAppendableHistogramVector(factory, addr, maxItems) - } + def apply(buffer: ByteBuffer): HistogramReader = apply(UnsafeUtils.addressFromDirectBuffer(buffer)) - def apply(p: BinaryVectorPtr): HistogramReader = - new ColumnarHistogramReader(p) + import WireFormat._ + + def apply(p: BinaryVectorPtr): HistogramReader = BinaryVector.vectorType(p) match { + case x if x == WireFormat(VECTORTYPE_HISTOGRAM, SUBTYPE_H_SIMPLE) => new RowHistogramReader(Ptr.U8(p)) + } } /** - * A HistogramVector appender composed of individual primitive columns. - * Just a POC to get started quickly and as a reference. + * A HistogramVector appender storing compressed histogram values for less storage space. + * This is a Section-based vector - sections of up to 64 histograms are stored at a time. + * It stores histograms up to a maximum allowed size (since histograms are variable length) * Note that the bucket schema is not set until getting the first item. - * After the bucket definition: - * An array [u64] of native pointers to the individual columns - * - * TODO: initialize num bytes and vector type stuff * * Read/Write/Lock semantics: everything is gated by the number of elements. * When it is 0, nothing is initialized so the reader guards against that. * When it is > 0, then all structures are initialized. */ -class ColumnarAppendableHistogramVector(factory: MemFactory, - val addr: BinaryVectorPtr, - maxItems: Int) extends BinaryAppendableVector[UnsafeBuffer] { +class AppendableHistogramVector(factory: MemFactory, + vectPtr: Ptr.U8, + val maxBytes: Int) extends BinaryAppendableVector[DirectBuffer] with SectionWriter { import HistogramVector._ import BinaryHistogram._ - resetNumHistograms(addr) - private var bucketAppenders: Option[Array[BinaryAppendableVector[Long]]] = None + // Initialize header + BinaryVector.writeMajorAndSubType(addr, WireFormat.VECTORTYPE_HISTOGRAM, + WireFormat.SUBTYPE_H_SIMPLE) + reset() + + final def addr: BinaryVectorPtr = vectPtr.addr + final def maxElementsPerSection: Int = 64 val dispose = () => { - // first, free memory from each appender - bucketAppenders.foreach(_.foreach(_.dispose())) // free our own memory factory.freeMemory(addr) } - final def numBytes: Int = UnsafeUtils.getInt(addr) + 4 - final def maxBytes: Int = numBytes - final def length: Int = getNumHistograms(addr) + final def numBytes: Int = vectPtr.asI32.getI32 + 4 + final def length: Int = getNumHistograms(vectPtr) final def isAvailable(index: Int): Boolean = true final def isAllNA: Boolean = (length == 0) final def noNAs: Boolean = (length > 0) - private val valueBuf = new UnsafeBuffer(Array.empty[Byte]) + private def setNumBytes(len: Int): Unit = { + require(len >= 0) + vectPtr.asI32.asMut.set(len) + } - // NOTE: to eliminate allocations, re-use the UnsafeBuffer and keep passing the same instance to addData - final def addData(buf: UnsafeBuffer): AddResponse = { - val numItems = getNumHistograms(addr) + // NOTE: to eliminate allocations, re-use the DirectBuffer and keep passing the same instance to addData + final def addData(buf: DirectBuffer): AddResponse = { val h = BinHistogram(buf) + // Validate it's a valid bin histogram + if (buf.capacity < 5 || !isValidFormatCode(h.formatCode) || + h.formatCode == HistFormat_Null) { + return InvalidHistogram + } + if (h.bucketDefNumBytes > h.totalLength) return InvalidHistogram + + val numItems = getNumHistograms(vectPtr) val numBuckets = h.numBuckets if (numItems == 0) { // Copy the bucket definition and set the bucket def size UnsafeUtils.unsafe.copyMemory(buf.byteArray, h.bucketDefOffset, - UnsafeUtils.ZeroPointer, bucketDefAddr(addr).addr, h.bucketDefNumBytes) + UnsafeUtils.ZeroPointer, bucketDefAddr(vectPtr).addr, h.bucketDefNumBytes) UnsafeUtils.setShort(addr + OffsetBucketDefSize, h.bucketDefNumBytes.toShort) UnsafeUtils.setByte(addr + OffsetFormatCode, h.formatCode) - // initialize the buckets - initBuckets(numBuckets) - } else if (numItems >= maxItems) { - return VectorTooSmall(0, 0) + // Initialize the first section + val firstSectPtr = afterBucketDefAddr(vectPtr) + initSectionWriter(firstSectPtr, ((vectPtr + maxBytes).addr - firstSectPtr.addr).toInt) } else { // check the bucket schema is identical. If not, return BucketSchemaMismatch - if (!matchBucketDef(h, addr)) return BucketSchemaMismatch + if (!matchBucketDef(h, vectPtr)) return BucketSchemaMismatch } - // Now, iterate through the counters and add them to each individual vector - val hist = h.toHistogram - bucketAppenders.foreach { appenders => - for { b <- 0 until numBuckets optimized } { - val resp = appenders(b).addData(hist.bucketValue(b).toLong) - require(resp == Ack) - } + val res = appendBlob(buf.byteArray, buf.addressOffset + h.valuesIndex, h.valuesNumBytes) + if (res == Ack) { + // set new number of bytes first. Remember to exclude initial 4 byte length prefix + setNumBytes(maxBytes - bytesLeft - 4) + // Finally, increase # histograms which is the ultimate safe gate for access by readers + incrNumHistograms(vectPtr) } - - incrNumHistograms(addr) - Ack + res } final def addNA(): AddResponse = Ack // TODO: Add a 0 to every appender - def addFromReaderNoNA(reader: RowReader, col: Int): AddResponse = - addData(reader.blobAsBuffer(col).asInstanceOf[UnsafeBuffer]) - def copyToBuffer: Buffer[UnsafeBuffer] = ??? - def apply(index: Int): UnsafeBuffer = ??? + def addFromReaderNoNA(reader: RowReader, col: Int): AddResponse = addData(reader.blobAsBuffer(col)) + def copyToBuffer: Buffer[DirectBuffer] = ??? + def apply(index: Int): DirectBuffer = ??? def finishCompaction(newAddress: BinaryRegion.NativePointer): BinaryVectorPtr = newAddress // NOTE: do not access reader below unless this vect is nonempty. TODO: fix this, or don't if we don't use this class - lazy val reader: VectorDataReader = new ColumnarHistogramReader(addr) + lazy val reader: VectorDataReader = new RowHistogramReader(vectPtr) def reset(): Unit = { - bucketAppenders.foreach(_.foreach(_.dispose())) - bucketAppenders = None - resetNumHistograms(addr) - } - - // Optimize each bucket's appenders, then create a new region with the same headers but pointing at the - // optimized vectors. - // TODO: this is NOT safe for persistence and recovery, as pointers cannot be persisted or recovered. - // For us to really make persistence of this work, we would need to pursue one of these strategies: - // 1) Change code of each LongAppendingVector to tell us how much optimized bytes take up for each bucket, - // then do a giant allocation including every bucket, and use relative pointers, not absolute, to point - // to each one; (or possibly a different kind of allocator) - // 2) Use BlockIDs and offsets instead of absolute pointers, and persist entire blocks. - override def optimize(memFactory: MemFactory, hint: EncodingHint = AutoDetect): BinaryVectorPtr = { - val optimizedBuckets = bucketAppenders.map { appenders => - appenders.map(_.optimize(memFactory, hint)) - }.getOrElse(Array.empty[BinaryVectorPtr]) - - val newHeaderAddr = memFactory.allocateOffheap(numBytes) - // Copy headers including bucket def - val bucketPtrOffset = (afterBucketDefAddr(addr).addr - addr).toInt - UnsafeUtils.copy(addr, newHeaderAddr, bucketPtrOffset) - - for { b <- 0 until optimizedBuckets.size optimized } { - UnsafeUtils.setLong(newHeaderAddr + bucketPtrOffset + 8*b, optimizedBuckets(b)) - } - - newHeaderAddr + resetNumHistograms(vectPtr) + setNumBytes(OffsetNumBuckets + 2) } - // NOTE: allocating vectors during ingestion is a REALLY BAD idea. For one if one runs out of memory then - // it will fail but ingestion into other vectors might succeed, resulting in undefined switchBuffers behaviors. - private def initBuckets(numBuckets: Int): Unit = { - val bucketPointersAddr = afterBucketDefAddr(addr).addr - val appenders = (0 until numBuckets).map { b => - val appender = LongBinaryVector.appendingVectorNoNA(factory, maxItems) - UnsafeUtils.setLong(bucketPointersAddr + 8*b, appender.addr) - appender - } - bucketAppenders = Some(appenders.toArray) - - // Initialize number of bytes in this histogram header - UnsafeUtils.setInt(addr, (bucketPointersAddr - addr).toInt + 8 * numBuckets) - } + // We don't optimize -- for now. Histograms are already stored compressed. + // In future, play with other optimization strategies, such as delta encoding. } trait HistogramReader extends VectorDataReader { @@ -320,21 +283,76 @@ trait HistogramReader extends VectorDataReader { def sum(start: Int, end: Int): MutableHistogram } -class ColumnarHistogramReader(histVect: BinaryVectorPtr) extends HistogramReader { +/** + * A reader for row-based Histogram vectors. Mostly contains logic to skip around the vector to find the right + * record pointer. + */ +class RowHistogramReader(histVect: Ptr.U8) extends HistogramReader { import HistogramVector._ final def length: Int = getNumHistograms(histVect) val numBuckets = if (length > 0) getNumBuckets(histVect) else 0 - val bucketAddrs = if (length > 0) { - val bucketAddrBase = afterBucketDefAddr(histVect).addr - (0 until numBuckets).map(b => UnsafeUtils.getLong(bucketAddrBase + 8 * b)).toArray - } else { - Array.empty[BinaryVectorPtr] - } - val readers = if (length > 0) bucketAddrs.map(LongBinaryVector.apply) else Array.empty[LongVectorDataReader] + var curSection: Section = _ + var curElemNo = 0 + var sectStartingElemNo = 0 + var curHist: Ptr.U8 = _ + if (length > 0) setFirstSection() val buckets = HistogramBuckets(bucketDefAddr(histVect).add(-2), formatCode(histVect)) - val returnHist = MutableHistogram.empty(buckets) + val returnHist = LongHistogram(buckets, new Array[Long](buckets.numBuckets)) + val endAddr = histVect + histVect.asI32.getI32 + 4 + + private def setFirstSection(): Unit = { + curSection = Section.fromPtr(afterBucketDefAddr(histVect)) + curHist = curSection.firstElem + curElemNo = 0 + sectStartingElemNo = 0 + } + + // Assume that most read patterns move the "cursor" or element # forward. Since we track the current section + // moving forward or jumping to next section is easy. Jumping backwards within current section is not too bad - + // we restart at beg of current section. Going back before current section is expensive, then we start over. + // TODO: split this out into a SectionReader trait + private def locate(elemNo: Int): Ptr.U8 = { + require(elemNo >= 0 && elemNo < length, s"$elemNo is out of vector bounds [0, $length)") + if (elemNo == curElemNo) { + curHist + } else if (elemNo > curElemNo) { + // Jump forward to next section until we are in section containing elemNo. BUT, don't jump beyond cur length + while (elemNo >= (sectStartingElemNo + curSection.numElements) && curSection.endAddr.addr < endAddr.addr) { + curElemNo = sectStartingElemNo + curSection.numElements + curSection = Section.fromPtr(curSection.endAddr) + sectStartingElemNo = curElemNo + curHist = curSection.firstElem + } + + curHist = skipAhead(curHist, elemNo - curElemNo) + curElemNo = elemNo + curHist + } else { // go backwards then go forwards + // Is it still within current section? If so restart search at beg of section + if (elemNo >= sectStartingElemNo) { + curElemNo = sectStartingElemNo + curHist = curSection.firstElem + } else { + // Otherwise restart search at beginning + setFirstSection() + } + locate(elemNo) + } + } + + // Skips ahead numElems elements starting at startPtr and returns the new pointer. NOTE: numElems might be 0. + private def skipAhead(startPtr: Ptr.U8, numElems: Int): Ptr.U8 = { + require(numElems >= 0) + var togo = numElems + var ptr = startPtr + while (togo > 0) { + ptr += ptr.asU16.getU16 + 2 + togo -= 1 + } + ptr + } /** * Iterates through each histogram. Note this is expensive due to materializing the Histogram object @@ -356,18 +374,22 @@ class ColumnarHistogramReader(histVect: BinaryVectorPtr) extends HistogramReader // WARNING: histogram returned is shared between calls, do not reuse! final def apply(index: Int): Histogram = { require(length > 0) - for { b <- 0 until numBuckets optimized } { - returnHist.values(b) = readers(b).apply(bucketAddrs(b), index) - } + val histPtr = locate(index) + val histLen = histPtr.asU16.getU16 + val buf = BinaryHistogram.valuesBuf + buf.wrap(histPtr.add(2).addr, histLen) + NibblePack.unpackToSink(buf, NibblePack.DeltaSink(returnHist.values), numBuckets) returnHist } // sum_over_time returning a Histogram with sums for each bucket. Start and end are inclusive row numbers + // NOTE: for now this is just a dumb implementation that decompresses each histogram fully final def sum(start: Int, end: Int): MutableHistogram = { require(length > 0 && start >= 0 && end < length) - for { b <- 0 until numBuckets optimized } { - returnHist.values(b) = readers(b).sum(bucketAddrs(b), start, end) + val summedHist = MutableHistogram.empty(buckets) + for { i <- start to end optimized } { + summedHist.add(apply(i).asInstanceOf[HistogramWithBuckets]) } - returnHist + summedHist } } \ No newline at end of file diff --git a/memory/src/test/scala/filodb.memory/format/vectors/HistogramVectorTest.scala b/memory/src/test/scala/filodb.memory/format/vectors/HistogramVectorTest.scala index 24a5655ea4..9133079745 100644 --- a/memory/src/test/scala/filodb.memory/format/vectors/HistogramVectorTest.scala +++ b/memory/src/test/scala/filodb.memory/format/vectors/HistogramVectorTest.scala @@ -8,11 +8,11 @@ class HistogramVectorTest extends NativeVectorTest { import HistogramTest._ it("should throw exceptions trying to query empty HistogramVector") { - val appender = HistogramVector.appendingColumnar(memFactory, 8, 100) + val appender = HistogramVector.appending(memFactory, 1024) appender.length shouldEqual 0 appender.isAllNA shouldEqual true - val reader = appender.reader.asInstanceOf[ColumnarHistogramReader] + val reader = appender.reader.asInstanceOf[RowHistogramReader] reader.length(appender.addr) shouldEqual 0 reader.numBuckets shouldEqual 0 @@ -30,7 +30,7 @@ class HistogramVectorTest extends NativeVectorTest { } it("should accept BinaryHistograms of the same schema and be able to query them") { - val appender = HistogramVector.appendingColumnar(memFactory, 8, 50) + val appender = HistogramVector.appending(memFactory, 1024) rawLongBuckets.foreach { rawBuckets => BinaryHistogram.writeDelta(bucketScheme, rawBuckets, buffer) appender.addData(buffer) shouldEqual Ack @@ -38,7 +38,7 @@ class HistogramVectorTest extends NativeVectorTest { appender.length shouldEqual rawHistBuckets.length - val reader = appender.reader.asInstanceOf[ColumnarHistogramReader] + val reader = appender.reader.asInstanceOf[RowHistogramReader] reader.length shouldEqual rawHistBuckets.length (0 until rawHistBuckets.length).foreach { i => @@ -51,7 +51,7 @@ class HistogramVectorTest extends NativeVectorTest { } it("should optimize histograms and be able to query optimized vectors") { - val appender = HistogramVector.appendingColumnar(memFactory, 8, 50) + val appender = HistogramVector.appending(memFactory, 1024) rawLongBuckets.foreach { rawBuckets => BinaryHistogram.writeDelta(bucketScheme, rawBuckets, buffer) appender.addData(buffer) shouldEqual Ack @@ -59,7 +59,7 @@ class HistogramVectorTest extends NativeVectorTest { appender.length shouldEqual rawHistBuckets.length - val reader = appender.reader.asInstanceOf[ColumnarHistogramReader] + val reader = appender.reader.asInstanceOf[RowHistogramReader] reader.length shouldEqual rawHistBuckets.length (0 until rawHistBuckets.length).foreach { i => @@ -68,8 +68,8 @@ class HistogramVectorTest extends NativeVectorTest { } val optimized = appender.optimize(memFactory) - val optReader = new ColumnarHistogramReader(optimized) - optReader.length shouldEqual rawHistBuckets.length + val optReader = HistogramVector(BinaryVector.asBuffer(optimized)) + optReader.length(optimized) shouldEqual rawHistBuckets.length (0 until rawHistBuckets.length).foreach { i => val h = optReader(i) verifyHistogram(h, i) @@ -84,7 +84,7 @@ class HistogramVectorTest extends NativeVectorTest { } it("should reject BinaryHistograms of schema different from first schema ingested") { - val appender = HistogramVector.appendingColumnar(memFactory, 8, 50) + val appender = HistogramVector.appending(memFactory, 1024) rawLongBuckets.foreach { rawBuckets => BinaryHistogram.writeDelta(bucketScheme, rawBuckets, buffer) appender.addData(buffer) shouldEqual Ack @@ -93,19 +93,54 @@ class HistogramVectorTest extends NativeVectorTest { appender.length shouldEqual rawHistBuckets.length // A record using a different schema - BinaryHistogram.writeDelta(HistogramBuckets.binaryBuckets64, Array[Long](0, 1, 2, 0), buffer) + BinaryHistogram.writeDelta(HistogramBuckets.binaryBuckets64, Array.fill(64)(1L), buffer) appender.addData(buffer) shouldEqual BucketSchemaMismatch } + it("should reject initially invalid BinaryHistogram") { + val appender = HistogramVector.appending(memFactory, 1024) + + // Create some garbage + buffer.putStringWithoutLengthAscii(0, "monkeying") + + appender.addData(buffer) shouldEqual InvalidHistogram + appender.length shouldEqual 0 + + // Reject null histograms also + buffer.putShort(0, 1) + buffer.putByte(2, 0) + appender.addData(buffer) shouldEqual InvalidHistogram + appender.length shouldEqual 0 + } + it("should reject new adds when vector is full") { - val appender = HistogramVector.appendingColumnar(memFactory, 8, 4) + val appender = HistogramVector.appending(memFactory, 76) rawLongBuckets.foreach { rawBuckets => BinaryHistogram.writeDelta(bucketScheme, rawBuckets, buffer) appender.addData(buffer) shouldEqual Ack } + appender.numBytes shouldEqual HistogramVector.OffsetBucketDef + 2+8+8 + 4 + (12+11+8+11) appender.length shouldEqual rawHistBuckets.length - appender.addData(buffer) shouldEqual VectorTooSmall(0, 0) + appender.addData(buffer) shouldBe a[VectorTooSmall] + } + + // Test for Section reader code in RowHistogramReader, that we can jump back and forth + it("should be able to randomly look up any element in long HistogramVector") { + val numElements = 150 + val appender = HistogramVector.appending(memFactory, numElements * 20) + (0 until numElements).foreach { i => + BinaryHistogram.writeDelta(bucketScheme, rawLongBuckets(i % rawLongBuckets.length), buffer) + appender.addData(buffer) shouldEqual Ack + } + + val optimized = appender.optimize(memFactory) + val optReader = HistogramVector(BinaryVector.asBuffer(optimized)) + + for { _ <- 0 to 50 } { + val elemNo = scala.util.Random.nextInt(numElements) + verifyHistogram(optReader(elemNo), elemNo % (rawLongBuckets.length)) + } } } \ No newline at end of file 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 9fb6537bd1..482d8ff18b 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala @@ -23,8 +23,8 @@ trait RawDataWindowingSpec extends FunSpec with Matchers with BeforeAndAfterAll private val blockStore = new PageAlignedBlockManager(100 * 1024 * 1024, new MemoryStats(Map("test"-> "test")), null, 16) protected val ingestBlockHolder = new BlockMemFactory(blockStore, None, timeseriesDataset.blockMetaSize, true) - val maxChunkSize = 200 - protected val tsBufferPool = new WriteBufferPool(TestData.nativeMem, timeseriesDataset, maxChunkSize, 100) + val storeConf = TestData.storeConf.copy(maxChunksSize = 200) + protected val tsBufferPool = new WriteBufferPool(TestData.nativeMem, timeseriesDataset, storeConf) override def afterAll(): Unit = { blockStore.releaseBlocks() @@ -63,7 +63,7 @@ trait RawDataWindowingSpec extends FunSpec with Matchers with BeforeAndAfterAll def emptyAggHist: bv.MutableHistogram = bv.MutableHistogram.empty(MMD.histBucketScheme) protected val histIngestBH = new BlockMemFactory(blockStore, None, MMD.histDataset.blockMetaSize, true) - protected val histBufferPool = new WriteBufferPool(TestData.nativeMem, MMD.histDataset, 100, 10) + protected val histBufferPool = new WriteBufferPool(TestData.nativeMem, MMD.histDataset, TestData.storeConf) // Designed explicitly to work with linearHistSeries records and histDataset from MachineMetricsData def histogramRV(numSamples: Int = 100, numBuckets: Int = 8): From 1525ec9fcf6b236507b3c0976371ef8d778d0634 Mon Sep 17 00:00:00 2001 From: "Brian S. O'Neill" Date: Wed, 13 Mar 2019 13:32:43 -0700 Subject: [PATCH 16/41] bug(coordinator): Finish dataset initialization using a message to avoid potential thread-safety issues. (#277) --- .../scala/filodb.coordinator/NodeClusterActor.scala | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/NodeClusterActor.scala b/coordinator/src/main/scala/filodb.coordinator/NodeClusterActor.scala index 39c05d7382..137850bffb 100644 --- a/coordinator/src/main/scala/filodb.coordinator/NodeClusterActor.scala +++ b/coordinator/src/main/scala/filodb.coordinator/NodeClusterActor.scala @@ -17,6 +17,7 @@ import filodb.core.downsample.DownsampleConfig import filodb.core.metadata.Dataset import filodb.core.store.{AssignShardConfig, IngestionConfig, MetaStore, StoreConfig, UnassignShardConfig} +//scalastyle:off number.of.types object NodeClusterActor { sealed trait ClusterActorEvent @@ -79,6 +80,9 @@ object NodeClusterActor { source.downsampleConfig) } + // Only used during initial setup, called during recovery and before calling initiateShardStateRecovery. + private final case class SetupDatasetFinished(ref: DatasetRef) + // A dummy source to use for tests and when you just want to push new records in val noOpSource = IngestionSource(classOf[NoOpStreamFactory].getName) @@ -314,10 +318,11 @@ private[filodb] class NodeClusterActor(settings: FilodbSettings, // The initial recovery handler: recover dataset setup/ingestion config first def datasetHandler: Receive = LoggingReceive { case e: SetupDataset => - setupDataset(e, sender()) map { _ => - initDatasets -= e.ref - if (initDatasets.isEmpty) initiateShardStateRecovery() - } + setupDataset(e, sender()) map { _ => self ! SetupDatasetFinished(e.ref) } + case e: SetupDatasetFinished => { + initDatasets -= e.ref + if (initDatasets.isEmpty) initiateShardStateRecovery() + } case GetDatasetFromRef(r) => sender() ! datasets(r) } From 7db9fe9529054d0086001dd9a7e5afafa945f852 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Thu, 14 Mar 2019 14:24:04 -0700 Subject: [PATCH 17/41] feat(query): histogram_bucket function (#278) --- README.md | 3 ++- project/FiloBuild.scala | 4 +++- .../main/scala/filodb/query/PlanEnums.scala | 2 ++ .../query/exec/rangefn/InstantFunction.scala | 21 ++++++++++++++++++ .../exec/rangefn/InstantFunctionSpec.scala | 22 +++++++++++++++++++ 5 files changed, 50 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index 0524c30133..c23c24c26d 100644 --- a/README.md +++ b/README.md @@ -390,7 +390,8 @@ One major difference FiloDB has from the Prometheus data model is that FiloDB su * There is no need to append `_bucket` to the metric name. * However, you need to select the histogram column like `__col__="hist"` -* TODO: document ways of selecting the histogram bucket +* To compute quantiles: `histogram_quantile(0.7, rate(http_req_latency{app="foo",__col__="hist"}[5m]))` +* To extract a bucket: `histogram_bucket(100.0, http_req_latency{app="foo",__col__="hist"})` ### Using the FiloDB HTTP API diff --git a/project/FiloBuild.scala b/project/FiloBuild.scala index c467ec89a5..b9bbd13695 100644 --- a/project/FiloBuild.scala +++ b/project/FiloBuild.scala @@ -55,6 +55,7 @@ object FiloBuild extends Build { .in(file("query")) .settings(libraryDependencies ++= queryDeps) .settings(commonSettings: _*) + .settings(scalacOptions += "-language:postfixOps") .settings(name := "filodb-query") .dependsOn(core % "compile->compile; test->test") @@ -232,7 +233,8 @@ object FiloBuild extends Build { ) lazy val queryDeps = commonDeps ++ Seq( - "com.tdunning" % "t-digest" % "3.1" + "com.tdunning" % "t-digest" % "3.1", + scalaxyDep ) lazy val coordDeps = commonDeps ++ Seq( diff --git a/query/src/main/scala/filodb/query/PlanEnums.scala b/query/src/main/scala/filodb/query/PlanEnums.scala index 948f54ff89..4384122c84 100644 --- a/query/src/main/scala/filodb/query/PlanEnums.scala +++ b/query/src/main/scala/filodb/query/PlanEnums.scala @@ -24,6 +24,8 @@ object InstantFunctionId extends Enum[InstantFunctionId] { case object HistogramQuantile extends InstantFunctionId("histogram_quantile") + case object HistogramBucket extends InstantFunctionId("histogram_bucket") + case object Ln extends InstantFunctionId("ln") case object Log10 extends InstantFunctionId("log10") diff --git a/query/src/main/scala/filodb/query/exec/rangefn/InstantFunction.scala b/query/src/main/scala/filodb/query/exec/rangefn/InstantFunction.scala index 40ce24889c..0f18ad6270 100644 --- a/query/src/main/scala/filodb/query/exec/rangefn/InstantFunction.scala +++ b/query/src/main/scala/filodb/query/exec/rangefn/InstantFunction.scala @@ -1,5 +1,7 @@ package filodb.query.exec.rangefn +import scalaxy.loops._ + import filodb.memory.format.vectors.Histogram import filodb.query.InstantFunctionId import filodb.query.InstantFunctionId.{Log2, Sqrt, _} @@ -88,6 +90,7 @@ object InstantFunction { */ def histogram(function: InstantFunctionId, funcParams: Seq[Any]): HistogramInstantFunction = function match { case HistogramQuantile => HistogramQuantileImpl(funcParams) + case HistogramBucket => HistogramBucketImpl(funcParams) case _ => throw new UnsupportedOperationException(s"$function not supported.") } } @@ -248,4 +251,22 @@ case class HistogramQuantileImpl(funcParams: Seq[Any]) extends HistToDoubleIFunc val q = funcParams(0).asInstanceOf[Number].doubleValue() final def apply(value: Histogram): Double = value.quantile(q) +} + +/** + * Function to extract one bucket from any histogram (could be computed, not just raw). + * @param funcParams - a single value which is the Double bucket or "le" to extract. If it does not correspond + * to any existing bucket then NaN is returned. + */ +case class HistogramBucketImpl(funcParams: Seq[Any]) extends HistToDoubleIFunction { + require(funcParams.length == 1, "Bucket/le required for histogram bucket") + require(funcParams(0).isInstanceOf[Number], "histogram_bucket parameter must be a number") + val bucket = funcParams(0).asInstanceOf[Number].doubleValue() + + final def apply(value: Histogram): Double = { + for { b <- 0 until value.numBuckets optimized } { + if (Math.abs(value.bucketTop(b) - bucket) <= 1E-10) return value.bucketValue(b) + } + Double.NaN + } } \ No newline at end of file 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 41aa70edf5..96459a47b4 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/InstantFunctionSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/InstantFunctionSpec.scala @@ -210,6 +210,17 @@ class InstantFunctionSpec extends RawDataWindowingSpec with ScalaFutures { val ivMapper = exec.InstantVectorFunctionMapper(InstantFunctionId.HistogramQuantile, Seq("b012")) ivMapper(Observable.fromIterable(sampleBase), queryConfig, 1000, histSchema) } should have message "requirement failed: histogram_quantile parameter must be a number" + + // histogram bucket + the[IllegalArgumentException] thrownBy { + val ivMapper = exec.InstantVectorFunctionMapper(InstantFunctionId.HistogramBucket) + ivMapper(Observable.fromIterable(sampleBase), queryConfig, 1000, histSchema) + } should have message "requirement failed: Bucket/le required for histogram bucket" + + the[IllegalArgumentException] thrownBy { + val ivMapper = exec.InstantVectorFunctionMapper(InstantFunctionId.HistogramBucket, Seq("b012")) + ivMapper(Observable.fromIterable(sampleBase), queryConfig, 1000, histSchema) + } should have message "requirement failed: histogram_bucket parameter must be a number" } it ("should fail with wrong calculation") { @@ -235,6 +246,17 @@ class InstantFunctionSpec extends RawDataWindowingSpec with ScalaFutures { InstantFunctionId.HistogramQuantile, Seq(0.4), histSchema) } + it("should compute histogram_bucket on Histogram RV") { + val (data, histRV) = histogramRV(numSamples = 10) + val expected = Seq(1.0, 2.0, 3.0, 4.0, 4.0, 4.0, 4.0, 4.0) + applyFunctionAndAssertResult(Array(histRV), Array(expected.toIterator), + InstantFunctionId.HistogramBucket, Seq(16.0), histSchema) + + // Specifying a nonexistant bucket returns NaN + applyFunctionAndAssertResult(Array(histRV), Array(Seq.fill(8)(Double.NaN).toIterator), + InstantFunctionId.HistogramBucket, Seq(9.0), histSchema) + } + private def applyFunctionAndAssertResult(samples: Array[RangeVector], expectedVal: Array[Iterator[Double]], instantFunctionId: InstantFunctionId, funcParams: Seq[Any] = Nil, schema: ResultSchema = resultSchema): Unit = { From 6cb4419de7ea1723abe146b9a763062600ddad31 Mon Sep 17 00:00:00 2001 From: TanviBhavsar Date: Fri, 15 Mar 2019 15:30:29 -0700 Subject: [PATCH 18/41] bug(query): fix bug in Count to add input value in reduce only when it is not NaN --- .../filodb/query/exec/AggrOverRangeVectors.scala | 13 ++++--------- 1 file changed, 4 insertions(+), 9 deletions(-) diff --git a/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala b/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala index 0236719f80..010261b522 100644 --- a/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala +++ b/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala @@ -387,20 +387,15 @@ object CountRowAggregator extends RowAggregator { def zero: CountHolder = new CountHolder() def newRowToMapInto: MutableRowReader = new TransientRow() def map(rvk: RangeVectorKey, item: RowReader, mapInto: MutableRowReader): RowReader = { - if (!item.getDouble(1).isNaN) { - mapInto.setLong(0, item.getLong(0)) - mapInto.setDouble(1, 1d) - } - else { - mapInto.setLong(0, item.getLong(0)) - mapInto.setDouble(1, 0d) - } + mapInto.setLong(0, item.getLong(0)) + mapInto.setDouble(1, if (item.getDouble(1).isNaN) 0d else 1d) mapInto } def reduceAggregate(acc: CountHolder, aggRes: RowReader): CountHolder = { if (acc.count.isNaN && aggRes.getDouble(1) > 0) acc.count = 0d; acc.timestamp = aggRes.getLong(0) - acc.count += aggRes.getDouble(1) + if (!aggRes.getDouble(1).isNaN) + acc.count += aggRes.getDouble(1) acc } def present(aggRangeVector: RangeVector, limit: Int): Seq[RangeVector] = Seq(aggRangeVector) From a66de97aead1363125fe075536d4caa1bf050e32 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Mon, 18 Mar 2019 08:49:55 -0700 Subject: [PATCH 19/41] feat(core): hSum sum_over_time Histogram downsampler (#281) Will require dataset changes to enable --- .../binaryrecord2/RecordSchema.scala | 2 + .../downsample/ChunkDownsampler.scala | 34 ++++++++++ .../downsample/ShardDownsampler.scala | 2 + .../src/test/scala/filodb.core/TestData.scala | 24 ++++++- .../downsample/ShardDownsamplerSpec.scala | 62 +++++++++++++++---- .../rangefn/AggrOverTimeFunctionsSpec.scala | 15 +---- 6 files changed, 113 insertions(+), 26 deletions(-) diff --git a/core/src/main/scala/filodb.core/binaryrecord2/RecordSchema.scala b/core/src/main/scala/filodb.core/binaryrecord2/RecordSchema.scala index 5524ec0adf..a06c58131e 100644 --- a/core/src/main/scala/filodb.core/binaryrecord2/RecordSchema.scala +++ b/core/src/main/scala/filodb.core/binaryrecord2/RecordSchema.scala @@ -42,6 +42,8 @@ final class RecordSchema(val columns: Seq[ColumnInfo], import RecordSchema._ import BinaryRegion.NativePointer + override def toString: String = s"RecordSchema<$columns, $partitionFieldStart>" + val colNames = columns.map(_.name) val columnTypes = columns.map(_.colType) require(columnTypes.nonEmpty, "columnTypes cannot be empty") diff --git a/core/src/main/scala/filodb.core/downsample/ChunkDownsampler.scala b/core/src/main/scala/filodb.core/downsample/ChunkDownsampler.scala index 47e7f11310..c4b0e544d3 100644 --- a/core/src/main/scala/filodb.core/downsample/ChunkDownsampler.scala +++ b/core/src/main/scala/filodb.core/downsample/ChunkDownsampler.scala @@ -7,6 +7,7 @@ import enumeratum.{Enum, EnumEntry} import filodb.core.memstore.TimeSeriesPartition import filodb.core.metadata.Column.ColumnType import filodb.core.store.ChunkSetInfo +import filodb.memory.format.{vectors => bv} /** * Enum of supported downsampling function names @@ -21,6 +22,7 @@ object DownsamplerName extends Enum[DownsamplerName] { case object MinD extends DownsamplerName("dMin", classOf[MinDownsampler]) case object MaxD extends DownsamplerName("dMax", classOf[MaxDownsampler]) case object SumD extends DownsamplerName("dSum", classOf[SumDownsampler]) + case object SumH extends DownsamplerName("hSum", classOf[HistSumDownsampler]) case object CountD extends DownsamplerName("dCount", classOf[CountDownsampler]) case object AvgD extends DownsamplerName("dAvg", classOf[AvgDownsampler]) case object AvgAcD extends DownsamplerName("dAvgAc", classOf[AvgAcDownsampler]) @@ -94,6 +96,26 @@ trait TimeChunkDownsampler extends ChunkDownsampler { endRow: Int): Long } +/** + * Chunk downsampler trait for downsampling histogram columns -> histogram columns + */ +trait HistChunkDownsampler extends ChunkDownsampler { + override val colType: ColumnType = ColumnType.HistogramColumn + + /** + * Downsamples Chunk using histogram column Ids configured and emit histogram value + * @param part Time series partition to extract data from + * @param chunkset The chunksetInfo that needs to be downsampled + * @param startRow The start row number for the downsample period (inclusive) + * @param endRow The end row number for the downsample period (inclusive) + * @return downsampled value to emit + */ + def downsampleChunk(part: TimeSeriesPartition, + chunkset: ChunkSetInfo, + startRow: Int, + endRow: Int): bv.Histogram +} + /** * Downsamples by calculating sum of values in one column */ @@ -110,6 +132,18 @@ case class SumDownsampler(override val colIds: Seq[Int]) extends DoubleChunkDown } } +case class HistSumDownsampler(val colIds: Seq[Int]) extends HistChunkDownsampler { + require(colIds.length == 1, s"Sum downsample requires only one column. Got ${colIds.length}") + val name = DownsamplerName.SumH + def downsampleChunk(part: TimeSeriesPartition, + chunkset: ChunkSetInfo, + startRow: Int, + endRow: Int): bv.Histogram = { + val histReader = part.chunkReader(colIds(0), chunkset.vectorPtr(colIds(0))).asHistReader + histReader.sum(startRow, endRow) + } +} + /** * Downsamples by calculating count of values in one column */ diff --git a/core/src/main/scala/filodb.core/downsample/ShardDownsampler.scala b/core/src/main/scala/filodb.core/downsample/ShardDownsampler.scala index d48177ea8c..9af51c7586 100644 --- a/core/src/main/scala/filodb.core/downsample/ShardDownsampler.scala +++ b/core/src/main/scala/filodb.core/downsample/ShardDownsampler.scala @@ -99,6 +99,8 @@ class ShardDownsampler(dataset: Dataset, builder.addLong(d.downsampleChunk(part, chunkset, startRowNum, endRowNum)) case d: DoubleChunkDownsampler => builder.addDouble(d.downsampleChunk(part, chunkset, startRowNum, endRowNum)) + case h: HistChunkDownsampler => + builder.addBlob(h.downsampleChunk(part, chunkset, startRowNum, endRowNum).serialize()) } // add partKey finally builder.addPartKeyRecordFields(part.partKeyBase, part.partKeyOffset, dataset.partKeySchema) diff --git a/core/src/test/scala/filodb.core/TestData.scala b/core/src/test/scala/filodb.core/TestData.scala index a4771eba1d..743920ea8b 100644 --- a/core/src/test/scala/filodb.core/TestData.scala +++ b/core/src/test/scala/filodb.core/TestData.scala @@ -8,15 +8,16 @@ import monix.reactive.Observable import org.joda.time.DateTime import filodb.core.binaryrecord2.RecordBuilder -import filodb.core.memstore.SomeData +import filodb.core.memstore.{SomeData, TimeSeriesPartitionSpec, WriteBufferPool} import filodb.core.metadata.Column.ColumnType import filodb.core.metadata.{Dataset, DatasetOptions} +import filodb.core.query.RawDataRangeVector import filodb.core.store._ import filodb.core.Types.{PartitionKey, UTF8Map} import filodb.memory.format._ import filodb.memory.format.ZeroCopyUTF8String._ import filodb.memory.format.{vectors => bv} -import filodb.memory.{BinaryRegionLarge, MemFactory, NativeMemoryManager} +import filodb.memory._ object TestData { def toChunkSetStream(ds: Dataset, @@ -317,6 +318,25 @@ object MachineMetricsData { extraTags ++ Map("__name__".utf8 -> "http_requests_total".utf8, "dc".utf8 -> s"${n % numSeries}".utf8)) } } + + val histKeyBuilder = new RecordBuilder(TestData.nativeMem, histDataset.partKeySchema, 2048) + val histPartKey = histKeyBuilder.addFromObjects(extraTags) + + val blockStore = new PageAlignedBlockManager(100 * 1024 * 1024, new MemoryStats(Map("test"-> "test")), null, 16) + private val histIngestBH = new BlockMemFactory(blockStore, None, histDataset.blockMetaSize, true) + private val histBufferPool = new WriteBufferPool(TestData.nativeMem, histDataset, TestData.storeConf) + + // Designed explicitly to work with linearHistSeries records and histDataset from MachineMetricsData + def histogramRV(startTS: Long, pubFreq: Long = 10000L, numSamples: Int = 100, numBuckets: Int = 8): + (Stream[Seq[Any]], RawDataRangeVector) = { + val histData = linearHistSeries(startTS, 1, pubFreq.toInt, numBuckets).take(numSamples) + val container = records(histDataset, histData).records + val part = TimeSeriesPartitionSpec.makePart(0, histDataset, partKey=histPartKey, bufferPool=histBufferPool) + container.iterate(histDataset.ingestionSchema).foreach { row => part.ingest(row, histIngestBH) } + // Now flush and ingest the rest to ensure two separate chunks + part.switchBuffers(histIngestBH, encode = true) + (histData, RawDataRangeVector(null, part, AllChunkScan, Array(0, 3))) // select timestamp and histogram columns only + } } // A simulation of custom machine metrics data - for testing extractTimeBucket diff --git a/core/src/test/scala/filodb.core/downsample/ShardDownsamplerSpec.scala b/core/src/test/scala/filodb.core/downsample/ShardDownsamplerSpec.scala index 885f6c4b74..cfe227eebc 100644 --- a/core/src/test/scala/filodb.core/downsample/ShardDownsamplerSpec.scala +++ b/core/src/test/scala/filodb.core/downsample/ShardDownsamplerSpec.scala @@ -4,7 +4,7 @@ import scala.collection.mutable import org.scalatest.{BeforeAndAfterAll, FunSpec, Matchers} -import filodb.core.TestData +import filodb.core.{TestData, MachineMetricsData => MMD} import filodb.core.binaryrecord2.{MapItemConsumer, RecordBuilder, RecordContainer} import filodb.core.memstore.{TimeSeriesPartition, TimeSeriesPartitionSpec, TimeSeriesShardStats, WriteBufferPool} import filodb.core.metadata.{Dataset, DatasetOptions} @@ -12,7 +12,7 @@ import filodb.core.metadata.Column.ColumnType._ import filodb.core.query.RawDataRangeVector import filodb.core.store.AllChunkScan import filodb.memory._ -import filodb.memory.format.{TupleRowReader, ZeroCopyUTF8String} +import filodb.memory.format.{TupleRowReader, ZeroCopyUTF8String, vectors => bv} // scalastyle:off null class ShardDownsamplerSpec extends FunSpec with Matchers with BeforeAndAfterAll { @@ -25,15 +25,13 @@ class ShardDownsamplerSpec extends FunSpec with Matchers with BeforeAndAfterAll DatasetOptions(Seq("__name__", "job"), "__name__", "value")).get val customDataset = Dataset.make("custom2", - Seq("name:string", "namespace:string","instance:string"), - Seq("timestamp:ts", "count:double", "min:double", "max:double", "total:double", "avg:double"), + Seq("name:string", "namespace:string", "instance:string"), + Seq("timestamp:ts", "count:double", "min:double", "max:double", "total:double", "avg:double", "h:hist"), Seq("timestamp"), - Seq("tTime(0)", "dSum(1)", "dMin(2)", "dMax(3)", "dSum(4)", "dAvgAc(5@1)"), + Seq("tTime(0)", "dSum(1)", "dMin(2)", "dMax(3)", "dSum(4)", "dAvgAc(5@1)", "hSum(6)"), DatasetOptions(Seq("name", "namespace"), "name", "total")).get - private val blockStore = new PageAlignedBlockManager(100 * 1024 * 1024, - new MemoryStats(Map("test"-> "test")), null, 16) - + private val blockStore = MMD.blockStore protected val ingestBlockHolder = new BlockMemFactory(blockStore, None, promDataset.blockMetaSize, true) val storeConf = TestData.storeConf.copy(maxChunksSize = 200) @@ -83,9 +81,9 @@ class ShardDownsamplerSpec extends FunSpec with Matchers with BeforeAndAfterAll new TimeSeriesShardStats(customDataset.ref, 0)) val dsSchema = downsampleOps.downsampleIngestSchema() dsSchema.columns.map(_.name) shouldEqual - Seq("tTime", "dSum", "dMin", "dMax", "dSum", "dAvgAc", "name", "namespace", "instance") + Seq("tTime", "dSum", "dMin", "dMax", "dSum", "dAvgAc", "hSum", "name", "namespace", "instance") dsSchema.columns.map(_.colType) shouldEqual - Seq(TimestampColumn, DoubleColumn, DoubleColumn, DoubleColumn, DoubleColumn, DoubleColumn, + Seq(TimestampColumn, DoubleColumn, DoubleColumn, DoubleColumn, DoubleColumn, DoubleColumn, HistogramColumn, StringColumn, StringColumn, StringColumn) } @@ -96,7 +94,7 @@ class ShardDownsamplerSpec extends FunSpec with Matchers with BeforeAndAfterAll val dsSchema = downsampleOps.downsampleSchema val dsRecords = downsampleOps.newEmptyDownsampleRecords - downsampleOps.populateDownsampleRecords(rv.partition.asInstanceOf[TimeSeriesPartition],chunkInfos, dsRecords) + downsampleOps.populateDownsampleRecords(rv.partition.asInstanceOf[TimeSeriesPartition], chunkInfos, dsRecords) // with resolution 5000 val downsampledData1 = dsRecords(0).builder.optimalContainerBytes().flatMap { con => @@ -205,4 +203,46 @@ class ShardDownsamplerSpec extends FunSpec with Matchers with BeforeAndAfterAll downsampledData2.map(_._6) shouldEqual expectedAvgs2 } + + val histDSDownsamplers = Seq("tTime(0)", "tTime(1)", "tTime(2)", "hSum(3)") + val histDSDataset = MMD.histDataset.copy( + downsamplers = Dataset.validateDownsamplers(histDSDownsamplers).get) + + // Create downsampleOps for histogram dataset. Samples every 10s, downsample freq 60s/1min + val downsampleOpsH = new ShardDownsampler(histDSDataset, 0, true, Seq(60000), NoOpDownsamplePublisher, + new TimeSeriesShardStats(histDSDataset.ref, 0)) + + def emptyAggHist: bv.MutableHistogram = bv.MutableHistogram.empty(MMD.histBucketScheme) + + it("should downsample histogram schema/dataset correctly") { + val startTS = 610000L // So we can group by minute easily + val (data, rv) = MMD.histogramRV(startTS, numSamples = 200) + val chunkInfos = rv.chunkInfos(0L, Long.MaxValue) + val dsSchema = downsampleOpsH.downsampleSchema + val dsRecords = downsampleOpsH.newEmptyDownsampleRecords + + downsampleOpsH.populateDownsampleRecords(rv.partition.asInstanceOf[TimeSeriesPartition], chunkInfos, dsRecords) + + val downsampledData1 = dsRecords(0).builder.optimalContainerBytes().flatMap { con => + val c = RecordContainer(con) + + c.iterate(dsSchema).map {r => + val timestamp = r.getLong(0) + val count= r.getLong(1) + val sum = r.getLong(2) + val hist = r.getHistogram(3) + (timestamp, count, sum, hist) + } + } + + val expectedSums = data.grouped(6).toSeq.map { dataRows => + dataRows.map(_(3).asInstanceOf[bv.MutableHistogram]) + .foldLeft(emptyAggHist) { case (agg, h) => agg.add(h); agg } + } + + // Skip comparing the last sample because end of chunk=100 rows is not evenly divisible by 6 + downsampledData1.zip(expectedSums).take(100/6).foreach { case (dsData, expected) => + dsData._4 shouldEqual expected + } + } } 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 482d8ff18b..2583d6382d 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala @@ -62,20 +62,9 @@ trait RawDataWindowingSpec extends FunSpec with Matchers with BeforeAndAfterAll // Call this only after calling histogramRV def emptyAggHist: bv.MutableHistogram = bv.MutableHistogram.empty(MMD.histBucketScheme) - protected val histIngestBH = new BlockMemFactory(blockStore, None, MMD.histDataset.blockMetaSize, true) - protected val histBufferPool = new WriteBufferPool(TestData.nativeMem, MMD.histDataset, TestData.storeConf) - // Designed explicitly to work with linearHistSeries records and histDataset from MachineMetricsData - def histogramRV(numSamples: Int = 100, numBuckets: Int = 8): - (Stream[Seq[Any]], RawDataRangeVector) = { - val histData = MMD.linearHistSeries(defaultStartTS, 1, pubFreq.toInt, numBuckets).take(numSamples) - val container = MMD.records(MMD.histDataset, histData).records - val part = TimeSeriesPartitionSpec.makePart(0, MMD.histDataset, bufferPool = histBufferPool) - container.iterate(MMD.histDataset.ingestionSchema).foreach { row => part.ingest(row, histIngestBH) } - // Now flush and ingest the rest to ensure two separate chunks - part.switchBuffers(histIngestBH, encode = true) - (histData, RawDataRangeVector(null, part, AllChunkScan, Array(0, 3))) // select timestamp and histogram columns only - } + def histogramRV(numSamples: Int = 100, numBuckets: Int = 8): (Stream[Seq[Any]], RawDataRangeVector) = + MMD.histogramRV(defaultStartTS, pubFreq, numSamples, numBuckets) def chunkedWindowIt(data: Seq[Double], rv: RawDataRangeVector, From 988a753655cb95640de70702b71dd7abe2681de6 Mon Sep 17 00:00:00 2001 From: "Brian S. O'Neill" Date: Tue, 19 Mar 2019 09:28:15 -0700 Subject: [PATCH 20/41] bug(core): Fix duplicate shard setup detection. (#285) --- .../filodb.core/memstore/TimeSeriesMemStore.scala | 2 +- .../filodb.core/memstore/TimeSeriesMemStoreSpec.scala | 10 ++++++++++ 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala index 1c23272a92..cdf573c5b1 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala @@ -48,7 +48,7 @@ extends MemStore with StrictLogging { def setup(dataset: Dataset, shard: Int, storeConf: StoreConfig, downsample: DownsampleConfig = DownsampleConfig.disabled): Unit = synchronized { val shards = datasets.getOrElseUpdate(dataset.ref, new NonBlockingHashMapLong[TimeSeriesShard](32, false)) - if (shards contains shard) { + if (shards.containsKey(shard)) { throw ShardAlreadySetup(dataset.ref, shard) } else { val publisher = downsamplePublishers.getOrElseUpdate(dataset.ref, makeAndStartPublisher(downsample)) diff --git a/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala b/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala index 48a2d1ab07..ba89a16d42 100644 --- a/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala +++ b/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala @@ -38,6 +38,16 @@ class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter w memStore.metastore.clearAllData() } + it("should detect duplicate setup") { + memStore.setup(dataset1, 0, TestData.storeConf) + try { + memStore.setup(dataset1, 0, TestData.storeConf) + fail() + } catch { + case e: ShardAlreadySetup => { } // expected + } + } + // Look mama! Real-time time series ingestion and querying across multiple partitions! it("should ingest into multiple series and be able to query across all partitions in real time") { memStore.setup(dataset1, 0, TestData.storeConf) From 49447f240bb503d464b731958744a925d5664de2 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Tue, 19 Mar 2019 15:19:30 -0700 Subject: [PATCH 21/41] feat(core): Add key=value optional parameters to each Column definition (#280) NOTE: BREAKING CHANGES to dataset definition if HistogramColumn used --- README.md | 2 + .../scala/filodb.core/metadata/Column.scala | 60 +++++++++++++------ .../filodb.core/metadata/ComputedColumn.scala | 2 + .../scala/filodb.core/metadata/Dataset.scala | 5 +- .../scala/filodb.core/query/KeyFilter.scala | 2 +- .../src/test/scala/filodb.core/TestData.scala | 4 +- .../downsample/ShardDownsamplerSpec.scala | 2 +- .../filodb.core/metadata/ColumnSpec.scala | 8 +++ .../filodb.core/metadata/DatasetSpec.scala | 25 ++++++++ .../scala/filodb.spark/TypeConverters.scala | 2 +- 10 files changed, 88 insertions(+), 24 deletions(-) diff --git a/README.md b/README.md index c23c24c26d..e389864f9f 100644 --- a/README.md +++ b/README.md @@ -321,6 +321,8 @@ FiloDB is designed to scale to ingest and query millions of discrete time series The **partition key** differentiates time series and also controls distribution of time series across the cluster. For more information on sharding, see the sharding section below. Components of a partition key, including individual key/values of `MapColumn`s, are indexed and used for filtering in queries. +The data points use a configurable schema consisting of multiple columns. Each column definition consists of `name:columntype`, with optional parameters. For examples, see the examples below, or see the introductory walk-through above where two datasets are created. + ### Prometheus FiloDB Schema for Operational Metrics * Partition key = `tags:map` diff --git a/core/src/main/scala/filodb.core/metadata/Column.scala b/core/src/main/scala/filodb.core/metadata/Column.scala index c612deee07..3ed10d561b 100644 --- a/core/src/main/scala/filodb.core/metadata/Column.scala +++ b/core/src/main/scala/filodb.core/metadata/Column.scala @@ -1,7 +1,9 @@ package filodb.core.metadata import scala.reflect.ClassTag +import scala.util.Try +import com.typesafe.config.{Config, ConfigFactory} import com.typesafe.scalalogging.StrictLogging import enumeratum.{Enum, EnumEntry} import org.scalactic._ @@ -20,6 +22,7 @@ trait Column { def name: String def columnType: Column.ColumnType def extractor: TypedFieldExtractor[_] + def params: Config // More type safe than just using ==, if we ever change the type of ColumnId // TODO(velvia): remove this and just use id @@ -31,12 +34,17 @@ trait Column { */ case class DataColumn(id: Int, name: String, - columnType: Column.ColumnType) extends Column { + columnType: Column.ColumnType, + params: Config = ConfigFactory.empty) extends Column { + import collection.JavaConverters._ + // Use this for efficient serialization over the wire. // We leave out the dataset because that is almost always inferred from context. // NOTE: this is one reason why column names cannot have commas - override def toString: String = - s"[$id,$name,$columnType]" + override def toString: String = { + val paramStrs = params.entrySet.asScala.map { e => s"${e.getKey}=${e.getValue.render}" } + (Seq(id, name, columnType).map(_.toString) ++ paramStrs).mkString("[", ",", "]") + } def extractor: TypedFieldExtractor[_] = columnType.keyType.extractor } @@ -47,17 +55,26 @@ object DataColumn { */ def fromString(str: String): DataColumn = { val parts = str.drop(1).dropRight(1).split(',') - DataColumn(parts(0).toInt, parts(1), Column.ColumnType.withName(parts(2))) + val params = ConfigFactory.parseString(parts.drop(3).mkString("\n")) + DataColumn(parts(0).toInt, parts(1), Column.ColumnType.withName(parts(2)), params) } } object Column extends StrictLogging { + import Dataset._ + import TrySugar._ + sealed trait ColumnType extends EnumEntry { def typeName: String // NOTE: due to a Spark serialization bug, this cannot be a val // (https://github.com/apache/spark/pull/7122) def clazz: Class[_] def keyType: SingleKeyTypeBase[_] + + /** + * Validates the params found in the column definition. By default no checking is done. + */ + def validateParams(params: Config): Unit Or One[BadSchema] = Good(()) } sealed abstract class RichColumnType[T : ClassTag : SingleKeyTypeBase](val typeName: String) @@ -76,10 +93,15 @@ object Column extends StrictLogging { case object TimestampColumn extends RichColumnType[Long]("ts") case object MapColumn extends RichColumnType[UTF8Map]("map") case object BinaryRecordColumn extends RichColumnType[ZeroCopyUTF8String]("br") - // TODO: find a way to annotate histograms as rate-based (Prometheus increasing over time) or - // non-increasing over time - // These histograms for now are non-increasing over time (but increasing from bucket to bucket) - case object HistogramColumn extends RichColumnType[bv.Histogram]("hist") + // HistogramColumn requires the following params: + // counter=[true,false] # If true, histogram values increase over time (eg Prometheus histograms) + case object HistogramColumn extends RichColumnType[bv.Histogram]("hist") { + override def validateParams(params: Config): Unit Or One[BadSchema] = + Try({ + params.getBoolean("counter"); + () // needed to explicitly return Unit + }).toOr.badMap(x => One(BadColumnParams(x.toString))) + } } val typeNameToColType = ColumnType.values.map { colType => colType.typeName -> colType }.toMap @@ -92,7 +114,7 @@ object Column extends StrictLogging { */ def columnsToKeyType(columns: Seq[Column]): KeyType = columns match { case Nil => throw new IllegalArgumentException("Empty columns supplied") - case Seq(DataColumn(_, _, columnType)) => columnType.keyType + case Seq(DataColumn(_, _, columnType, _) ) => columnType.keyType case Seq(ComputedColumn(_, _, _, columnType, _, _)) => columnType.keyType case cols: Seq[Column] => val keyTypes = cols.map { col => columnsToKeyType(Seq(col)).asInstanceOf[SingleKeyType] } @@ -103,12 +125,11 @@ object Column extends StrictLogging { * Converts a list of data columns to Filo VectorInfos for building Filo vectors */ def toFiloSchema(columns: Seq[Column]): Seq[VectorInfo] = columns.collect { - case DataColumn(_, name, colType) => VectorInfo(name, colType.clazz) + case DataColumn(_, name, colType, _) => VectorInfo(name, colType.clazz) } import OptionSugar._ - import Dataset._ val illicitCharsRegex = "[:() ,\u0001]+"r /** @@ -126,24 +147,29 @@ object Column extends StrictLogging { * @param nextId the next column ID to use * @return Good(DataColumn) or Bad(BadSchema) */ - def validateColumn(name: String, typeName: String, nextId: Int): DataColumn Or One[BadSchema] = + def validateColumn(name: String, typeName: String, nextId: Int, params: Config): DataColumn Or One[BadSchema] = for { nothing <- validateColumnName(name) colType <- typeNameToColType.get(typeName).toOr(One(BadColumnType(typeName))) } - yield { DataColumn(nextId, name, colType) } + yield { DataColumn(nextId, name, colType, params) } import Accumulation._ /** - * Creates and validates a set of DataColumns from a list of "columnName:columnType" strings - * @param nameTypeList a Seq of "columnName:columnType" strings. Valid types are in ColumnType + * Creates and validates a set of DataColumns from a list of "columnName:columnType:params" strings. + * Validation errors (in the form of BadSchema) are returned for malformed input, illegal column names, + * unknown types, or params that don't parse correctly. Some column types (such as histogram) have required params. + * @param nameTypeList a Seq of "columnName:columnType:params" strings. Valid types are in ColumnType * @param startingId column IDs are assigned starting with startingId incrementally */ def makeColumnsFromNameTypeList(nameTypeList: Seq[String], startingId: Int = 0): Seq[Column] Or BadSchema = nameTypeList.zipWithIndex.map { case (nameType, idx) => val parts = nameType.split(':') - for { nameAndType <- if (parts.size == 2) Good((parts(0), parts(1))) else Bad(One(NotNameColonType(nameType))) + for { nameAndType <- if (parts.size >= 2) Good((parts(0), parts(1))) else Bad(One(NotNameColonType(nameType))) + params <- Try(ConfigFactory.parseString(parts.drop(2).mkString("\n"))).toOr + .badMap(x => One(BadColumnParams(x.toString))) (name, colType) = nameAndType - col <- validateColumn(name, colType, startingId + idx) } + col <- validateColumn(name, colType, startingId + idx, params) + _ <- col.columnType.validateParams(params) } yield { col } }.combined.badMap { errs => ColumnErrors(errs.toSeq) } } \ No newline at end of file diff --git a/core/src/main/scala/filodb.core/metadata/ComputedColumn.scala b/core/src/main/scala/filodb.core/metadata/ComputedColumn.scala index 1ba475d4c0..e1e4a9a8d5 100644 --- a/core/src/main/scala/filodb.core/metadata/ComputedColumn.scala +++ b/core/src/main/scala/filodb.core/metadata/ComputedColumn.scala @@ -2,6 +2,7 @@ package filodb.core.metadata import scala.language.existentials +import com.typesafe.config.Config import org.scalactic._ import filodb.core.KeyType @@ -17,6 +18,7 @@ case class ComputedColumn(id: Int, sourceIndices: Seq[Int], // index into schema of source column val extractor: TypedFieldExtractor[_]) extends Column { def name: String = expr + def params: Config = ??? } object ComputedColumn { diff --git a/core/src/main/scala/filodb.core/metadata/Dataset.scala b/core/src/main/scala/filodb.core/metadata/Dataset.scala index ad8536ffba..b7725dfc69 100644 --- a/core/src/main/scala/filodb.core/metadata/Dataset.scala +++ b/core/src/main/scala/filodb.core/metadata/Dataset.scala @@ -256,6 +256,7 @@ object Dataset { case class BadColumnType(colType: String) extends BadSchema case class BadColumnName(colName: String, reason: String) extends BadSchema case class NotNameColonType(nameTypeString: String) extends BadSchema + case class BadColumnParams(msg: String) extends BadSchema case class ColumnErrors(errs: Seq[BadSchema]) extends BadSchema case class UnknownRowKeyColumn(keyColumn: String) extends BadSchema case class IllegalMapColumn(reason: String) extends BadSchema @@ -328,8 +329,8 @@ object Dataset { /** * Creates and validates a new Dataset * @param name The name of the dataset - * @param partitionColNameTypes list of partition columns in name:type form - * @param dataColNameTypes list of data columns in name:type form + * @param partitionColNameTypes list of partition columns in name:type[:params] form + * @param dataColNameTypes list of data columns in name:type[:params] form * @param keyColumnNames the key column names, no :type * @return Good(Dataset) or Bad(BadSchema) */ diff --git a/core/src/main/scala/filodb.core/query/KeyFilter.scala b/core/src/main/scala/filodb.core/query/KeyFilter.scala index 7fddc1623d..3191465eb8 100644 --- a/core/src/main/scala/filodb.core/query/KeyFilter.scala +++ b/core/src/main/scala/filodb.core/query/KeyFilter.scala @@ -84,7 +84,7 @@ object KeyFilter { def mapColumns(columns: Seq[Column], columnNames: Seq[String]): Map[String, (Int, Column)] = { columns.zipWithIndex.collect { - case d @ (DataColumn(_, name, _), idx) => name -> (idx -> d._1) + case d @ (DataColumn(_, name, _, _), idx) => name -> (idx -> d._1) }.toMap.filterKeys { name => columnNames.contains(name) } } diff --git a/core/src/test/scala/filodb.core/TestData.scala b/core/src/test/scala/filodb.core/TestData.scala index 743920ea8b..adae778c24 100644 --- a/core/src/test/scala/filodb.core/TestData.scala +++ b/core/src/test/scala/filodb.core/TestData.scala @@ -57,7 +57,7 @@ object TestData { object NamesTestData { def mapper(rows: Seq[Product]): Seq[RowReader] = rows.map(TupleRowReader) - val dataColSpecs = Seq("first:string", "last:string", "age:long") + val dataColSpecs = Seq("first:string", "last:string", "age:long:interval=10") val dataset = Dataset("dataset", Seq("seg:int"), dataColSpecs, "age") // NOTE: first 3 columns are the data columns, thus names could be used for either complete record @@ -297,7 +297,7 @@ object MachineMetricsData { val extraTagsLen = extraTags.map { case (k, v) => k.numBytes + v.numBytes }.sum val histDataset = Dataset("histogram", Seq("tags:map"), - Seq("timestamp:ts", "count:long", "sum:long", "h:hist")) + Seq("timestamp:ts", "count:long", "sum:long", "h:hist:counter=false")) var histBucketScheme: bv.HistogramBuckets = _ def linearHistSeries(startTs: Long = 100000L, numSeries: Int = 10, timeStep: Int = 1000, numBuckets: Int = 8): diff --git a/core/src/test/scala/filodb.core/downsample/ShardDownsamplerSpec.scala b/core/src/test/scala/filodb.core/downsample/ShardDownsamplerSpec.scala index cfe227eebc..da76c1e532 100644 --- a/core/src/test/scala/filodb.core/downsample/ShardDownsamplerSpec.scala +++ b/core/src/test/scala/filodb.core/downsample/ShardDownsamplerSpec.scala @@ -26,7 +26,7 @@ class ShardDownsamplerSpec extends FunSpec with Matchers with BeforeAndAfterAll val customDataset = Dataset.make("custom2", Seq("name:string", "namespace:string", "instance:string"), - Seq("timestamp:ts", "count:double", "min:double", "max:double", "total:double", "avg:double", "h:hist"), + Seq("timestamp:ts", "count:double", "min:double", "max:double", "total:double", "avg:double", "h:hist:counter=false"), Seq("timestamp"), Seq("tTime(0)", "dSum(1)", "dMin(2)", "dMax(3)", "dSum(4)", "dAvgAc(5@1)", "hSum(6)"), DatasetOptions(Seq("name", "namespace"), "name", "total")).get diff --git a/core/src/test/scala/filodb.core/metadata/ColumnSpec.scala b/core/src/test/scala/filodb.core/metadata/ColumnSpec.scala index 9e95c9162d..87516adcf4 100644 --- a/core/src/test/scala/filodb.core/metadata/ColumnSpec.scala +++ b/core/src/test/scala/filodb.core/metadata/ColumnSpec.scala @@ -1,5 +1,7 @@ package filodb.core.metadata +import com.typesafe.config.ConfigFactory + import org.scalatest.{FunSpec, Matchers} class ColumnSpec extends FunSpec with Matchers { @@ -8,6 +10,10 @@ class ColumnSpec extends FunSpec with Matchers { val firstColumn = DataColumn(0, "first", ColumnType.StringColumn) val ageColumn = DataColumn(2, "age", ColumnType.IntColumn) + val histColumnOpts = DataColumn(3, "hist", ColumnType.HistogramColumn, + ConfigFactory.parseString("counter = true")) + val histColumn2 = DataColumn(4, "h2", ColumnType.HistogramColumn, + ConfigFactory.parseString("counter = true\nsize=20000")) describe("Column validations") { it("should check that regular column names don't have : in front") { @@ -33,6 +39,8 @@ class ColumnSpec extends FunSpec with Matchers { it("should serialize and deserialize properly") { DataColumn.fromString(firstColumn.toString) should equal (firstColumn) DataColumn.fromString(ageColumn.toString) should equal (ageColumn) + DataColumn.fromString(histColumnOpts.toString) should equal (histColumnOpts) + DataColumn.fromString(histColumn2.toString) should equal (histColumn2) } } } \ No newline at end of file diff --git a/core/src/test/scala/filodb.core/metadata/DatasetSpec.scala b/core/src/test/scala/filodb.core/metadata/DatasetSpec.scala index 9942ed6aef..09853207a1 100644 --- a/core/src/test/scala/filodb.core/metadata/DatasetSpec.scala +++ b/core/src/test/scala/filodb.core/metadata/DatasetSpec.scala @@ -21,6 +21,31 @@ class DatasetSpec extends FunSpec with Matchers { } } + it("should return BadColumnParams if name:type:params portion not valid key=value pairs") { + val resp1 = Dataset.make("dataset", Seq("part:string"), dataColSpecs :+ "column2:a:b", Seq("age")) + resp1.isBad shouldEqual true + resp1.swap.get shouldBe a[ColumnErrors] + val errors = resp1.swap.get.asInstanceOf[ColumnErrors].errs + errors should have length 1 + errors.head shouldBe a[BadColumnParams] + } + + it("should return BadColumnParams if required param config not specified") { + val resp1 = Dataset.make("dataset", Seq("part:string"), dataColSpecs :+ "h:hist:foo=bar", Seq("age")) + resp1.isBad shouldEqual true + resp1.swap.get shouldBe a[ColumnErrors] + val errors = resp1.swap.get.asInstanceOf[ColumnErrors].errs + errors should have length 1 + errors.head shouldBe a[BadColumnParams] + + val resp2 = Dataset.make("dataset", Seq("part:string"), dataColSpecs :+ "h:hist:counter=bar", Seq("age")) + resp2.isBad shouldEqual true + resp2.swap.get shouldBe a[ColumnErrors] + val errors2 = resp2.swap.get.asInstanceOf[ColumnErrors].errs + errors2 should have length 1 + errors2.head shouldBe a[BadColumnParams] + } + it("should return BadColumnName if illegal chars in column name") { val resp1 = Dataset.make("dataset", Seq("part:string"), Seq("col, umn1:string"), Seq("age")) resp1.isBad shouldEqual true diff --git a/spark/src/main/scala/filodb.spark/TypeConverters.scala b/spark/src/main/scala/filodb.spark/TypeConverters.scala index 77175c4c3d..8586218bfa 100644 --- a/spark/src/main/scala/filodb.spark/TypeConverters.scala +++ b/spark/src/main/scala/filodb.spark/TypeConverters.scala @@ -14,7 +14,7 @@ object TypeConverters { ) def columnsToSqlFields(columns: Seq[Column]): Seq[StructField] = - columns.map { case DataColumn(_, name, colType) => + columns.map { case DataColumn(_, name, colType, _) => StructField(name, colTypeToSqlType(colType), true) } From 459f06be04dad0dc38992292680644e23984f19f Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Tue, 19 Mar 2019 17:04:18 -0700 Subject: [PATCH 22/41] feat(memory, core): Add logs/stats for block manager when out of memory --- memory/src/main/scala/filodb.memory/BlockManager.scala | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/memory/src/main/scala/filodb.memory/BlockManager.scala b/memory/src/main/scala/filodb.memory/BlockManager.scala index 14a807207b..7096858a8b 100644 --- a/memory/src/main/scala/filodb.memory/BlockManager.scala +++ b/memory/src/main/scala/filodb.memory/BlockManager.scala @@ -60,6 +60,7 @@ trait BlockManager { class MemoryStats(tags: Map[String, String]) { val usedBlocksMetric = Kamon.gauge("blockstore-used-blocks").refine(tags) val freeBlocksMetric = Kamon.gauge("blockstore-free-blocks").refine(tags) + val requestedBlocksMetric = Kamon.counter("blockstore-blocks-requested").refine(tags) val usedBlocksTimeOrderedMetric = Kamon.gauge("blockstore-used-time-ordered-blocks").refine(tags) val timeOrderedBlocksReclaimedMetric = Kamon.counter("blockstore-time-ordered-blocks-reclaimed").refine(tags) val blocksReclaimedMetric = Kamon.counter("blockstore-blocks-reclaimed").refine(tags) @@ -126,6 +127,7 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, lock.lock() try { val num: Int = Math.ceil(memorySize / blockSizeInBytes).toInt + stats.requestedBlocksMetric.increment(num) if (freeBlocks.size < num) tryReclaim(num) @@ -138,6 +140,7 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, } allocated } else { + logger.warn(s"Out of blocks to allocate! num_blocks=$num num_bytes=$memorySize freeBlocks=${freeBlocks.size}") Seq.empty[Block] } } finally { @@ -186,6 +189,11 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, if (entry.getValue.isEmpty) timeOrderedListIt.remove() } if (reclaimed < num) reclaimFrom(usedBlocks, stats.blocksReclaimedMetric) + // if we do not get required blocks even after reclaim call + if (reclaimed < num) { + logger.warn(s"$num blocks to reclaim but only reclaimed $reclaimed. usedblocks=${usedBlocks.size} " + + s"usedBlocksTimeOrdered=${usedBlocksTimeOrdered.asScala.toList.map{case(n,l) => (n, l.size)}}") + } def reclaimFrom(list: util.LinkedList[Block], reclaimedCounter: Counter): Unit = { val entries = list.iterator From 337a28bea6ec34e38187d600e92b0c12637259f8 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Tue, 19 Mar 2019 18:31:19 -0700 Subject: [PATCH 23/41] feat(core): Correct support for intermittently publishing time series (#282) Correctly support intermittently publishing metrics by assigning previously assigned partId instead of a duplicate partId for the same partKey. We do this by looking up the evicted re-ingesting partKey in Lucene by translating it into Seq[ColumnFilters]. Costly lookup is first gated using a bloom filter. Other bugs fixed: * Increased akka serialized message limitation * ODPed partition was being assigned a new partId incorrectly. Now ODPed partition uses the same partId. --- conf/logback-dev.xml | 9 +- conf/timeseries-dev-source.conf | 13 ++ core/src/main/resources/filodb-defaults.conf | 6 +- .../memstore/DemandPagedChunkStore.scala | 6 +- .../memstore/OnDemandPagingShard.scala | 8 +- .../memstore/PartKeyLuceneIndex.scala | 8 +- .../memstore/TimeSeriesShard.scala | 117 +++++++++++++++--- .../filodb.core/store/IngestionConfig.scala | 10 +- .../memstore/TimeSeriesMemStoreSpec.scala | 48 ++++++- project/FiloBuild.scala | 1 + .../query/exec/HistogramQuantileMapper.scala | 3 +- 11 files changed, 191 insertions(+), 38 deletions(-) diff --git a/conf/logback-dev.xml b/conf/logback-dev.xml index 440700c24e..88b81cdd38 100644 --- a/conf/logback-dev.xml +++ b/conf/logback-dev.xml @@ -8,11 +8,14 @@ - - + + - + + + + diff --git a/conf/timeseries-dev-source.conf b/conf/timeseries-dev-source.conf index f8ce6b1baa..961f17c2d4 100644 --- a/conf/timeseries-dev-source.conf +++ b/conf/timeseries-dev-source.conf @@ -59,6 +59,19 @@ # Amount of time to delay before retrying # retry-delay = 15s + + # Capacity of Bloom filter used to track evicted partitions. + # Tune this based on how much time series churn is expected before a FiloDB node + # will be restarted for upgrade/maintenance. Do not take into account churn created by + # time series that are purged due to retention. When a time series is not ingesting for retention + # period, it is purged, not evicted. Purged PartKeys are not added to Bloom Filter. + # + # To calculate Bloom Filter size: + # console> BloomFilter[String](5000000, falsePositiveRate = 0.01).numberOfBits + # res9: Long = 47925292 + # Thats about 6MB + evicted-pk-bloom-filter-capacity = 50000 + } downsample { # can be disabled by setting this flag to false diff --git a/core/src/main/resources/filodb-defaults.conf b/core/src/main/resources/filodb-defaults.conf index 9b5a6138a7..d4f72108bf 100644 --- a/core/src/main/resources/filodb-defaults.conf +++ b/core/src/main/resources/filodb-defaults.conf @@ -274,9 +274,9 @@ akka { # Leave out the hostname, it will be automatically determined. # The Akka port will be overridden by filodb.spark.* settings port = 0 - send-buffer-size = 512000b - receive-buffer-size = 512000b - maximum-frame-size = 10 MiB + send-buffer-size = 1024000b + receive-buffer-size = 1024000b + maximum-frame-size = 25 MiB } } } diff --git a/core/src/main/scala/filodb.core/memstore/DemandPagedChunkStore.scala b/core/src/main/scala/filodb.core/memstore/DemandPagedChunkStore.scala index 27eed2b621..f0ce48be28 100644 --- a/core/src/main/scala/filodb.core/memstore/DemandPagedChunkStore.scala +++ b/core/src/main/scala/filodb.core/memstore/DemandPagedChunkStore.scala @@ -80,9 +80,9 @@ extends RawToPartitionMaker with StrictLogging { val inserted = tsPart.addChunkInfoIfAbsent(chunkID, infoAddr) if (!inserted) { - logger.info(s"Chunks not copied to ${tsPart.stringPartition}, already has chunk $chunkID. " + - s"Chunk time range (${ChunkSetInfo.getStartTime(infoBytes)}, ${ChunkSetInfo.getEndTime(infoBytes)})" + - s" partition earliestTime=${tsPart.earliestTime}") + logger.info(s"Chunks not copied to partId=${tsPart.partID} ${tsPart.stringPartition}, already has chunk " + + s"$chunkID. Chunk time range (${ChunkSetInfo.getStartTime(infoBytes)}, " + + s"${ChunkSetInfo.getEndTime(infoBytes)}) partition earliestTime=${tsPart.earliestTime}") } } tsPart diff --git a/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala b/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala index ea9b96e56a..13a009522f 100644 --- a/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala +++ b/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala @@ -38,7 +38,7 @@ TimeSeriesShard(dataset, storeConfig, shardNum, rawStore, metastore, evictionPol // TODO: make this configurable private val strategy = OverflowStrategy.BackPressure(1000) - private def startODPSpan: Span = Kamon.buildSpan(s"odp-cassandra-latency") + private def startODPSpan(): Span = Kamon.buildSpan(s"odp-cassandra-latency") .withTag("dataset", dataset.name) .withTag("shard", shardNum) .start() @@ -88,7 +88,7 @@ TimeSeriesShard(dataset, storeConfig, shardNum, rawStore, metastore, evictionPol val multiPart = MultiPartitionScan(partKeyBytesToPage, shardNum) shardStats.partitionsQueried.increment(partKeyBytesToPage.length) if (partKeyBytesToPage.nonEmpty) { - val span = startODPSpan + val span = startODPSpan() rawStore.readRawPartitions(dataset, allDataCols, multiPart, computeBoundingMethod(methods)) // NOTE: this executes the partMaker single threaded. Needed for now due to concurrency constraints. // In the future optimize this if needed. @@ -102,7 +102,7 @@ TimeSeriesShard(dataset, storeConfig, shardNum, rawStore, metastore, evictionPol Observable.fromTask(odpPartTask(indexIt, partKeyBytesToPage, methods, chunkMethod)).flatMap { odpParts => shardStats.partitionsQueried.increment(partKeyBytesToPage.length) if (partKeyBytesToPage.nonEmpty) { - val span = startODPSpan + val span = startODPSpan() Observable.fromIterable(partKeyBytesToPage.zip(methods)) .mapAsync(storeConfig.demandPagingParallelism) { case (partBytes, method) => rawStore.readRawPartitions(dataset, allDataCols, SinglePartitionScan(partBytes, shardNum), method) @@ -151,7 +151,7 @@ TimeSeriesShard(dataset, storeConfig, shardNum, rawStore, metastore, evictionPol for { partKeyBytesRef <- partKeyIndex.partKeyFromPartId(id) unsafeKeyOffset = PartKeyLuceneIndex.bytesRefToUnsafeOffset(partKeyBytesRef.offset) group = partKeyGroup(dataset.partKeySchema, partKeyBytesRef.bytes, unsafeKeyOffset, numGroups) - part <- Option(createNewPartition(partKeyBytesRef.bytes, unsafeKeyOffset, group, 4)) } yield { + part <- Option(createNewPartition(partKeyBytesRef.bytes, unsafeKeyOffset, group, id, 4)) } yield { val stamp = partSetLock.writeLock() try { partSet.add(part) diff --git a/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala b/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala index fdf76100f0..a2dea2638a 100644 --- a/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala +++ b/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala @@ -418,6 +418,12 @@ class PartKeyLuceneIndex(dataset: Dataset, def partIdsFromFilters(columnFilters: Seq[ColumnFilter], startTime: Long, endTime: Long): IntIterator = { + partIdsFromFilters2(columnFilters, startTime, endTime).intIterator() + } + + def partIdsFromFilters2(columnFilters: Seq[ColumnFilter], + startTime: Long, + endTime: Long): PartIdCollector = { val partKeySpan = Kamon.buildSpan("index-partition-lookup-latency") .withTag("dataset", dataset.name) .withTag("shard", shardNum) @@ -435,7 +441,7 @@ class PartKeyLuceneIndex(dataset: Dataset, val collector = new PartIdCollector() // passing zero for unlimited results searcher.search(query, collector) partKeySpan.finish() - collector.intIterator() + collector } } diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala index c50005d42f..db2e60dea8 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala @@ -6,6 +6,8 @@ import scala.concurrent.{Await, ExecutionContext, Future} import scala.concurrent.duration._ import scala.util.{Random, Try} +import bloomfilter.CanGenerateHashFrom +import bloomfilter.mutable.BloomFilter import com.googlecode.javaewah.{EWAHCompressedBitmap, IntIterator} import com.typesafe.scalalogging.StrictLogging import debox.Buffer @@ -23,7 +25,7 @@ import filodb.core.binaryrecord2._ import filodb.core.downsample.{DownsampleConfig, DownsamplePublisher, ShardDownsampler} import filodb.core.metadata.Column.ColumnType import filodb.core.metadata.Dataset -import filodb.core.query.{ColumnFilter, ColumnInfo} +import filodb.core.query.{ColumnFilter, ColumnInfo, Filter} import filodb.core.store._ import filodb.memory._ import filodb.memory.format.{UnsafeUtils, ZeroCopyUTF8String} @@ -82,6 +84,11 @@ class TimeSeriesShardStats(dataset: DatasetRef, shardNum: Int) { val bufferPoolSize = Kamon.gauge("memstore-writebuffer-pool-size").refine(tags) val indexEntries = Kamon.gauge("memstore-index-entries").refine(tags) val indexBytes = Kamon.gauge("memstore-index-ram-bytes").refine(tags) + + val evictedPartKeyBloomFilterQueries = Kamon.counter("evicted-pk-bloom-filter-queries").refine(tags) + val evictedPartKeyBloomFilterFalsePositives = Kamon.counter("evicted-pk-bloom-filter-fp").refine(tags) + val evictedPkBloomFilterSize = Kamon.gauge("evicted-pk-bloom-filter-approx-size").refine(tags) + val evictedPartIdLookupMultiMatch = Kamon.counter("evicted-partId-lookup-multi-match").refine(tags) } object TimeSeriesShard { @@ -130,6 +137,9 @@ object TimeSeriesShard { def partKeyGroup(schema: RecordSchema, partKeyBase: Any, partKeyOffset: Long, numGroups: Int): Int = { Math.abs(schema.partitionHash(partKeyBase, partKeyOffset) % numGroups) } + + private[memstore] final case class PartKey(base: Any, offset: Long) + private[memstore] final val CREATE_NEW_PARTID = -1 } trait PartitionIterator extends Iterator[TimeSeriesPartition] { @@ -314,6 +324,13 @@ class TimeSeriesShard(val dataset: Dataset, private final val shardDownsampler = new ShardDownsampler(dataset, shardNum, downsampleConfig.enabled, downsampleConfig.resolutions, downsamplePublisher, shardStats) + private[memstore] val evictedPartKeys = + BloomFilter[PartKey](storeConfig.evictedPkBfCapacity, falsePositiveRate = 0.01)(new CanGenerateHashFrom[PartKey] { + override def generateHash(from: PartKey): Long = { + dataset.partKeySchema.partitionHash(from.base, from.offset) + } + }) + case class InMemPartitionIterator(intIt: IntIterator) extends PartitionIterator { var nextPart = UnsafeUtils.ZeroPointer.asInstanceOf[TimeSeriesPartition] val skippedPartIDs = debox.Buffer.empty[Int] @@ -447,7 +464,7 @@ class TimeSeriesShard(val dataset: Dataset, // We cant look it up in lucene because we havent flushed index yet val partId = partSet.getWithPartKeyBR(partKeyBaseOnHeap, partKeyOffset) match { case None => val group = partKeyGroup(dataset.partKeySchema, partKeyBaseOnHeap, partKeyOffset, numGroups) - val part = createNewPartition(partKeyBaseOnHeap, partKeyOffset, group, 4) + val part = createNewPartition(partKeyBaseOnHeap, partKeyOffset, group, CREATE_NEW_PARTID, 4) // In theory, we should not get an OutOfMemPartition here since // it should have occurred before node failed too, and with data sropped, // index would not be updated. But if for some reason we see it, drop data @@ -653,7 +670,7 @@ class TimeSeriesShard(val dataset: Dataset, shardStats.numActivelyIngestingParts.set(cardinality) } - private def addPartKeyToTimebucket(indexRb: RecordBuilder, p: TimeSeriesPartition) = { + private def addPartKeyToTimebucketRb(indexRb: RecordBuilder, p: TimeSeriesPartition) = { var startTime = partKeyIndex.startTimeFromPartId(p.partID) if (startTime == -1) startTime = p.earliestTime// can remotely happen since lucene reads are eventually consistent val endTime = if (isActivelyIngesting(p.partID)) { @@ -667,7 +684,7 @@ class TimeSeriesShard(val dataset: Dataset, indexRb.addLong(endTime) // Need to add 4 to include the length bytes indexRb.addBlob(p.partKeyBase, p.partKeyOffset, BinaryRegionLarge.numBytes(p.partKeyBase, p.partKeyOffset) + 4) - logger.debug(s"Added into timebucket partId ${p.partID} in dataset=${dataset.ref} shard=$shardNum " + + logger.debug(s"Added into timebucket RB partId ${p.partID} in dataset=${dataset.ref} shard=$shardNum " + s"partKey[${p.stringPartition}] with startTime=$startTime endTime=$endTime") indexRb.endRecord(false) } @@ -797,7 +814,7 @@ class TimeSeriesShard(val dataset: Dataset, /* create time bucket using record builder */ val timeBucketRb = new RecordBuilder(MemFactory.onHeapFactory, indexTimeBucketSchema, indexTimeBucketSegmentSize) InMemPartitionIterator(timeBucketBitmaps.get(cmd.timeBucket).intIterator).foreach { p => - addPartKeyToTimebucket(timeBucketRb, p) + addPartKeyToTimebucketRb(timeBucketRb, p) } val numPartKeysInBucket = timeBucketBitmaps.get(cmd.timeBucket).cardinality() logger.debug(s"Number of records in timebucket=${cmd.timeBucket} of " + @@ -923,31 +940,76 @@ class TimeSeriesShard(val dataset: Dataset, private[filodb] def getOrAddPartition(recordBase: Any, recordOff: Long, group: Int, ingestOffset: Long) = { var part = partSet.getWithIngestBR(recordBase, recordOff) if (part == null) { - part = addPartition(recordBase, recordOff, group, ingestOffset) + part = addPartition(recordBase, recordOff, group) } part } // scalastyle:on - private def addPartition(recordBase: Any, recordOff: Long, group: Int, ingestOffset: Long) = { + /** + * Looks up the previously assigned partId of a possibly evicted partition. + * @return partId >=0 if one is found, CREATE_NEW_PARTID (-1) if not found. + */ + private def lookupPreviouslyAssignedPartId(partKeyBase: Array[Byte], partKeyOffset: Long): Int = { + shardStats.evictedPartKeyBloomFilterQueries.increment() + if (evictedPartKeys.mightContain(PartKey(partKeyBase, partKeyOffset))) { + val filters = dataset.partKeySchema.toStringPairs(partKeyBase, partKeyOffset) + .map { pair => ColumnFilter(pair._1, Filter.Equals(pair._2)) } + val matches = partKeyIndex.partIdsFromFilters2(filters, 0, Long.MaxValue) + matches.result.cardinality() match { + case 0 => shardStats.evictedPartKeyBloomFilterFalsePositives.increment() + CREATE_NEW_PARTID + case c if c >= 1 => // NOTE: if we hit one partition, we cannot directly call it out as the result without + // verifying the partKey since the matching partition may have had an additional tag + if (c > 1) shardStats.evictedPartIdLookupMultiMatch.increment() + val iter = matches.result.intIterator() + var partId = -1 + do { + // find the most specific match for the given ingestion record + val nextPartId = iter.next + partKeyIndex.partKeyFromPartId(nextPartId).foreach { candidate => + if (dataset.partKeySchema.equals(partKeyBase, partKeyOffset, + candidate.bytes, PartKeyLuceneIndex.bytesRefToUnsafeOffset(candidate.offset))) { + partId = nextPartId + logger.debug(s"There is already a partId $partId assigned for " + + s"${dataset.partKeySchema.stringify(partKeyBase, partKeyOffset)}") + } + } + } while (iter.hasNext && partId != -1) + if (partId == CREATE_NEW_PARTID) + shardStats.evictedPartKeyBloomFilterFalsePositives.increment() + partId + } + } else CREATE_NEW_PARTID + } + + /** + * Adds new partition with appropriate partId. If it is a newly seen partKey, then new partId is assigned. + * If it is a previously seen partKey that is already in index, it reassigns same partId so that indexes + * are still valid. + * + * This method also updates lucene index and time bucket bitmaps properly. + */ + private def addPartition(recordBase: Any, recordOff: Long, group: Int) = { val partKeyOffset = recordComp.buildPartKeyFromIngest(recordBase, recordOff, partKeyBuilder) - val newPart = createNewPartition(partKeyArray, partKeyOffset, group) + val previousPartId = lookupPreviouslyAssignedPartId(partKeyArray, partKeyOffset) + val newPart = createNewPartition(partKeyArray, partKeyOffset, group, previousPartId) if (newPart != OutOfMemPartition) { val partId = newPart.partID // NOTE: Don't use binRecordReader here. recordOffset might not be set correctly val startTime = dataset.ingestionSchema.getLong(recordBase, recordOff, timestampColId) - partKeyIndex.addPartKey(newPart.partKeyBytes, partId, startTime)() - timeBucketBitmaps.get(currentIndexTimeBucket).set(partId) - activelyIngesting.synchronized { activelyIngesting.set(partId) } + if (previousPartId == CREATE_NEW_PARTID) { + // add new lucene entry if this partKey was never seen before + partKeyIndex.addPartKey(newPart.partKeyBytes, partId, startTime)() + } + timeBucketBitmaps.get(currentIndexTimeBucket).set(partId) // causes current time bucket to include this partId + activelyIngesting.synchronized { activelyIngesting.set(partId) } // causes endTime to be set to Long.MaxValue val stamp = partSetLock.writeLock() try { partSet.add(newPart) } finally { partSetLock.unlockWrite(stamp) } - - logger.trace(s"Created new partition ${newPart.stringPartition} on dataset=${dataset.ref} " + - s"shard $shardNum at offset $ingestOffset") } newPart } @@ -973,8 +1035,15 @@ class TimeSeriesShard(val dataset: Dataset, s"shard=$shardNum", e); disableAddPartitions() } + /** + * Creates new partition and adds them to the shard data structures. DOES NOT update + * lucene index. It is the caller's responsibility to add or skip that step depending on the situation. + * + * @param usePartId pass CREATE_NEW_PARTID to force creation of new partId instead of using one that is passed in + */ protected def createNewPartition(partKeyBase: Array[Byte], partKeyOffset: Long, - group: Int, initMapSize: Int = initInfoMapSize): TimeSeriesPartition = + group: Int, usePartId: Int, + initMapSize: Int = initInfoMapSize): TimeSeriesPartition = // Check and evict, if after eviction we still don't have enough memory, then don't proceed if (addPartitionsDisabled() || !ensureFreeSpace()) { OutOfMemPartition } else { @@ -982,13 +1051,18 @@ class TimeSeriesShard(val dataset: Dataset, // NOTE: allocateAndCopy and allocNew below could fail if there isn't enough memory. It is CRUCIAL // that min-write-buffers-free setting is large enough to accommodate the below use cases ALWAYS val (_, partKeyAddr, _) = BinaryRegionLarge.allocateAndCopy(partKeyBase, partKeyOffset, bufferMemoryManager) - val partId = nextPartitionID - incrementPartitionID() + val partId = if (usePartId == CREATE_NEW_PARTID) { + val id = nextPartitionID + incrementPartitionID() + id + } else usePartId val newPart = new TimeSeriesPartition( partId, dataset, partKeyAddr, shardNum, bufferPool, shardStats, bufferMemoryManager, initMapSize) partitions.put(partId, newPart) shardStats.partitionsCreated.increment partitionGroups(group).set(partId) + logger.debug(s"Created new partition with partId ${newPart.partID} ${newPart.stringPartition} on " + + s"dataset=${dataset.ref} shard $shardNum") newPart } @@ -1060,9 +1134,13 @@ class TimeSeriesShard(val dataset: Dataset, if (isActivelyIngesting(partitionObj.partID)) logger.warn(s"Partition ${partitionObj.partID} is ingesting, but it was eligible for eviction. How?") if (endTime == PartKeyLuceneIndex.NOT_FOUND || endTime == Long.MaxValue) { - logger.warn(s"endTime ${endTime} was not correct. how?", new IllegalStateException()) + logger.warn(s"endTime $endTime was not correct. how?", new IllegalStateException()) } else { logger.debug(s"Evicting partId=${partitionObj.partID} from dataset=${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.add(PartKey(partitionObj.partKeyBase, partitionObj.partKeyOffset)) + // The previously created PartKey is just meant for bloom filter and will be GCed removePartition(partitionObj) partsRemoved += 1 maxEndTime = Math.max(maxEndTime, endTime) @@ -1071,6 +1149,7 @@ class TimeSeriesShard(val dataset: Dataset, partsSkipped += 1 } } + shardStats.evictedPkBloomFilterSize.set(evictedPartKeys.approximateElementCount()) evictionWatermark = maxEndTime + 1 // Plus one needed since there is a possibility that all partitions evicted in this round have same endTime, // and there may be more partitions that are not evicted with same endTime. If we didnt advance the watermark, @@ -1166,6 +1245,7 @@ class TimeSeriesShard(val dataset: Dataset, logger.info(s"Clearing all MemStore state for dataset=${dataset.ref} shard=$shardNum") partitions.values.asScala.foreach(removePartition) partKeyIndex.reset() + // TODO unable to reset/clear bloom filter ingested = 0L for { group <- 0 until numGroups } { partitionGroups(group) = new EWAHCompressedBitmap() @@ -1174,6 +1254,7 @@ class TimeSeriesShard(val dataset: Dataset, } def shutdown(): Unit = { + evictedPartKeys.dispose() reset() // Not really needed, but clear everything just to be consistent logger.info(s"Shutting down dataset=${dataset.ref} shard=$shardNum") /* Don't explcitly free the memory just yet. These classes instead rely on a finalize diff --git a/core/src/main/scala/filodb.core/store/IngestionConfig.scala b/core/src/main/scala/filodb.core/store/IngestionConfig.scala index dac1a3a7cd..bb8353a92c 100644 --- a/core/src/main/scala/filodb.core/store/IngestionConfig.scala +++ b/core/src/main/scala/filodb.core/store/IngestionConfig.scala @@ -31,7 +31,8 @@ final case class StoreConfig(flushInterval: FiniteDuration, // Use a MultiPartitionScan (instead of single partition at a time) for on-demand paging multiPartitionODP: Boolean, demandPagingParallelism: Int, - demandPagingEnabled: Boolean) { + demandPagingEnabled: Boolean, + evictedPkBfCapacity: Int) { import collection.JavaConverters._ def toConfig: Config = ConfigFactory.parseMap(Map("flush-interval" -> (flushInterval.toSeconds + "s"), @@ -51,7 +52,8 @@ final case class StoreConfig(flushInterval: FiniteDuration, "part-index-flush-min-delay" -> (partIndexFlushMinDelaySeconds + "s"), "multi-partition-odp" -> multiPartitionODP, "demand-paging-parallelism" -> demandPagingParallelism, - "demand-paging-enabled" -> demandPagingEnabled).asJava) + "demand-paging-enabled" -> demandPagingEnabled, + "evicted-pk-bloom-filter-capacity" -> evictedPkBfCapacity).asJava) } final case class AssignShardConfig(address: String, shardList: Seq[Int]) @@ -77,6 +79,7 @@ object StoreConfig { |multi-partition-odp = false |demand-paging-parallelism = 4 |demand-paging-enabled = true + |evicted-pk-bloom-filter-capacity = 5000000 |""".stripMargin) /** Pass in the config inside the store {} */ def apply(storeConfig: Config): StoreConfig = { @@ -98,7 +101,8 @@ object StoreConfig { config.as[FiniteDuration]("part-index-flush-min-delay").toSeconds.toInt, config.getBoolean("multi-partition-odp"), config.getInt("demand-paging-parallelism"), - config.getBoolean("demand-paging-enabled")) + config.getBoolean("demand-paging-enabled"), + config.getInt("evicted-pk-bloom-filter-capacity")) } } diff --git a/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala b/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala index ba89a16d42..ef52a9399e 100644 --- a/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala +++ b/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala @@ -12,7 +12,7 @@ import org.scalatest.time.{Millis, Seconds, Span} import filodb.core._ import filodb.core.binaryrecord2.{RecordBuilder, RecordContainer} -import filodb.core.memstore.TimeSeriesShard.{indexTimeBucketSchema, indexTimeBucketSegmentSize} +import filodb.core.memstore.TimeSeriesShard.{indexTimeBucketSchema, indexTimeBucketSegmentSize, PartKey} import filodb.core.metadata.Dataset import filodb.core.query.{ColumnFilter, Filter} import filodb.core.store.{FilteredPartitionScan, InMemoryMetaStore, NullColumnStore, SinglePartitionScan} @@ -441,11 +441,55 @@ class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter w .toListL.runAsync .futureValue .asInstanceOf[Seq[TimeSeriesPartition]] - parts.map(_.partID) shouldEqual Seq(22) // newly created partitions get a new ID + parts.map(_.partID) shouldEqual Seq(0) // newly created ODP partitions get earlier partId dataset1.partKeySchema.asJavaString(parts.head.partKeyBase, parts.head.partKeyOffset, 0) shouldEqual "Series 0" memStore.numPartitions(dataset1.ref, 0) shouldEqual 21 } + it("should assign same previously assigned partId using bloom filter when evicted series starts re-ingesting") { + memStore.setup(dataset1, 0, TestData.storeConf) + + // Ingest normal multi series data with 10 partitions. Should have 10 partitions. + val data = records(dataset1, linearMultiSeries().take(10)) + memStore.ingest(dataset1.ref, 0, data) + + memStore.commitIndexForTesting(dataset1.ref) + + val shard0 = memStore.getShard(dataset1.ref, 0).get + val shard0Partitions = shard0.partitions + + memStore.numPartitions(dataset1.ref, 0) shouldEqual 10 + memStore.labelValues(dataset1.ref, 0, "series").toSeq should have length (10) + var part0 = shard0Partitions.get(0) + dataset1.partKeySchema.asJavaString(part0.partKeyBase, part0.partKeyOffset, 0) shouldEqual "Series 0" + val pkBytes = dataset1.partKeySchema.asByteArray(part0.partKeyBase, part0.partKeyOffset) + val pk = PartKey(pkBytes, UnsafeUtils.arayOffset) + shard0.evictedPartKeys.mightContain(pk) shouldEqual false + + // Purposely mark two partitions endTime as occurring a while ago to mark them eligible for eviction + // We also need to switch buffers so that internally ingestionEndTime() is accurate + markPartitionsForEviction(0 to 1) + + // Now, ingest 20 partitions. First two partitions ingested should be evicted. + val data2 = records(dataset1, linearMultiSeries(numSeries = 22).drop(2).take(20)) + memStore.ingest(dataset1.ref, 0, data2) + Thread sleep 1000 // see if this will make things pass sooner + + memStore.numPartitions(dataset1.ref, 0) shouldEqual 20 + + // scalastyle:off null + shard0Partitions.get(0) shouldEqual null // since partId 0 has been evicted + shard0.evictedPartKeys.mightContain(pk) shouldEqual true + + // now re-ingest data for evicted partition with partKey "Series 0" + val data3 = records(dataset1, linearMultiSeries().take(1)) + memStore.ingest(dataset1.ref, 0, data3) + + // the partId assigned should still be 0 + part0 = shard0Partitions.get(0) + dataset1.partKeySchema.asJavaString(part0.partKeyBase, part0.partKeyOffset, 0) shouldEqual "Series 0" + } + it("should be able to skip ingestion/add partitions if there is no more space left") { memStore.setup(dataset1, 0, TestData.storeConf) diff --git a/project/FiloBuild.scala b/project/FiloBuild.scala index b9bbd13695..b6a1607cdf 100644 --- a/project/FiloBuild.scala +++ b/project/FiloBuild.scala @@ -222,6 +222,7 @@ object FiloBuild extends Build { "com.github.rholder.fauxflake" % "fauxflake-core" % "1.1.0", "org.scalactic" %% "scalactic" % "2.2.6" withJavadoc(), "org.apache.lucene" % "lucene-core" % "7.3.0" withJavadoc(), + "com.github.alexandrnikitin" %% "bloom-filter" % "0.11.0", scalaxyDep ) diff --git a/query/src/main/scala/filodb/query/exec/HistogramQuantileMapper.scala b/query/src/main/scala/filodb/query/exec/HistogramQuantileMapper.scala index 31275d15ed..27f9a0047a 100644 --- a/query/src/main/scala/filodb/query/exec/HistogramQuantileMapper.scala +++ b/query/src/main/scala/filodb/query/exec/HistogramQuantileMapper.scala @@ -2,6 +2,7 @@ package filodb.query.exec import monix.reactive.Observable import org.agrona.MutableDirectBuffer +import scalaxy.loops._ import filodb.core.query._ import filodb.memory.format.{RowReader, ZeroCopyUTF8String} @@ -75,7 +76,7 @@ case class HistogramQuantileMapper(funcParams: Seq[Any]) extends RangeVectorTran val row = new TransientRow() override def hasNext: Boolean = samples.forall(_.hasNext) override def next(): RowReader = { - for { i <- samples.indices } { + for { i <- 0 until samples.size optimized } { val nxt = samples(i).next() buckets(i).rate = nxt.getDouble(1) row.timestamp = nxt.getLong(0) From 0ba01b63149c5971cb0fae7ae3fc7d2aba8794c8 Mon Sep 17 00:00:00 2001 From: Jackson Jeyapaul Date: Thu, 21 Mar 2019 16:57:42 -0700 Subject: [PATCH 24/41] misc(core): Reduce zipkin message batchsize (#290) * misc(core): Reduce zipkin message maxBytes out * Fix review comments * Update batch size to 128kb --- .../scala/filodb.core/zipkin/Zipkin.scala | 26 ++++++++++++------- 1 file changed, 16 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/filodb.core/zipkin/Zipkin.scala b/core/src/main/scala/filodb.core/zipkin/Zipkin.scala index b9d203bc51..c0bf13e59d 100644 --- a/core/src/main/scala/filodb.core/zipkin/Zipkin.scala +++ b/core/src/main/scala/filodb.core/zipkin/Zipkin.scala @@ -2,13 +2,15 @@ package filodb.core.zipkin import java.net.InetAddress +import scala.util.Try + import com.typesafe.config.Config import kamon.{Kamon, SpanReporter} import kamon.trace.Span.{FinishedSpan => KamonSpan, Mark, TagValue} import kamon.util.Clock import org.slf4j.LoggerFactory -import scala.util.Try import zipkin2.{Endpoint, Span => ZipkinSpan} +import zipkin2.codec.Encoding import zipkin2.reporter.AsyncReporter import zipkin2.reporter.okhttp3.OkHttpSender @@ -126,15 +128,19 @@ class ZipkinReporter extends SpanReporter { val zipkinHost = Kamon.config().getString(HostConfigKey) val zipkinPort = Kamon.config().getInt(PortConfigKey) - if (zipkinEndpoint == null || zipkinEndpoint.trim.isEmpty) { - AsyncReporter.create( - OkHttpSender.create(s"http://$zipkinHost:$zipkinPort/api/v2/spans") - ) - } else { - AsyncReporter.create( - OkHttpSender.create(s"$zipkinEndpoint/api/v2/spans") - ) - } + val url = if (zipkinEndpoint == null || zipkinEndpoint.trim.isEmpty) + s"http://$zipkinHost:$zipkinPort/api/v2/spans" + else + s"$zipkinEndpoint/api/v2/spans" + + AsyncReporter.create( + OkHttpSender.newBuilder() + .encoding(Encoding.JSON) + .endpoint(url) + .maxRequests(64) + .messageMaxBytes(1024 * 128) + .build() + ) } //scalastyle:on null From 1867b07e4c4123e97898f1dc497b14fe3a38190a Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Fri, 22 Mar 2019 15:25:32 -0700 Subject: [PATCH 25/41] bug(coord): Send missing snapshot message on updateFromExternalShardEvent (#289) There is a missing broadcast of snapshot after updateFromExternalShardEvent. Added missing step. --- .../filodb.coordinator/ShardManager.scala | 1 + .../IngestionStreamSpec.scala | 30 +++++++++++++++++++ .../NodeCoordinatorActorSpec.scala | 8 +++-- .../filodb.coordinator/ShardManagerSpec.scala | 19 ++++++++++++ 4 files changed, 56 insertions(+), 2 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/ShardManager.scala b/coordinator/src/main/scala/filodb.coordinator/ShardManager.scala index 5fb32d92be..b846929831 100644 --- a/coordinator/src/main/scala/filodb.coordinator/ShardManager.scala +++ b/coordinator/src/main/scala/filodb.coordinator/ShardManager.scala @@ -449,6 +449,7 @@ private[coordinator] final class ShardManager(settings: FilodbSettings, // Above condition ensures that we respond to shard events only from the node shard is currently assigned to. // Needed to avoid race conditions where IngestionStopped for an old assignment comes after shard is reassigned. updateFromShardEvent(event) + publishSnapshot(event.ref) // reassign shard if IngestionError. Exclude previous node since it had error shards. event match { case _: IngestionError => diff --git a/coordinator/src/test/scala/filodb.coordinator/IngestionStreamSpec.scala b/coordinator/src/test/scala/filodb.coordinator/IngestionStreamSpec.scala index 8fd624167d..0748692a64 100644 --- a/coordinator/src/test/scala/filodb.coordinator/IngestionStreamSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/IngestionStreamSpec.scala @@ -119,6 +119,14 @@ class IngestionStreamSpec extends ActorTest(IngestionStreamSpec.getNewSystem) wi val invalidShard = -1 coordinatorActor ! StartShardIngestion(dataset6.ref, invalidShard, None) + // first it becomes active + expectMsgPF(within) { + case CurrentShardSnapshot(dataset6.ref, mapper) => + mapper.shardsForCoord(coordinatorActor) shouldEqual Seq(0) + mapper.statuses.head shouldEqual ShardStatusActive + } + + // then it becomes stopped expectMsgPF(within) { case CurrentShardSnapshot(dataset6.ref, mapper) => mapper.shardsForCoord(coordinatorActor) shouldEqual Seq(0) @@ -132,6 +140,13 @@ class IngestionStreamSpec extends ActorTest(IngestionStreamSpec.getNewSystem) wi // this functionality already is built into the shard actor: shardActor ! RemoveSubscription(ref) setup(dataset33.ref, "/GDELT-sample-test.csv", rowsToRead = 5, None) + // first it becomes active + expectMsgPF(within) { + case CurrentShardSnapshot(dataset33.ref, mapper) => + mapper.shardsForCoord(coordinatorActor) shouldEqual Seq(0) + mapper.statuses.head shouldEqual ShardStatusActive + } + // Wait for all messages to be ingested expectMsgPF(within) { case CurrentShardSnapshot(dataset33.ref, mapper) => @@ -175,6 +190,21 @@ class IngestionStreamSpec extends ActorTest(IngestionStreamSpec.getNewSystem) wi // expectMsg(IngestionStopped(dataset33.ref, 0)) // Unfortunately since we do not get every message we cannot actually check the progression of recovery + + for { i <- 0 until 3} { + expectMsgPF(within) { + case CurrentShardSnapshot(dataset33.ref, mapper) => + mapper.shardsForCoord(coordinatorActor) shouldEqual Seq(0) + mapper.statuses.head.isInstanceOf[ShardStatusRecovery] shouldEqual true + } + } + + expectMsgPF(within) { + case CurrentShardSnapshot(dataset33.ref, mapper) => + mapper.shardsForCoord(coordinatorActor) shouldEqual Seq(0) + mapper.statuses.head shouldEqual ShardStatusActive + } + expectMsgPF(within) { case CurrentShardSnapshot(dataset33.ref, mapper) => mapper.shardsForCoord(coordinatorActor) shouldEqual Seq(0) diff --git a/coordinator/src/test/scala/filodb.coordinator/NodeCoordinatorActorSpec.scala b/coordinator/src/test/scala/filodb.coordinator/NodeCoordinatorActorSpec.scala index f5e0dbcc46..274c37cb54 100644 --- a/coordinator/src/test/scala/filodb.coordinator/NodeCoordinatorActorSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/NodeCoordinatorActorSpec.scala @@ -99,8 +99,11 @@ class NodeCoordinatorActorSpec extends ActorTest(NodeCoordinatorActorSpec.getNew val sd = SetupDataset(dataset.ref, resources, noOpSource, TestData.storeConf) shardManager.addDataset(sd, dataset, self) shardManager.subscribe(probe.ref, dataset.ref) - probe.expectMsgPF() { case CurrentShardSnapshot(ds, mapper) => - shardMap = mapper + probe.expectMsgPF() { case CurrentShardSnapshot(ds, mapper) => } // for subscription + for { i <- 0 until numShards } { // for each shard assignment + probe.expectMsgPF() { case CurrentShardSnapshot(ds, mapper) => + shardMap = mapper + } } } @@ -264,6 +267,7 @@ class NodeCoordinatorActorSpec extends ActorTest(NodeCoordinatorActorSpec.getNew it("should concatenate raw series from multiple shards") { val ref = setupTimeSeries(2) + probe.expectMsgPF() { case CurrentShardSnapshot(ds, mapper) => } // Same series is ingested into two shards. I know, this should not happen in real life. probe.send(coordinatorActor, IngestRows(ref, 0, records(dataset1, linearMultiSeries().take(30)))) probe.expectMsg(Ack(0L)) diff --git a/coordinator/src/test/scala/filodb.coordinator/ShardManagerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/ShardManagerSpec.scala index 08b85eaa06..7033d5afb4 100644 --- a/coordinator/src/test/scala/filodb.coordinator/ShardManagerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/ShardManagerSpec.scala @@ -263,6 +263,19 @@ class ShardManagerSpec extends AkkaSpec { coord4.expectMsgAllOf( StartShardIngestion(dataset1, 0, None)) + // We are asserting two snapshot messages of same type below. + // The first one should have been one where shard 0 is unassigned and in error state. + // Subsequently, the reassignment should have caused it to change. + // However, we are forced to assert on the final state because the mutation + // of the shard map object has already happened by the time updateFromExternalShardEvent returns + // and we are able to only validate the final state here on both objects. + subscriber.expectMsgPF() { case s: CurrentShardSnapshot => + s.map.shardsForCoord(coord3.ref) shouldEqual Seq(1, 2) + s.map.shardsForCoord(coord2.ref) shouldEqual Seq(3, 6, 7) + s.map.shardsForCoord(coord4.ref) shouldEqual Seq(0, 4, 5) + s.map.unassignedShards shouldEqual Nil + } + subscriber.expectMsgPF() { case s: CurrentShardSnapshot => s.map.shardsForCoord(coord3.ref) shouldEqual Seq(1, 2) s.map.shardsForCoord(coord2.ref) shouldEqual Seq(3, 6, 7) @@ -279,6 +292,12 @@ class ShardManagerSpec extends AkkaSpec { coord3.expectNoMessage() coord2.expectNoMessage() coord4.expectNoMessage() + subscriber.expectMsgPF() { case s: CurrentShardSnapshot => + s.map.shardsForCoord(coord3.ref) shouldEqual Seq(1, 2) + s.map.shardsForCoord(coord2.ref) shouldEqual Seq(3, 6, 7) + s.map.shardsForCoord(coord4.ref) shouldEqual Seq(4, 5) + s.map.unassignedShards shouldEqual Seq(0) + } subscriber.expectNoMessage() } From 582fd22066ca9ca9b21ecc2a87ea8126e1924ce6 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Fri, 22 Mar 2019 16:02:25 -0700 Subject: [PATCH 26/41] feat(query): Histogram sum() over RangeVectors (#288) --- README.md | 7 +- .../format/vectors/Histogram.scala | 57 +++++- .../format/vectors/HistogramVector.scala | 2 +- .../format/vectors/HistogramTest.scala | 35 ++++ .../query/exec/AggrOverRangeVectors.scala | 60 ++++-- .../filodb/query/exec/BinaryJoinExec.scala | 3 +- .../filodb/query/exec/DistConcatExec.scala | 3 +- .../scala/filodb/query/exec/ExecPlan.scala | 5 +- .../filodb/query/exec/MetadataExecPlan.scala | 12 +- .../filodb/query/exec/StitchRvsExec.scala | 3 +- .../query/exec/AggrOverRangeVectorsSpec.scala | 185 +++++++++--------- .../query/exec/BinaryJoinExecSpec.scala | 15 +- .../exec/SelectRawPartitionsExecSpec.scala | 20 +- 13 files changed, 271 insertions(+), 136 deletions(-) diff --git a/README.md b/README.md index e389864f9f..1ed775acfb 100644 --- a/README.md +++ b/README.md @@ -388,12 +388,15 @@ Example of debugging chunk metadata using the CLI: ### First-Class Histogram Support -One major difference FiloDB has from the Prometheus data model is that FiloDB supports histograms as a first-class entity. In Prometheus, histograms are stored with each bucket in its own time series differentiated by the `le` tag. In FiloDB, there is a `HistogramColumn` which stores all the buckets together for significantly improved compression, especially over the wire during ingestion, as well as significantly faster query speeds. Here are the differences users need to be aware of when using `HistogramColumn`: +One major difference FiloDB has from the Prometheus data model is that FiloDB supports histograms as a first-class entity. In Prometheus, histograms are stored with each bucket in its own time series differentiated by the `le` tag. In FiloDB, there is a `HistogramColumn` which stores all the buckets together for significantly improved compression, especially over the wire during ingestion, as well as significantly faster query speeds (up to two orders of magnitude). There is no "le" tag or individual time series for each bucket. Here are the differences users need to be aware of when using `HistogramColumn`: * There is no need to append `_bucket` to the metric name. * However, you need to select the histogram column like `__col__="hist"` -* To compute quantiles: `histogram_quantile(0.7, rate(http_req_latency{app="foo",__col__="hist"}[5m]))` +* To compute quantiles: `histogram_quantile(0.7, sum_over_time(http_req_latency{app="foo",__col__="hist"}[5m]))` * To extract a bucket: `histogram_bucket(100.0, http_req_latency{app="foo",__col__="hist"})` +* Sum over multiple Histogram time series: `sum(sum_over_time(http_req_latency{app="foo",__col__="hist"}[5m]))` - you could then compute quantile over the sum. + - NOTE: Do NOT use `group by (le)` when summing `HistogramColumns`. This is not appropriate as the "le" tag is not used. FiloDB knows how to sum multiple histograms together correctly without grouping tricks. + - FiloDB prevents many incorrect histogram aggregations in Prometheus when using `HistogramColumn`, such as handling of multiple histogram schemas across time series and across time. ### Using the FiloDB HTTP API 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 07d92c8b8e..c4cd7e0197 100644 --- a/memory/src/main/scala/filodb.memory/format/vectors/Histogram.scala +++ b/memory/src/main/scala/filodb.memory/format/vectors/Histogram.scala @@ -132,6 +132,13 @@ trait HistogramWithBuckets extends Histogram { def buckets: HistogramBuckets final def numBuckets: Int = buckets.numBuckets final def bucketTop(no: Int): Double = buckets.bucketTop(no) + final def valueArray: Array[Double] = { + val values = new Array[Double](numBuckets) + for { b <- 0 until numBuckets optimized } { + values(b) = bucketValue(b) + } + values + } } final case class LongHistogram(buckets: HistogramBuckets, values: Array[Long]) extends HistogramWithBuckets { @@ -172,10 +179,12 @@ final case class MutableHistogram(buckets: HistogramBuckets, values: Array[Doubl final def copy: Histogram = MutableHistogram(buckets, values.clone) /** - * Adds the values from another MutableHistogram having the same bucket schema. If it does not, then - * an exception is thrown -- for now. Modifies itself. + * Adds the values from another Histogram. + * If the other histogram has the same bucket scheme, then the values are just added per bucket. + * If the scheme is different, then an approximation is used so that the resulting histogram has + * an approximate sum of the individual distributions, with the original scheme. Modifies itself. */ - final def add(other: HistogramWithBuckets): Unit = + final def addNoCorrection(other: HistogramWithBuckets): Unit = if (buckets == other.buckets) { // If it was NaN before, reset to 0 to sum another hist if (values(0).isNaN) java.util.Arrays.fill(values, 0.0) @@ -183,13 +192,53 @@ final case class MutableHistogram(buckets: HistogramBuckets, values: Array[Doubl values(b) += other.bucketValue(b) } } else { - throw new UnsupportedOperationException(s"Cannot add other with buckets ${other.buckets} to myself $buckets") + throw new UnsupportedOperationException(s"Cannot add histogram of scheme ${other.buckets} to $buckets") + // TODO: In the future, support adding buckets of different scheme. Below is an example + // NOTE: there are two issues here: below add picks the existing bucket scheme (not commutative) + // and the newer different buckets are lost (one may want more granularity) + // var ourBucketNo = 0 + // for { b <- 0 until other.numBuckets optimized } { + // // Find our first bucket greater than or equal to their bucket + // while (ourBucketNo < numBuckets && bucketTop(ourBucketNo) < other.bucketTop(b)) ourBucketNo += 1 + // if (ourBucketNo < numBuckets) { + // values(ourBucketNo) += other.bucketValue(b) + // } + // } } + + /** + * Adds the values from another Histogram, making a monotonic correction to ensure correctness + */ + final def add(other: HistogramWithBuckets): Unit = { + addNoCorrection(other) + makeMonotonic() + } + + /** + * Fixes any issue with monotonicity of supplied bucket values. + * Bucket values should monotonically increase. It may not be the case + * if the bucket values are not atomically obtained from the same scrape, + * or if bucket le values change over time (esp from aggregation) causing NaN on missing buckets. + */ + final def makeMonotonic(): Unit = { + var max = 0d + for { b <- 0 until values.size optimized } { + // When bucket no longer used NaN will be seen. Non-increasing values can be seen when + // newer buckets are introduced and not all instances are updated with that bucket. + if (values(b) < max || values(b).isNaN) values(b) = max // assign previous max + else if (values(b) > max) max = values(b) // update max + } + } } object MutableHistogram { def empty(buckets: HistogramBuckets): MutableHistogram = MutableHistogram(buckets, Array.fill(buckets.numBuckets)(Double.NaN)) + + 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 de12fcd7c9..cc23454556 100644 --- a/memory/src/main/scala/filodb.memory/format/vectors/HistogramVector.scala +++ b/memory/src/main/scala/filodb.memory/format/vectors/HistogramVector.scala @@ -388,7 +388,7 @@ class RowHistogramReader(histVect: Ptr.U8) extends HistogramReader { require(length > 0 && start >= 0 && end < length) val summedHist = MutableHistogram.empty(buckets) for { i <- start to end optimized } { - summedHist.add(apply(i).asInstanceOf[HistogramWithBuckets]) + summedHist.addNoCorrection(apply(i).asInstanceOf[HistogramWithBuckets]) } summedHist } 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 dbe9ac83f5..140ce72e36 100644 --- a/memory/src/test/scala/filodb.memory/format/vectors/HistogramTest.scala +++ b/memory/src/test/scala/filodb.memory/format/vectors/HistogramTest.scala @@ -91,5 +91,40 @@ class HistogramTest extends NativeVectorTest { hist should not equal (hist2) hist.values shouldEqual addedBuckets } + + it("should correctly add & makeMonotonic histograms containing NaNs") { + val hist1 = mutableHistograms(0).copy.asInstanceOf[MutableHistogram] + val histWNans = mutableHistograms(1) + histWNans.values(0) = Double.NaN + histWNans.values(2) = Double.NaN + + hist1.add(histWNans) + var current = 0d + hist1.valueArray.foreach { d => + d should be >= (current) + current = d + } + } + + // Test this later when different schemas are supported + ignore("should add histogram w/ diff bucket scheme and result in monotonically increasing histogram") { + val hist1 = mutableHistograms(0).copy.asInstanceOf[MutableHistogram] + + val scheme2 = GeometricBuckets(2.0, 6.0, 3) + val hist2 = LongHistogram(scheme2, Array(10L, 20L, 40L)) + + hist1.add(hist2) + hist1.makeMonotonic() + + hist1.buckets shouldEqual bucketScheme // scheme does not change - for now + + // New buckets should be increasing, and values should not be less than before + var current = 0d + hist1.valueArray.zipWithIndex.foreach { case (d, i) => + d should be >= (current) + d should be >= rawHistBuckets(0)(i) + current = d + } + } } } \ No newline at end of file diff --git a/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala b/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala index 010261b522..332a4d7f4b 100644 --- a/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala +++ b/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala @@ -31,13 +31,16 @@ final case class ReduceAggregateExec(id: String, protected def args: String = s"aggrOp=$aggrOp, aggrParams=$aggrParams" - protected def compose(childResponses: Observable[(QueryResponse, Int)], + protected def compose(dataset: Dataset, + childResponses: Observable[(QueryResponse, Int)], queryConfig: QueryConfig): Observable[RangeVector] = { val results = childResponses.flatMap { case (QueryResult(_, _, result), _) => Observable.fromIterable(result) case (QueryError(_, ex), _) => throw ex } - RangeVectorAggregator.mapReduce(aggrOp, aggrParams, skipMapPhase = true, results, rv => rv.key) + val valColType = RangeVectorTransformer.valueColumnType(schemaOfCompose(dataset)) + val aggregator = RowAggregator(aggrOp, aggrParams, valColType) + RangeVectorAggregator.mapReduce(aggregator, skipMapPhase = true, results, rv => rv.key) } } @@ -54,12 +57,13 @@ final case class AggregateMapReduce(aggrOp: AggregationOperator, protected[exec] def args: String = s"aggrOp=$aggrOp, aggrParams=$aggrParams, without=$without, by=$by" - val aggregator = RowAggregator(aggrOp, aggrParams) def apply(source: Observable[RangeVector], queryConfig: QueryConfig, limit: Int, sourceSchema: ResultSchema): Observable[RangeVector] = { + val valColType = RangeVectorTransformer.valueColumnType(sourceSchema) + val aggregator = RowAggregator(aggrOp, aggrParams, valColType) def grouping(rv: RangeVector): RangeVectorKey = { val groupBy: Map[ZeroCopyUTF8String, ZeroCopyUTF8String] = if (by.nonEmpty) rv.key.labelValues.filter(lv => byLabels.contains(lv._1)) @@ -67,10 +71,12 @@ final case class AggregateMapReduce(aggrOp: AggregationOperator, else Map.empty CustomRangeVectorKey(groupBy) } - RangeVectorAggregator.mapReduce(aggrOp, aggrParams, skipMapPhase = false, source, grouping) + RangeVectorAggregator.mapReduce(aggregator, skipMapPhase = false, source, grouping) } override def schema(dataset: Dataset, source: ResultSchema): ResultSchema = { + val valColType = RangeVectorTransformer.valueColumnType(source) + val aggregator = RowAggregator(aggrOp, aggrParams, valColType) // TODO we assume that second column needs to be aggregated. Other dataset types need to be accommodated. aggregator.reductionSchema(source) } @@ -80,16 +86,19 @@ final case class AggregatePresenter(aggrOp: AggregationOperator, aggrParams: Seq[Any]) extends RangeVectorTransformer { protected[exec] def args: String = s"aggrOp=$aggrOp, aggrParams=$aggrParams" - val aggregator = RowAggregator(aggrOp, aggrParams) def apply(source: Observable[RangeVector], queryConfig: QueryConfig, limit: Int, sourceSchema: ResultSchema): Observable[RangeVector] = { - RangeVectorAggregator.present(aggrOp, aggrParams, source, limit) + val valColType = RangeVectorTransformer.valueColumnType(sourceSchema) + val aggregator = RowAggregator(aggrOp, aggrParams, valColType) + RangeVectorAggregator.present(aggregator, source, limit) } override def schema(dataset: Dataset, source: ResultSchema): ResultSchema = { + val valColType = RangeVectorTransformer.valueColumnType(source) + val aggregator = RowAggregator(aggrOp, aggrParams, valColType) aggregator.presentationSchema(source) } } @@ -108,12 +117,10 @@ object RangeVectorAggregator { * This method is the facade for map and reduce steps of the aggregation. * In the reduction-only (non-leaf) phases, skipMapPhase should be true. */ - def mapReduce(aggrOp: AggregationOperator, - params: Seq[Any], + def mapReduce(rowAgg: RowAggregator, skipMapPhase: Boolean, source: Observable[RangeVector], grouping: RangeVector => RangeVectorKey): Observable[RangeVector] = { - val rowAgg = RowAggregator(aggrOp, params) // row aggregator // reduce the range vectors using the foldLeft construct. This results in one aggregate per group. val task = source.toListL.map { rvs => // now reduce each group and create one result range vector per group @@ -129,11 +136,9 @@ object RangeVectorAggregator { /** * This method is the facade for the present step of the aggregation */ - def present(aggrOp: AggregationOperator, - params: Seq[Any], + def present(aggregator: RowAggregator, source: Observable[RangeVector], limit: Int): Observable[RangeVector] = { - val aggregator = RowAggregator(aggrOp, params) source.flatMap(rv => Observable.fromIterable(aggregator.present(rv, limit))) } @@ -267,11 +272,12 @@ object RowAggregator { /** * Factory for RowAggregator */ - def apply(aggrOp: AggregationOperator, params: Seq[Any]): RowAggregator = { + def apply(aggrOp: AggregationOperator, params: Seq[Any], dataColType: ColumnType): RowAggregator = { aggrOp match { case Min => MinRowAggregator case Max => MaxRowAggregator - case Sum => SumRowAggregator + case Sum if dataColType == ColumnType.DoubleColumn => SumRowAggregator + case Sum if dataColType == ColumnType.HistogramColumn => HistSumRowAggregator case Count => CountRowAggregator case Avg => AvgRowAggregator case TopK => new TopBottomKRowAggregator(params(0).asInstanceOf[Double].toInt, false) @@ -311,6 +317,32 @@ object SumRowAggregator extends RowAggregator { def presentationSchema(reductionSchema: ResultSchema): ResultSchema = reductionSchema } +object HistSumRowAggregator extends RowAggregator { + import filodb.memory.format.{vectors => bv} + + class HistSumHolder(var timestamp: Long = 0L, var h: bv.Histogram = bv.Histogram.empty) extends AggregateHolder { + val row = new TransientHistRow() + def toRowReader: MutableRowReader = { row.setValues(timestamp, h); row } + def resetToZero(): Unit = h = bv.Histogram.empty + } + type AggHolderType = HistSumHolder + def zero: HistSumHolder = new HistSumHolder + def newRowToMapInto: MutableRowReader = new TransientHistRow() + def map(rvk: RangeVectorKey, item: RowReader, mapInto: MutableRowReader): RowReader = item + def reduceAggregate(acc: HistSumHolder, aggRes: RowReader): HistSumHolder = { + acc.timestamp = aggRes.getLong(0) + acc.h match { + // sum is mutable histogram, copy to be sure it's our own copy + case hist if hist.numBuckets == 0 => acc.h = bv.MutableHistogram(aggRes.getHistogram(1)) + case hist: bv.MutableHistogram => hist.add(aggRes.getHistogram(1).asInstanceOf[bv.HistogramWithBuckets]) + } + acc + } + def present(aggRangeVector: RangeVector, limit: Int): Seq[RangeVector] = Seq(aggRangeVector) + def reductionSchema(source: ResultSchema): ResultSchema = source + def presentationSchema(reductionSchema: ResultSchema): ResultSchema = reductionSchema +} + /** * Map: Every sample is mapped to itself * ReduceMappedRow: Same as ReduceAggregate since every row is mapped into an aggregate diff --git a/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala b/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala index b480daa54a..f774076242 100644 --- a/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala +++ b/query/src/main/scala/filodb/query/exec/BinaryJoinExec.scala @@ -58,7 +58,8 @@ final case class BinaryJoinExec(id: String, protected def args: String = s"binaryOp=$binaryOp, on=$on, ignoring=$ignoring" - protected[exec] def compose(childResponses: Observable[(QueryResponse, Int)], + protected[exec] def compose(dataset: Dataset, + childResponses: Observable[(QueryResponse, Int)], queryConfig: QueryConfig): Observable[RangeVector] = { val taskOfResults = childResponses.map { case (QueryResult(_, _, result), i) => (result, i) diff --git a/query/src/main/scala/filodb/query/exec/DistConcatExec.scala b/query/src/main/scala/filodb/query/exec/DistConcatExec.scala index 25afaa5b69..ecb3cc335a 100644 --- a/query/src/main/scala/filodb/query/exec/DistConcatExec.scala +++ b/query/src/main/scala/filodb/query/exec/DistConcatExec.scala @@ -19,7 +19,8 @@ final case class DistConcatExec(id: String, protected def schemaOfCompose(dataset: Dataset): ResultSchema = children.head.schema(dataset) - protected def compose(childResponses: Observable[(QueryResponse, Int)], + protected def compose(dataset: Dataset, + childResponses: Observable[(QueryResponse, Int)], queryConfig: QueryConfig): Observable[RangeVector] = { qLogger.debug(s"DistConcatExec: Concatenating results") childResponses.flatMap { diff --git a/query/src/main/scala/filodb/query/exec/ExecPlan.scala b/query/src/main/scala/filodb/query/exec/ExecPlan.scala index a2562f8198..fe4e1be5df 100644 --- a/query/src/main/scala/filodb/query/exec/ExecPlan.scala +++ b/query/src/main/scala/filodb/query/exec/ExecPlan.scala @@ -244,7 +244,7 @@ abstract class NonLeafExecPlan extends ExecPlan { }.map((_, i)) } } - compose(childTasks, queryConfig) + compose(dataset, childTasks, queryConfig) } final protected def schemaOfDoExecute(dataset: Dataset): ResultSchema = schemaOfCompose(dataset) @@ -258,7 +258,8 @@ abstract class NonLeafExecPlan extends ExecPlan { * Sub-class non-leaf nodes should provide their own implementation of how * to compose the sub-query results here. */ - protected def compose(childResponses: Observable[(QueryResponse, Int)], + protected def compose(dataset: Dataset, + childResponses: Observable[(QueryResponse, Int)], queryConfig: QueryConfig): Observable[RangeVector] } diff --git a/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala b/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala index 0bbc9dcde8..0ab53f6768 100644 --- a/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala +++ b/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala @@ -39,9 +39,9 @@ final case class PartKeysDistConcatExec(id: String, /** * Compose the sub-query/leaf results here. */ - override protected def compose(childResponses: Observable[(QueryResponse, Int)], queryConfig: QueryConfig): - Observable[RangeVector] = { - + protected def compose(dataset: Dataset, + childResponses: Observable[(QueryResponse, Int)], + queryConfig: QueryConfig): Observable[RangeVector] = { qLogger.debug(s"NonLeafMetadataExecPlan: Concatenating results") val taskOfResults = childResponses.map { case (QueryResult(_, _, result), _) => result @@ -74,9 +74,9 @@ final case class LabelValuesDistConcatExec(id: String, /** * Compose the sub-query/leaf results here. */ - override protected def compose(childResponses: Observable[(QueryResponse, Int)], queryConfig: QueryConfig): - Observable[RangeVector] = { - + protected def compose(dataset: Dataset, + childResponses: Observable[(QueryResponse, Int)], + queryConfig: QueryConfig): Observable[RangeVector] = { qLogger.debug(s"NonLeafMetadataExecPlan: Concatenating results") val taskOfResults = childResponses.map { case (QueryResult(_, _, result), _) => result diff --git a/query/src/main/scala/filodb/query/exec/StitchRvsExec.scala b/query/src/main/scala/filodb/query/exec/StitchRvsExec.scala index 32ec5307af..6cd87b09cf 100644 --- a/query/src/main/scala/filodb/query/exec/StitchRvsExec.scala +++ b/query/src/main/scala/filodb/query/exec/StitchRvsExec.scala @@ -65,7 +65,8 @@ final case class StitchRvsExec(id: String, protected def schemaOfCompose(dataset: Dataset): ResultSchema = children.head.schema(dataset) - protected def compose(childResponses: Observable[(QueryResponse, Int)], + protected def compose(dataset: Dataset, + childResponses: Observable[(QueryResponse, Int)], queryConfig: QueryConfig): Observable[RangeVector] = { qLogger.debug(s"StitchRvsExec: Stitching results:") val stitched = childResponses.map { diff --git a/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala b/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala index 77010f9941..2b8f60aa9c 100644 --- a/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala +++ b/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala @@ -4,21 +4,17 @@ import scala.annotation.tailrec import scala.util.Random import com.tdunning.math.stats.TDigest -import com.typesafe.config.ConfigFactory import monix.execution.Scheduler.Implicits.global import monix.reactive.Observable -import org.scalatest.{FunSpec, Matchers} import org.scalatest.concurrent.ScalaFutures import filodb.core.metadata.Column.ColumnType import filodb.core.query._ import filodb.memory.format.{RowReader, ZeroCopyUTF8String} -import filodb.query.{AggregationOperator, QueryConfig} +import filodb.query.AggregationOperator +import filodb.query.exec.rangefn.RawDataWindowingSpec -class AggrOverRangeVectorsSpec extends FunSpec with Matchers with ScalaFutures { - - val config = ConfigFactory.load("application_test.conf").getConfig("filodb") - val queryConfig = new QueryConfig(config.getConfig("query")) +class AggrOverRangeVectorsSpec extends RawDataWindowingSpec with ScalaFutures { val rand = new Random() val error = 0.0000001d @@ -38,8 +34,8 @@ class AggrOverRangeVectorsSpec extends FunSpec with Matchers with ScalaFutures { }) // Sum - val resultObs = RangeVectorAggregator.mapReduce(AggregationOperator.Sum, - Nil, false, Observable.fromIterable(samples), noGrouping) + val agg1 = RowAggregator(AggregationOperator.Sum, Nil, ColumnType.DoubleColumn) + val resultObs = RangeVectorAggregator.mapReduce(agg1, false, Observable.fromIterable(samples), noGrouping) val result = resultObs.toListL.runAsync.futureValue result.size shouldEqual 1 result(0).key shouldEqual noKey @@ -47,8 +43,8 @@ class AggrOverRangeVectorsSpec extends FunSpec with Matchers with ScalaFutures { compareIter(result(0).rows.map(_.getDouble(1)), readyToAggr.map(_.map(_.getDouble(1)).sum).iterator) // Min - val resultObs2 = RangeVectorAggregator.mapReduce(AggregationOperator.Min, - Nil, false, Observable.fromIterable(samples), noGrouping) + val agg2 = RowAggregator(AggregationOperator.Min, Nil, ColumnType.DoubleColumn) + val resultObs2 = RangeVectorAggregator.mapReduce(agg2, false, Observable.fromIterable(samples), noGrouping) val result2 = resultObs2.toListL.runAsync.futureValue result2.size shouldEqual 1 result2(0).key shouldEqual noKey @@ -56,10 +52,9 @@ class AggrOverRangeVectorsSpec extends FunSpec with Matchers with ScalaFutures { compareIter(result2(0).rows.map(_.getDouble(1)), readyToAggr2.map(_.map(_.getDouble(1)).min).iterator) // Count - val resultObs3a = RangeVectorAggregator.mapReduce(AggregationOperator.Count, - Nil, false, Observable.fromIterable(samples), noGrouping) - val resultObs3 = RangeVectorAggregator.mapReduce(AggregationOperator.Count, - Nil, true, resultObs3a, rv=>rv.key) + val agg3 = RowAggregator(AggregationOperator.Count, Nil, ColumnType.DoubleColumn) + val resultObs3a = RangeVectorAggregator.mapReduce(agg3, false, Observable.fromIterable(samples), noGrouping) + val resultObs3 = RangeVectorAggregator.mapReduce(agg3, true, resultObs3a, rv=>rv.key) val result3 = resultObs3.toListL.runAsync.futureValue result3.size shouldEqual 1 result3(0).key shouldEqual noKey @@ -67,10 +62,9 @@ class AggrOverRangeVectorsSpec extends FunSpec with Matchers with ScalaFutures { compareIter(result3(0).rows.map(_.getDouble(1)), readyToAggr3.map(_.map(_.getDouble(1)).size.toDouble).iterator) // Avg - val resultObs4a = RangeVectorAggregator.mapReduce(AggregationOperator.Avg, - Nil, false, Observable.fromIterable(samples), noGrouping) - val resultObs4 = RangeVectorAggregator.mapReduce(AggregationOperator.Avg, - Nil, true, resultObs4a, rv=>rv.key) + val agg4 = RowAggregator(AggregationOperator.Avg, Nil, ColumnType.DoubleColumn) + val resultObs4a = RangeVectorAggregator.mapReduce(agg4, false, Observable.fromIterable(samples), noGrouping) + val resultObs4 = RangeVectorAggregator.mapReduce(agg4, true, resultObs4a, rv=>rv.key) val result4 = resultObs4.toListL.runAsync.futureValue result4.size shouldEqual 1 result4(0).key shouldEqual noKey @@ -80,10 +74,9 @@ class AggrOverRangeVectorsSpec extends FunSpec with Matchers with ScalaFutures { }.iterator) // BottomK - val resultObs5a = RangeVectorAggregator.mapReduce(AggregationOperator.BottomK, - Seq(3.0), false, Observable.fromIterable(samples), noGrouping) - val resultObs5 = RangeVectorAggregator.mapReduce(AggregationOperator.BottomK, - Seq(3.0), true, resultObs5a, rv=>rv.key) + val agg5 = RowAggregator(AggregationOperator.BottomK, Seq(3.0), ColumnType.DoubleColumn) + val resultObs5a = RangeVectorAggregator.mapReduce(agg5, false, Observable.fromIterable(samples), noGrouping) + val resultObs5 = RangeVectorAggregator.mapReduce(agg5, true, resultObs5a, rv=>rv.key) val result5 = resultObs5.toListL.runAsync.futureValue result5.size shouldEqual 1 result5(0).key shouldEqual noKey @@ -94,10 +87,9 @@ class AggrOverRangeVectorsSpec extends FunSpec with Matchers with ScalaFutures { }.iterator) // TopK - val resultObs6a = RangeVectorAggregator.mapReduce(AggregationOperator.TopK, - Seq(3.0), false, Observable.fromIterable(samples), noGrouping) - val resultObs6 = RangeVectorAggregator.mapReduce(AggregationOperator.TopK, - Seq(3.0), true, resultObs6a, rv=>rv.key) + val agg6 = RowAggregator(AggregationOperator.TopK, Seq(3.0), ColumnType.DoubleColumn) + val resultObs6a = RangeVectorAggregator.mapReduce(agg6, false, Observable.fromIterable(samples), noGrouping) + val resultObs6 = RangeVectorAggregator.mapReduce(agg6, true, resultObs6a, rv=>rv.key) val result6 = resultObs6.toListL.runAsync.futureValue result6.size shouldEqual 1 result6(0).key shouldEqual noKey @@ -108,11 +100,10 @@ class AggrOverRangeVectorsSpec extends FunSpec with Matchers with ScalaFutures { }.iterator) // Quantile - val resultObs7a = RangeVectorAggregator.mapReduce(AggregationOperator.Quantile, - Seq(0.70), false, Observable.fromIterable(samples), noGrouping) - val resultObs7 = RangeVectorAggregator.mapReduce(AggregationOperator.Quantile, - Seq(0.70), true, resultObs7a, rv=>rv.key) - val resultObs7b = RangeVectorAggregator.present(AggregationOperator.Quantile, Seq(0.70), resultObs7, 1000) + val agg7 = RowAggregator(AggregationOperator.Quantile, Seq(0.70), ColumnType.DoubleColumn) + val resultObs7a = RangeVectorAggregator.mapReduce(agg7, false, Observable.fromIterable(samples), noGrouping) + val resultObs7 = RangeVectorAggregator.mapReduce(agg7, true, resultObs7a, rv=>rv.key) + val resultObs7b = RangeVectorAggregator.present(agg7, resultObs7, 1000) val result7 = resultObs7b.toListL.runAsync.futureValue result7.size shouldEqual 1 result7(0).key shouldEqual noKey @@ -143,47 +134,44 @@ class AggrOverRangeVectorsSpec extends FunSpec with Matchers with ScalaFutures { ) // Sum - val resultObs = RangeVectorAggregator.mapReduce(AggregationOperator.Sum, - Nil, false, Observable.fromIterable(samples), noGrouping) + val agg1 = RowAggregator(AggregationOperator.Sum, Nil, ColumnType.DoubleColumn) + val resultObs = RangeVectorAggregator.mapReduce(agg1, false, Observable.fromIterable(samples), noGrouping) val result = resultObs.toListL.runAsync.futureValue result.size shouldEqual 1 result(0).key shouldEqual noKey compareIter(result(0).rows.map(_.getDouble(1)), Seq(6.7d, 15.4d).iterator) // Min - val resultObs2 = RangeVectorAggregator.mapReduce(AggregationOperator.Min, - Nil, false, Observable.fromIterable(samples), noGrouping) + val agg2 = RowAggregator(AggregationOperator.Min, Nil, ColumnType.DoubleColumn) + val resultObs2 = RangeVectorAggregator.mapReduce(agg2, false, Observable.fromIterable(samples), noGrouping) val result2 = resultObs2.toListL.runAsync.futureValue result2.size shouldEqual 1 result2(0).key shouldEqual noKey compareIter(result2(0).rows.map(_.getDouble(1)), Seq(2.1d, 4.4d).iterator) // Count - val resultObs3a = RangeVectorAggregator.mapReduce(AggregationOperator.Count, - Nil, false, Observable.fromIterable(samples), noGrouping) - val resultObs3 = RangeVectorAggregator.mapReduce(AggregationOperator.Count, - Nil, true, resultObs3a, rv=>rv.key) + val agg3 = RowAggregator(AggregationOperator.Count, Nil, ColumnType.DoubleColumn) + val resultObs3a = RangeVectorAggregator.mapReduce(agg3, false, Observable.fromIterable(samples), noGrouping) + val resultObs3 = RangeVectorAggregator.mapReduce(agg3, true, resultObs3a, rv=>rv.key) val result3 = resultObs3.toListL.runAsync.futureValue result3.size shouldEqual 1 result3(0).key shouldEqual noKey compareIter(result3(0).rows.map(_.getDouble(1)), Seq(2d, 3d).iterator) // Avg - val resultObs4a = RangeVectorAggregator.mapReduce(AggregationOperator.Avg, - Nil, false, Observable.fromIterable(samples), noGrouping) - val resultObs4 = RangeVectorAggregator.mapReduce(AggregationOperator.Avg, - Nil, true, resultObs4a, rv=>rv.key) + val agg4 = RowAggregator(AggregationOperator.Avg, Nil, ColumnType.DoubleColumn) + val resultObs4a = RangeVectorAggregator.mapReduce(agg4, false, Observable.fromIterable(samples), noGrouping) + val resultObs4 = RangeVectorAggregator.mapReduce(agg4, true, resultObs4a, rv=>rv.key) val result4 = resultObs4.toListL.runAsync.futureValue result4.size shouldEqual 1 result4(0).key shouldEqual noKey compareIter(result4(0).rows.map(_.getDouble(1)), Seq(3.35d, 5.133333333333333d).iterator) // BottomK - val resultObs5a = RangeVectorAggregator.mapReduce(AggregationOperator.BottomK, - Seq(2.0), false, Observable.fromIterable(samples), noGrouping) - val resultObs5 = RangeVectorAggregator.mapReduce(AggregationOperator.BottomK, - Seq(2.0), true, resultObs5a, rv=>rv.key) - val resultObs5b = RangeVectorAggregator.present(AggregationOperator.BottomK, Seq(2.0), resultObs5, 1000) + val agg5 = RowAggregator(AggregationOperator.BottomK, Seq(2.0), ColumnType.DoubleColumn) + val resultObs5a = RangeVectorAggregator.mapReduce(agg5, false, Observable.fromIterable(samples), noGrouping) + val resultObs5 = RangeVectorAggregator.mapReduce(agg5, true, resultObs5a, rv=>rv.key) + val resultObs5b = RangeVectorAggregator.present(agg5, resultObs5, 1000) val result5 = resultObs5.toListL.runAsync.futureValue result5.size shouldEqual 1 result5(0).key shouldEqual noKey @@ -195,11 +183,10 @@ class AggrOverRangeVectorsSpec extends FunSpec with Matchers with ScalaFutures { compareIter(result5b(0).rows.map(_.getDouble(1)), Seq(4.6d,2.1d,5.4d,4.4d).iterator) // TopK - val resultObs6a = RangeVectorAggregator.mapReduce(AggregationOperator.TopK, - Seq(2.0), false, Observable.fromIterable(samples), noGrouping) - val resultObs6 = RangeVectorAggregator.mapReduce(AggregationOperator.TopK, - Seq(2.0), true, resultObs6a, rv=>rv.key) - val resultObs6b = RangeVectorAggregator.present(AggregationOperator.TopK, Seq(2.0), resultObs6, 1000) + val agg6 = RowAggregator(AggregationOperator.TopK, Seq(2.0), ColumnType.DoubleColumn) + val resultObs6a = RangeVectorAggregator.mapReduce(agg6, false, Observable.fromIterable(samples), noGrouping) + val resultObs6 = RangeVectorAggregator.mapReduce(agg6, true, resultObs6a, rv=>rv.key) + val resultObs6b = RangeVectorAggregator.present(agg6, resultObs6, 1000) val result6 = resultObs6.toListL.runAsync.futureValue result6.size shouldEqual 1 result6(0).key shouldEqual noKey @@ -211,11 +198,10 @@ class AggrOverRangeVectorsSpec extends FunSpec with Matchers with ScalaFutures { compareIter(result6b(0).rows.map(_.getDouble(1)), Seq(2.1d,4.6d,5.4d,5.6d).iterator) // Quantile - val resultObs7a = RangeVectorAggregator.mapReduce(AggregationOperator.Quantile, - Seq(0.5), false, Observable.fromIterable(samples), noGrouping) - val resultObs7 = RangeVectorAggregator.mapReduce(AggregationOperator.Quantile, - Seq(0.5), true, resultObs7a, rv=>rv.key) - val resultObs7b = RangeVectorAggregator.present(AggregationOperator.Quantile, Seq(0.5), resultObs7, 1000) + val agg7 = RowAggregator(AggregationOperator.Quantile, Seq(0.5), ColumnType.DoubleColumn) + val resultObs7a = RangeVectorAggregator.mapReduce(agg7, false, Observable.fromIterable(samples), noGrouping) + val resultObs7 = RangeVectorAggregator.mapReduce(agg7, true, resultObs7a, rv=>rv.key) + val resultObs7b = RangeVectorAggregator.present(agg7, resultObs7, 1000) val result7 = resultObs7b.toListL.runAsync.futureValue result7.size shouldEqual 1 result7(0).key shouldEqual noKey @@ -230,10 +216,9 @@ class AggrOverRangeVectorsSpec extends FunSpec with Matchers with ScalaFutures { ) // Quantile - val resultObs7a = RangeVectorAggregator.mapReduce(AggregationOperator.Quantile, - Seq(0.5), false, Observable.fromIterable(samples), noGrouping) - val resultObs7 = RangeVectorAggregator.mapReduce(AggregationOperator.Quantile, - Seq(0.5), true, resultObs7a, rv=>rv.key) + val agg7 = RowAggregator(AggregationOperator.Quantile, Seq(0.5), ColumnType.DoubleColumn) + val resultObs7a = RangeVectorAggregator.mapReduce(agg7, false, Observable.fromIterable(samples), noGrouping) + val resultObs7 = RangeVectorAggregator.mapReduce(agg7, true, resultObs7a, rv=>rv.key) val result7 = resultObs7.toListL.runAsync.futureValue result7.size shouldEqual 1 @@ -242,7 +227,7 @@ class AggrOverRangeVectorsSpec extends FunSpec with Matchers with ScalaFutures { val builder = SerializableRangeVector.toBuilder(recSchema) val srv = SerializableRangeVector(result7(0), builder, recSchema) - val resultObs7b = RangeVectorAggregator.present(AggregationOperator.Quantile, Seq(0.5), Observable.now(srv), 1000) + val resultObs7b = RangeVectorAggregator.present(agg7, Observable.now(srv), 1000) val finalResult = resultObs7b.toListL.runAsync.futureValue compareIter(finalResult(0).rows.map(_.getDouble(1)), Seq(3.35d, 5.4d).iterator) @@ -261,14 +246,11 @@ class AggrOverRangeVectorsSpec extends FunSpec with Matchers with ScalaFutures { val s2 = Seq( (1541190600L, 1.0d), (1541190660L,1.0d), (1541190720L,1.0d), (1541190780L,1.0d), (1541190840L,1.0d), (1541190900L,1.0d), (1541190960L,1.0d)) - val mapped1 = RangeVectorAggregator.mapReduce(AggregationOperator.Avg, - Nil, false, Observable.fromIterable(Seq(toRv(s1))), noGrouping) - - val mapped2 = RangeVectorAggregator.mapReduce(AggregationOperator.Avg, - Nil, false, Observable.fromIterable(Seq(toRv(s2))), noGrouping) + val agg = RowAggregator(AggregationOperator.Avg, Nil, ColumnType.DoubleColumn) + val mapped1 = RangeVectorAggregator.mapReduce(agg, false, Observable.fromIterable(Seq(toRv(s1))), noGrouping) + val mapped2 = RangeVectorAggregator.mapReduce(agg, false, Observable.fromIterable(Seq(toRv(s2))), noGrouping) - val resultObs4 = RangeVectorAggregator.mapReduce(AggregationOperator.Avg, - Nil, true, mapped1 ++ mapped2, rv=>rv.key) + val resultObs4 = RangeVectorAggregator.mapReduce(agg, true, mapped1 ++ mapped2, rv=>rv.key) val result4 = resultObs4.toListL.runAsync.futureValue result4.size shouldEqual 1 result4(0).key shouldEqual noKey @@ -285,47 +267,44 @@ class AggrOverRangeVectorsSpec extends FunSpec with Matchers with ScalaFutures { ) // Sum - val resultObs = RangeVectorAggregator.mapReduce(AggregationOperator.Sum, - Nil, false, Observable.fromIterable(samples), noGrouping) + val agg1 = RowAggregator(AggregationOperator.Sum, Nil, ColumnType.DoubleColumn) + val resultObs = RangeVectorAggregator.mapReduce(agg1, false, Observable.fromIterable(samples), noGrouping) val result = resultObs.toListL.runAsync.futureValue result.size shouldEqual 1 result(0).key shouldEqual noKey compareIter(result(0).rows.map(_.getDouble(1)), Seq(Double.NaN, 15.4d).iterator) // Min - val resultObs2 = RangeVectorAggregator.mapReduce(AggregationOperator.Min, - Nil, false, Observable.fromIterable(samples), noGrouping) + val agg2 = RowAggregator(AggregationOperator.Min, Nil, ColumnType.DoubleColumn) + val resultObs2 = RangeVectorAggregator.mapReduce(agg2, false, Observable.fromIterable(samples), noGrouping) val result2 = resultObs2.toListL.runAsync.futureValue result2.size shouldEqual 1 result2(0).key shouldEqual noKey compareIter(result2(0).rows.map(_.getDouble(1)), Seq(Double.NaN, 4.4d).iterator) // Count - val resultObs3a = RangeVectorAggregator.mapReduce(AggregationOperator.Count, - Nil, false, Observable.fromIterable(samples), noGrouping) - val resultObs3 = RangeVectorAggregator.mapReduce(AggregationOperator.Count, - Nil, true, resultObs3a, rv => rv.key) + val agg3 = RowAggregator(AggregationOperator.Count, Nil, ColumnType.DoubleColumn) + val resultObs3a = RangeVectorAggregator.mapReduce(agg3, false, Observable.fromIterable(samples), noGrouping) + val resultObs3 = RangeVectorAggregator.mapReduce(agg3, true, resultObs3a, rv => rv.key) val result3 = resultObs3.toListL.runAsync.futureValue result3.size shouldEqual 1 result3(0).key shouldEqual noKey compareIter(result3(0).rows.map(_.getDouble(1)), Seq(Double.NaN, 3d).iterator) // Avg - val resultObs4a = RangeVectorAggregator.mapReduce(AggregationOperator.Avg, - Nil, false, Observable.fromIterable(samples), noGrouping) - val resultObs4 = RangeVectorAggregator.mapReduce(AggregationOperator.Avg, - Nil, true, resultObs4a, rv => rv.key) + val agg4 = RowAggregator(AggregationOperator.Avg, Nil, ColumnType.DoubleColumn) + val resultObs4a = RangeVectorAggregator.mapReduce(agg4, false, Observable.fromIterable(samples), noGrouping) + val resultObs4 = RangeVectorAggregator.mapReduce(agg4, true, resultObs4a, rv => rv.key) val result4 = resultObs4.toListL.runAsync.futureValue result4.size shouldEqual 1 result4(0).key shouldEqual noKey compareIter(result4(0).rows.map(_.getDouble(1)), Seq(Double.NaN, 5.133333333333333d).iterator) // BottomK - val resultObs5a = RangeVectorAggregator.mapReduce(AggregationOperator.BottomK, - Seq(2.0), false, Observable.fromIterable(samples), noGrouping) - val resultObs5 = RangeVectorAggregator.mapReduce(AggregationOperator.BottomK, - Seq(2.0), true, resultObs5a, rv=>rv.key) - val resultObs5b = RangeVectorAggregator.present(AggregationOperator.BottomK, Seq(2.0), resultObs5, 1000) + val agg5 = RowAggregator(AggregationOperator.BottomK, Seq(2.0), ColumnType.DoubleColumn) + val resultObs5a = RangeVectorAggregator.mapReduce(agg5, false, Observable.fromIterable(samples), noGrouping) + val resultObs5 = RangeVectorAggregator.mapReduce(agg5, true, resultObs5a, rv=>rv.key) + val resultObs5b = RangeVectorAggregator.present(agg5, resultObs5, 1000) val result5 = resultObs5.toListL.runAsync.futureValue result5.size shouldEqual 1 result5(0).key shouldEqual noKey @@ -339,11 +318,10 @@ class AggrOverRangeVectorsSpec extends FunSpec with Matchers with ScalaFutures { compareIter(result5b(0).rows.map(_.getDouble(1)), Seq(5.4d, 4.4d).iterator) // TopK - val resultObs6a = RangeVectorAggregator.mapReduce(AggregationOperator.TopK, - Seq(2.0), false, Observable.fromIterable(samples), noGrouping) - val resultObs6 = RangeVectorAggregator.mapReduce(AggregationOperator.TopK, - Seq(2.0), true, resultObs6a, rv=>rv.key) - val resultObs6b = RangeVectorAggregator.present(AggregationOperator.TopK, Seq(2.0), resultObs6, 1000) + val agg6 = RowAggregator(AggregationOperator.TopK, Seq(2.0), ColumnType.DoubleColumn) + val resultObs6a = RangeVectorAggregator.mapReduce(agg6, false, Observable.fromIterable(samples), noGrouping) + val resultObs6 = RangeVectorAggregator.mapReduce(agg6, true, resultObs6a, rv=>rv.key) + val resultObs6b = RangeVectorAggregator.present(agg6, resultObs6, 1000) val result6 = resultObs6.toListL.runAsync.futureValue result6.size shouldEqual 1 result6(0).key shouldEqual noKey @@ -353,7 +331,28 @@ class AggrOverRangeVectorsSpec extends FunSpec with Matchers with ScalaFutures { result6b.size shouldEqual 1 result6b(0).key shouldEqual ignoreKey compareIter(result6b(0).rows.map(_.getDouble(1)), Seq(5.4d,5.6d).iterator) + } + + import filodb.memory.format.{vectors => bv} + + it("should sum histogram RVs") { + val (data1, rv1) = histogramRV(numSamples = 5) + val (data2, rv2) = histogramRV(numSamples = 5) + val samples: Array[RangeVector] = Array(rv1, rv2) + + val agg1 = RowAggregator(AggregationOperator.Sum, Nil, ColumnType.HistogramColumn) + val resultObs = RangeVectorAggregator.mapReduce(agg1, false, Observable.fromIterable(samples), noGrouping) + val result = resultObs.toListL.runAsync.futureValue + result.size shouldEqual 1 + 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]) + h1 + }.toList + result(0).rows.map(_.getHistogram(1)).toList shouldEqual sums } @tailrec diff --git a/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala b/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala index 6e1b62c19b..3f458f50b3 100644 --- a/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala @@ -18,6 +18,7 @@ import filodb.memory.format.ZeroCopyUTF8String._ import filodb.query._ class BinaryJoinExecSpec extends FunSpec with Matchers with ScalaFutures { + import SelectRawPartitionsExecSpec._ val config = ConfigFactory.load("application_test.conf").getConfig("filodb") val queryConfig = new QueryConfig(config.getConfig("query")) @@ -57,7 +58,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, - new Array[ExecPlan](1), // empty since we test compose, not execute or doExecute + Array(dummyPlan), // cannot be empty as some compose's rely on the schema new Array[ExecPlan](1), // empty since we test compose, not execute or doExecute BinaryOperator.ADD, Cardinality.OneToOne, @@ -71,7 +72,8 @@ class BinaryJoinExecSpec extends FunSpec with Matchers with ScalaFutures { val rhs = QueryResult("someId", null, samplesRhs2.map(rv => SerializableRangeVector(rv, schema))) // scalastyle:on // note below that order of lhs and rhs is reversed, but index is right. Join should take that into account - val result = execPlan.compose(Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), queryConfig).toListL.runAsync.futureValue + val result = execPlan.compose(dataset, Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), queryConfig) + .toListL.runAsync.futureValue result.foreach { rv => rv.key.labelValues.contains("__name__".utf8) shouldEqual false @@ -102,7 +104,8 @@ class BinaryJoinExecSpec extends FunSpec with Matchers with ScalaFutures { val lhs = QueryResult("someId", null, samplesLhs.map(rv => SerializableRangeVector(rv, schema))) val rhs = QueryResult("someId", null, samplesRhs2.map(rv => SerializableRangeVector(rv, schema))) // scalastyle:on - val result = execPlan.compose(Observable.fromIterable(Seq((lhs, 0), (rhs, 1))), queryConfig).toListL.runAsync.futureValue + val result = execPlan.compose(dataset, Observable.fromIterable(Seq((lhs, 0), (rhs, 1))), queryConfig) + .toListL.runAsync.futureValue result.foreach { rv => rv.key.labelValues.contains("__name__".utf8) shouldEqual false @@ -142,7 +145,8 @@ class BinaryJoinExecSpec extends FunSpec with Matchers with ScalaFutures { val rhs = QueryResult("someId", null, samplesRhs2.map(rv => SerializableRangeVector(rv, schema))) // scalastyle:on - val fut = execPlan.compose(Observable.fromIterable(Seq((lhs, 0), (rhs, 1))), queryConfig).toListL.runAsync + val fut = execPlan.compose(dataset, Observable.fromIterable(Seq((lhs, 0), (rhs, 1))), queryConfig) + .toListL.runAsync ScalaFutures.whenReady(fut.failed) { e => e shouldBe a[BadQueryException] } @@ -175,7 +179,8 @@ class BinaryJoinExecSpec extends FunSpec with Matchers with ScalaFutures { val rhs = QueryResult("someId", null, samplesRhs.map(rv => SerializableRangeVector(rv, schema))) // scalastyle:on - val fut = execPlan.compose(Observable.fromIterable(Seq((lhs, 0), (rhs, 1))), queryConfig).toListL.runAsync + val fut = execPlan.compose(dataset, Observable.fromIterable(Seq((lhs, 0), (rhs, 1))), queryConfig) + .toListL.runAsync ScalaFutures.whenReady(fut.failed) { e => e shouldBe a[BadQueryException] } diff --git a/query/src/test/scala/filodb/query/exec/SelectRawPartitionsExecSpec.scala b/query/src/test/scala/filodb/query/exec/SelectRawPartitionsExecSpec.scala index f4b9779c4d..7b10d462d5 100644 --- a/query/src/test/scala/filodb/query/exec/SelectRawPartitionsExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/SelectRawPartitionsExecSpec.scala @@ -21,9 +21,23 @@ import filodb.memory.MemFactory import filodb.memory.format.{SeqRowReader, ZeroCopyUTF8String} import filodb.query._ +object SelectRawPartitionsExecSpec { + val dummyDispatcher = new PlanDispatcher { + override def dispatch(plan: ExecPlan) + (implicit sched: ExecutionContext, + timeout: FiniteDuration): Task[QueryResponse] = ??? + } + + val dataset = timeseriesDataset + + val dummyPlan = SelectRawPartitionsExec("someQueryId", System.currentTimeMillis, 100, dummyDispatcher, + timeseriesDataset.ref, 0, Nil, AllChunkScan, Seq(0, 1)) +} + class SelectRawPartitionsExecSpec extends FunSpec with Matchers with ScalaFutures with BeforeAndAfterAll { import ZeroCopyUTF8String._ import filodb.core.{MachineMetricsData => MMD} + import SelectRawPartitionsExecSpec._ implicit val defaultPatience = PatienceConfig(timeout = Span(30, Seconds), interval = Span(250, Millis)) @@ -70,12 +84,6 @@ class SelectRawPartitionsExecSpec extends FunSpec with Matchers with ScalaFuture memStore.shutdown() } - val dummyDispatcher = new PlanDispatcher { - override def dispatch(plan: ExecPlan) - (implicit sched: ExecutionContext, - timeout: FiniteDuration): Task[QueryResponse] = ??? - } - it ("should read raw samples from Memstore using AllChunksSelector") { import ZeroCopyUTF8String._ val filters = Seq (ColumnFilter("__name__", Filter.Equals("http_req_total".utf8)), From 5812e3925b7fb0f365ef26e99fa30bd2af19667b Mon Sep 17 00:00:00 2001 From: Jackson Jeyapaul Date: Mon, 25 Mar 2019 09:33:50 -0700 Subject: [PATCH 27/41] misc(core): Make zipkin message maxBytes out configurable (#294) --- core/src/main/scala/filodb.core/zipkin/Zipkin.scala | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/filodb.core/zipkin/Zipkin.scala b/core/src/main/scala/filodb.core/zipkin/Zipkin.scala index c0bf13e59d..3ccc721a1d 100644 --- a/core/src/main/scala/filodb.core/zipkin/Zipkin.scala +++ b/core/src/main/scala/filodb.core/zipkin/Zipkin.scala @@ -127,6 +127,8 @@ class ZipkinReporter extends SpanReporter { val zipkinEndpoint = Kamon.config().getString(ZipkinEndpoint) val zipkinHost = Kamon.config().getString(HostConfigKey) val zipkinPort = Kamon.config().getInt(PortConfigKey) + val maxRequests = Kamon.config().getInt(MaxRequests) + val messageMaxBytes = Kamon.config().getInt(MessageMaxBytes) val url = if (zipkinEndpoint == null || zipkinEndpoint.trim.isEmpty) s"http://$zipkinHost:$zipkinPort/api/v2/spans" @@ -137,8 +139,8 @@ class ZipkinReporter extends SpanReporter { OkHttpSender.newBuilder() .encoding(Encoding.JSON) .endpoint(url) - .maxRequests(64) - .messageMaxBytes(1024 * 128) + .maxRequests(if (maxRequests > 0) maxRequests else 128) + .messageMaxBytes(if (messageMaxBytes > 0) messageMaxBytes else 1024 * 256) .build() ) } @@ -157,6 +159,8 @@ object ZipkinReporter { private val ZipkinEndpoint = "kamon.zipkin.endpoint" private val HostConfigKey = "kamon.zipkin.host" private val PortConfigKey = "kamon.zipkin.port" + private val MaxRequests = "kamon.zipkin.max.requests" + private val MessageMaxBytes = "kamon.zipkin.message.max.bytes" private val SpanKindTag = "span.kind" private val SpanKindServer = "server" private val SpanKindClient = "client" From 6070bd91c54258d5e43324fbce5cdaef3fb3ef5c Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Mon, 25 Mar 2019 10:42:09 -0700 Subject: [PATCH 28/41] fix(query): Fix requirement preventing avg() aggregates from working (#295) --- .../scala/filodb/query/exec/RangeVectorTransformer.scala | 4 ++-- .../scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala | 7 +++++-- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala b/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala index 947fd44d0f..7e6d8cf684 100644 --- a/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala +++ b/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala @@ -45,8 +45,8 @@ trait RangeVectorTransformer extends java.io.Serializable { object RangeVectorTransformer { def valueColumnType(schema: ResultSchema): ColumnType = { - require(schema.isTimeSeries, "Cannot return periodic data from a dataset that is not time series based") - require(schema.columns.size == 2, "Cannot return periodic data from a dataset that is not time series based") + require(schema.isTimeSeries, s"Schema $schema is not time series based, cannot continue query") + require(schema.columns.size >= 2, s"Schema $schema has less than 2 columns, cannot continue query") schema.columns(1).colType } } diff --git a/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala b/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala index 2b8f60aa9c..65c6ee5cae 100644 --- a/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala +++ b/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala @@ -247,8 +247,11 @@ class AggrOverRangeVectorsSpec extends RawDataWindowingSpec with ScalaFutures { (1541190780L,1.0d), (1541190840L,1.0d), (1541190900L,1.0d), (1541190960L,1.0d)) val agg = RowAggregator(AggregationOperator.Avg, Nil, ColumnType.DoubleColumn) - val mapped1 = RangeVectorAggregator.mapReduce(agg, false, Observable.fromIterable(Seq(toRv(s1))), noGrouping) - val mapped2 = RangeVectorAggregator.mapReduce(agg, false, Observable.fromIterable(Seq(toRv(s2))), noGrouping) + val aggMR = AggregateMapReduce(AggregationOperator.Avg, Nil, Nil, Nil) + val srcSchema = ResultSchema(Seq(ColumnInfo("timestamp", ColumnType.LongColumn), + ColumnInfo("value", ColumnType.DoubleColumn)), 1) + val mapped1 = aggMR(Observable.fromIterable(Seq(toRv(s1))), queryConfig, 1000, srcSchema) + val mapped2 = aggMR(Observable.fromIterable(Seq(toRv(s2))), queryConfig, 1000, srcSchema) val resultObs4 = RangeVectorAggregator.mapReduce(agg, true, mapped1 ++ mapped2, rv=>rv.key) val result4 = resultObs4.toListL.runAsync.futureValue From 2f541810979810803048b354214a15d887e81a11 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Mon, 25 Mar 2019 16:37:19 -0700 Subject: [PATCH 29/41] bug(core): ODP from Cassandra only if Lucene index indicates that there is data (#291) Prior to this commit, if earliestTime in memory for a partition is more than queryStart time, we decide to page from cassandra. This results in unnecessary calls to Cassandra because there may not be any data. With this change, we page from cassandra only if Lucene index indicates that partition's startTime is earlier than the earliest time in memory, and the query is for earlier than the earliest time in memory. The code organization for ODP IMO has been less than ideal because there is a lot of things happening and leak in abstraction with non-functional side effects. I do understand everything has been done to reduce object allocation and double iteration of objects. I have tried to make some improvements to existing code, but we need to do a round of refactoring and cleanup to make code more understandable. --- .../MemstoreCassandraSinkSpec.scala | 1 + .../NodeCoordinatorActorSpec.scala | 3 +- .../memstore/OnDemandPagingShard.scala | 134 ++++++++++++------ .../memstore/PartKeyLuceneIndex.scala | 41 +++++- .../memstore/TimeSeriesPartition.scala | 6 +- .../memstore/TimeSeriesShard.scala | 42 ++++-- .../memstore/TimeSeriesMemStoreSpec.scala | 11 +- 7 files changed, 180 insertions(+), 58 deletions(-) diff --git a/cassandra/src/test/scala/filodb.cassandra/MemstoreCassandraSinkSpec.scala b/cassandra/src/test/scala/filodb.cassandra/MemstoreCassandraSinkSpec.scala index 7c97a5055c..ab1ee26530 100644 --- a/cassandra/src/test/scala/filodb.cassandra/MemstoreCassandraSinkSpec.scala +++ b/cassandra/src/test/scala/filodb.cassandra/MemstoreCassandraSinkSpec.scala @@ -46,6 +46,7 @@ class MemstoreCassandraSinkSpec extends AllTablesTest { memStore.store.sinkStats.chunksetsWritten should be >= 3 memStore.store.sinkStats.chunksetsWritten should be <= 4 + memStore.commitIndexForTesting(dataset1.ref) // Verify data still in MemStore... all of it val splits = memStore.getScanSplits(dataset1.ref, 1) val agg1 = memStore.scanRows(dataset1, Seq(1), FilteredPartitionScan(splits.head)) diff --git a/coordinator/src/test/scala/filodb.coordinator/NodeCoordinatorActorSpec.scala b/coordinator/src/test/scala/filodb.coordinator/NodeCoordinatorActorSpec.scala index 274c37cb54..e2369a1010 100644 --- a/coordinator/src/test/scala/filodb.coordinator/NodeCoordinatorActorSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/NodeCoordinatorActorSpec.scala @@ -194,7 +194,6 @@ class NodeCoordinatorActorSpec extends ActorTest(NodeCoordinatorActorSpec.getNew probe.send(coordinatorActor, IngestRows(ref, 0, records(dataset1, linearMultiSeries().take(40)))) probe.expectMsg(Ack(0L)) - memStore.commitIndexForTesting(dataset1.ref) // Try a filtered partition query val series2 = (2 to 4).map(n => s"Series $n").toSet.asInstanceOf[Set[Any]] @@ -203,6 +202,7 @@ class NodeCoordinatorActorSpec extends ActorTest(NodeCoordinatorActorSpec.getNew Aggregate(AggregationOperator.Avg, PeriodicSeries( RawSeries(AllChunksSelector, multiFilter, Seq("min")), 120000L, 10000L, 130000L)), qOpt) + memStore.commitIndexForTesting(dataset1.ref) probe.send(coordinatorActor, q2) probe.expectMsgPF() { case QueryResult(_, schema, vectors) => @@ -347,6 +347,7 @@ class NodeCoordinatorActorSpec extends ActorTest(NodeCoordinatorActorSpec.getNew probe.send(coordinatorActor, StatusActor.GetCurrentEvents) probe.expectMsg(Map(ref -> Seq(IngestionStarted(ref, 0, coordinatorActor)))) + memStore.commitIndexForTesting(dataset6.ref) // Also the original aggregator is sum(sum_over_time(....)) which is not quite represented by below plan // Below plan is really sum each time bucket val q2 = LogicalPlan2Query(ref, diff --git a/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala b/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala index 13a009522f..5787ee7996 100644 --- a/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala +++ b/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala @@ -61,35 +61,45 @@ TimeSeriesShard(dataset, storeConfig, shardNum, rawStore, metastore, evictionPol val allDataCols = dataset.dataColumns.map(_.id) // 1. Fetch partitions from memstore - val indexIt = iteratePartitions(partMethod, chunkMethod) - - // 2. Determine missing chunks per partition and what to fetch + val iterResult = iteratePartitions(partMethod, chunkMethod) + val partIdsNotInMemory = iterResult.partIdsNotInMemory + // 2. Now determine list of partitions to ODP and the time ranges to ODP val partKeyBytesToPage = new ArrayBuffer[Array[Byte]]() - val inMemoryPartitions = new ArrayBuffer[ReadablePartition]() - val methods = new ArrayBuffer[ChunkScanMethod] - indexIt.foreach { p => - chunksToFetch(p, chunkMethod, pagingEnabled).map { rawChunkMethod => - methods += rawChunkMethod // TODO: really determine range for all partitions - partKeyBytesToPage += p.partKeyBytes - }.getOrElse { - // add it to partitions which do not need to be ODP'ed, send these directly and first - inMemoryPartitions += p + val pagingMethods = new ArrayBuffer[ChunkScanMethod] + val inMemOdp = debox.Set.empty[Int] + + iterResult.partIdsInMemoryMayNeedOdp.foreach { case (pId, startTime) => + val p = partitions.get(pId) + if (p != null) { + val odpChunkScan = chunksToODP(p, chunkMethod, pagingEnabled, startTime) + odpChunkScan.foreach { rawChunkMethod => + pagingMethods += rawChunkMethod // TODO: really determine range for all partitions + partKeyBytesToPage += p.partKeyBytes + inMemOdp += p.partID + } + } else { + // in the very rare case that partition literally *just* got evicted + // we do not want to thrash by paging this partition back in. + logger.warn(s"Skipped ODP of partId $pId in dataset=${dataset.ref} " + + s"shard=$shardNum since we are very likely thrashing") } } - shardStats.partitionsQueried.increment(inMemoryPartitions.length) - logger.debug(s"dataset=${dataset.ref} shard=$shardNum Querying ${inMemoryPartitions.length} in memory " + - s"partitions, ODPing ${methods.length}") + logger.debug(s"Query on dataset=${dataset.ref} shard=$shardNum resulted in partial ODP of partIds ${inMemOdp}, " + + s"and full ODP of partIds ${iterResult.partIdsNotInMemory}") + + // partitions that do not need ODP are those that are not in the inMemOdp collection + val noOdpPartitions = iterResult.partsInMemoryIter.filterNot(p => inMemOdp(p.partID)) // NOTE: multiPartitionODP mode does not work with AllChunkScan and unit tests; namely missing partitions will not // return data that is in memory. TODO: fix - Observable.fromIterable(inMemoryPartitions) ++ { + val result = Observable.fromIterator(noOdpPartitions) ++ { if (storeConfig.multiPartitionODP) { - Observable.fromTask(odpPartTask(indexIt, partKeyBytesToPage, methods, chunkMethod)).flatMap { odpParts => + Observable.fromTask(odpPartTask(partIdsNotInMemory, partKeyBytesToPage, pagingMethods, + chunkMethod)).flatMap { odpParts => val multiPart = MultiPartitionScan(partKeyBytesToPage, shardNum) - shardStats.partitionsQueried.increment(partKeyBytesToPage.length) if (partKeyBytesToPage.nonEmpty) { val span = startODPSpan() - rawStore.readRawPartitions(dataset, allDataCols, multiPart, computeBoundingMethod(methods)) + rawStore.readRawPartitions(dataset, allDataCols, multiPart, computeBoundingMethod(pagingMethods)) // NOTE: this executes the partMaker single threaded. Needed for now due to concurrency constraints. // In the future optimize this if needed. .mapAsync { rawPart => partitionMaker.populateRawChunks(rawPart).executeOn(singleThreadPool) } @@ -99,11 +109,11 @@ TimeSeriesShard(dataset, storeConfig, shardNum, rawStore, metastore, evictionPol } else { Observable.empty } } } else { - Observable.fromTask(odpPartTask(indexIt, partKeyBytesToPage, methods, chunkMethod)).flatMap { odpParts => - shardStats.partitionsQueried.increment(partKeyBytesToPage.length) + Observable.fromTask(odpPartTask(partIdsNotInMemory, partKeyBytesToPage, pagingMethods, + chunkMethod)).flatMap { odpParts => if (partKeyBytesToPage.nonEmpty) { val span = startODPSpan() - Observable.fromIterable(partKeyBytesToPage.zip(methods)) + Observable.fromIterable(partKeyBytesToPage.zip(pagingMethods)) .mapAsync(storeConfig.demandPagingParallelism) { case (partBytes, method) => rawStore.readRawPartitions(dataset, allDataCols, SinglePartitionScan(partBytes, shardNum), method) .mapAsync { rawPart => partitionMaker.populateRawChunks(rawPart).executeOn(singleThreadPool) } @@ -117,14 +127,18 @@ TimeSeriesShard(dataset, storeConfig, shardNum, rawStore, metastore, evictionPol } } } + result.map { p => + shardStats.partitionsQueried.increment() + p + } } // 3. Deal with partitions no longer in memory but still indexed in Lucene. // Basically we need to create TSPartitions for them in the ingest thread -- if there's enough memory - private def odpPartTask(indexIt: PartitionIterator, partKeyBytesToPage: ArrayBuffer[Array[Byte]], + private def odpPartTask(partIdsNotInMemory: Buffer[Int], partKeyBytesToPage: ArrayBuffer[Array[Byte]], methods: ArrayBuffer[ChunkScanMethod], chunkMethod: ChunkScanMethod) = - if (indexIt.skippedPartIDs.nonEmpty) { - createODPPartitionsTask(indexIt.skippedPartIDs, { case (bytes, offset) => + if (partIdsNotInMemory.nonEmpty) { + createODPPartitionsTask(partIdsNotInMemory, { case (bytes, offset) => val partKeyBytes = if (offset == UnsafeUtils.arayOffset) bytes else BinaryRegionLarge.asNewByteArray(bytes, offset) partKeyBytesToPage += partKeyBytes @@ -140,7 +154,7 @@ TimeSeriesShard(dataset, storeConfig, shardNum, rawStore, metastore, evictionPol * It runs in ingestion thread so it can correctly verify which ones to actually create or not */ private def createODPPartitionsTask(partIDs: Buffer[Int], callback: (Array[Byte], Int) => Unit): - Task[Seq[TimeSeriesPartition]] = Task { + Task[Seq[TimeSeriesPartition]] = Task { require(partIDs.nonEmpty) partIDs.map { id => // for each partID: look up in partitions @@ -180,34 +194,68 @@ TimeSeriesShard(dataset, storeConfig, shardNum, rawStore, metastore, evictionPol TimeRangeChunkScan(minTime, maxTime) } - private def chunksToFetch(partition: ReadablePartition, - method: ChunkScanMethod, - enabled: Boolean): Option[ChunkScanMethod] = { + /** + * Check if ODP is really needed for this partition which is in memory + * @return Some(scanMethodForCass) if ODP is needed, None if ODP is not needed + */ + private def chunksToODP(partition: ReadablePartition, + method: ChunkScanMethod, + enabled: Boolean, + partStartTime: Long): Option[ChunkScanMethod] = { if (enabled) { method match { // For now, allChunkScan will always load from disk. This is almost never used, and without an index we have // no way of knowing what there is anyways. - case AllChunkScan => Some(AllChunkScan) + case AllChunkScan => Some(AllChunkScan) // Assume initial startKey of first chunk is the earliest - typically true unless we load in historical data // Compare desired time range with start key and see if in memory data covers desired range // Also assume we have in memory all data since first key. Just return the missing range of keys. - case r: TimeRangeChunkScan => - if (partition.numChunks > 0) { - val memStartTime = partition.earliestTime - val endQuery = memStartTime - 1 // do not include earliestTime, otherwise will pull in first chunk - if (r.startTime < memStartTime) { Some(TimeRangeChunkScan(r.startTime, endQuery)) } - else { None } - } else { - Some(r) // if no chunks ingested yet, read everything from disk - } - // Return only in-memory data - ie return none so we never ODP - case InMemoryChunkScan => None - // Write buffers are always in memory only - case WriteBufferChunkScan => None + case req: TimeRangeChunkScan => if (partition.numChunks > 0) { + val memStartTime = partition.earliestTime + if (req.startTime < memStartTime && partStartTime < memStartTime) { + // do not include earliestTime, otherwise will pull in first chunk + Some(TimeRangeChunkScan(req.startTime, memStartTime - 1)) + } + else None + } else Some(req) // if no chunks ingested yet, read everything from disk + case InMemoryChunkScan => None // Return only in-memory data - ie return none so we never ODP + case WriteBufferChunkScan => None // Write buffers are always in memory only } } else { None } + } + + override def iteratePartitions(partMethod: PartitionScanMethod, + chunkMethod: ChunkScanMethod): IterationResult = { + partMethod match { + case SinglePartitionScan(partition, _) => + IterationResult(ByteKeysPartitionIterator(Seq(partition)), debox.Map.empty, debox.Buffer.empty) + case MultiPartitionScan(partKeys, _) => + IterationResult(ByteKeysPartitionIterator(partKeys), debox.Map.empty, debox.Buffer.empty) + case FilteredPartitionScan(split, filters) => + // TODO: There are other filters that need to be added and translated to Lucene queries + val coll = partKeyIndex.partIdsFromFilters2(filters, chunkMethod.startTime, chunkMethod.endTime) + // first find out which partitions are being queried for data not in memory + val it1 = InMemPartitionIterator(coll.intIterator()) + val partIdsToPage = it1.filter(_.earliestTime > chunkMethod.startTime).map(_.partID) + val partIdsNotInMem = it1.skippedPartIDs + val startTimes = if (partIdsToPage.nonEmpty) { + val st = partKeyIndex.startTimeFromPartIds(partIdsToPage) + logger.debug(s"Some partitions have earliestTime > queryStartTime(${chunkMethod.startTime}); " + + s"startTime lookup for query in dataset=${dataset.ref} shard=$shardNum " + + s"resulted in startTimes=$st") + st + } + else { + logger.debug(s"StartTime lookup was not needed. All partition's data for query in dataset=${dataset.ref} " + + s"shard=$shardNum are in memory") + debox.Map.empty[Int, Long] + } + // now provide an iterator that additionally supplies the startTimes for + // those partitions that may need to be paged + IterationResult(new InMemPartitionIterator(coll.intIterator()), startTimes, partIdsNotInMem) + } } } diff --git a/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala b/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala index a2dea2638a..282c62b315 100644 --- a/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala +++ b/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala @@ -321,6 +321,19 @@ class PartKeyLuceneIndex(dataset: Dataset, collector.singleResult } + /** + * Called when a document is updated with new endTime + */ + def startTimeFromPartIds(partIds: Iterator[Int]): debox.Map[Int, Long] = { + val collector = new PartIdStartTimeCollector() + val booleanQuery = new BooleanQuery.Builder + partIds.foreach { pId => + booleanQuery.add(new TermQuery(new Term(PART_ID, pId.toString)), Occur.SHOULD) + } + searcherManager.acquire().search(booleanQuery.build(), collector) + collector.startTimes + } + /** * Called when a document is updated with new endTime */ @@ -423,7 +436,7 @@ class PartKeyLuceneIndex(dataset: Dataset, def partIdsFromFilters2(columnFilters: Seq[ColumnFilter], startTime: Long, - endTime: Long): PartIdCollector = { + endTime: Long): EWAHCompressedBitmap = { val partKeySpan = Kamon.buildSpan("index-partition-lookup-latency") .withTag("dataset", dataset.name) .withTag("shard", shardNum) @@ -441,7 +454,7 @@ class PartKeyLuceneIndex(dataset: Dataset, val collector = new PartIdCollector() // passing zero for unlimited results searcher.search(query, collector) partKeySpan.finish() - collector + collector.result } } @@ -596,6 +609,30 @@ class PartIdCollector extends SimpleCollector { def intIterator(): IntIterator = result.intIterator() } +class PartIdStartTimeCollector extends SimpleCollector { + val startTimes = debox.Map.empty[Int, Long] + private var partIdDv: NumericDocValues = _ + private var startTimeDv: NumericDocValues = _ + + override def needsScores(): Boolean = false + + override def doSetNextReader(context: LeafReaderContext): Unit = { + //set the subarray of the numeric values for all documents in the context + partIdDv = context.reader().getNumericDocValues(PartKeyLuceneIndex.PART_ID) + startTimeDv = context.reader().getNumericDocValues(PartKeyLuceneIndex.START_TIME) + } + + override def collect(doc: Int): Unit = { + if (partIdDv.advanceExact(doc) && startTimeDv.advanceExact(doc)) { + val partId = partIdDv.longValue().toInt + val startTime = startTimeDv.longValue() + startTimes(partId) = startTime + } else { + throw new IllegalStateException("This shouldn't happen since every document should have partIdDv and startTimeDv") + } + } +} + class ActionCollector(action: (Int, BytesRef) => Unit) extends SimpleCollector { private var partIdDv: NumericDocValues = _ private var partKeyDv: BinaryDocValues = _ diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala index b112d8f4e9..34bd7ce270 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala @@ -320,9 +320,13 @@ extends ChunkMap(memFactory, initMapSize) with ReadablePartition { final def unlock(): Unit = chunkmapReleaseShared() } + /** + * startTime of earliest chunk in memory. + * Long.MaxValue if no chunk is present + */ final def earliestTime: Long = { if (numChunks == 0) { - Long.MinValue + Long.MaxValue } else { // Acquire shared lock to safely access the native pointer. chunkmapWithShared(ChunkSetInfo(chunkmapDoGetFirst).startTime) diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala index db2e60dea8..75777a42ee 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala @@ -672,7 +672,8 @@ class TimeSeriesShard(val dataset: Dataset, private def addPartKeyToTimebucketRb(indexRb: RecordBuilder, p: TimeSeriesPartition) = { var startTime = partKeyIndex.startTimeFromPartId(p.partID) - if (startTime == -1) startTime = p.earliestTime// can remotely happen since lucene reads are eventually consistent + if (startTime == -1) startTime = p.earliestTime // can remotely happen since lucene reads are eventually consistent + if (startTime == Long.MaxValue) startTime = 0 // if for any reason we cant find the startTime, use 0 val endTime = if (isActivelyIngesting(p.partID)) { Long.MaxValue } else { @@ -956,13 +957,13 @@ class TimeSeriesShard(val dataset: Dataset, val filters = dataset.partKeySchema.toStringPairs(partKeyBase, partKeyOffset) .map { pair => ColumnFilter(pair._1, Filter.Equals(pair._2)) } val matches = partKeyIndex.partIdsFromFilters2(filters, 0, Long.MaxValue) - matches.result.cardinality() match { + matches.cardinality() match { case 0 => shardStats.evictedPartKeyBloomFilterFalsePositives.increment() CREATE_NEW_PARTID case c if c >= 1 => // NOTE: if we hit one partition, we cannot directly call it out as the result without // verifying the partKey since the matching partition may have had an additional tag if (c > 1) shardStats.evictedPartIdLookupMultiMatch.increment() - val iter = matches.result.intIterator() + val iter = matches.intIterator() var partId = -1 do { // find the most specific match for the given ingestion record @@ -1208,24 +1209,47 @@ class TimeSeriesShard(val dataset: Dataset, part.map(_.asInstanceOf[TimeSeriesPartition]) } + /** + * Result of a iteratePartitions method call. + * + * Note that there is a leak in abstraction here we should not be talking about ODP here. + * ODPagingShard really should not have been a sub-class of TimeSeriesShard. Instead + * composition should have been used instead of inheritance. Overriding the iteratePartitions() + * method is the best I could do to keep the leak minimal and not increase scope. + * + * TODO: clean this all up in a bigger refactoring effort later. + * + * @param partsInMemoryIter iterates through the in-Memory partitions, some of which may not need ODP. + * Caller needs to filter further + * @param partIdsInMemoryMayNeedOdp has partIds from partsInMemoryIter in memory that may need chunk ODP. Their + * startTimes from Lucene are included + * @param partIdsNotInMemory is a collection of partIds fully not in memory + */ + case class IterationResult(partsInMemoryIter: PartitionIterator, + partIdsInMemoryMayNeedOdp: debox.Map[Int, Long] = debox.Map.empty, + partIdsNotInMemory: debox.Buffer[Int] = debox.Buffer.empty) + + /** + * See documentation for IterationResult. + */ def iteratePartitions(partMethod: PartitionScanMethod, - chunkMethod: ChunkScanMethod): PartitionIterator = partMethod match { - case SinglePartitionScan(partition, _) => ByteKeysPartitionIterator(Seq(partition)) - case MultiPartitionScan(partKeys, _) => ByteKeysPartitionIterator(partKeys) + chunkMethod: ChunkScanMethod): IterationResult = partMethod match { + case SinglePartitionScan(partition, _) => IterationResult(ByteKeysPartitionIterator(Seq(partition))) + case MultiPartitionScan(partKeys, _) => IterationResult(ByteKeysPartitionIterator(partKeys)) case FilteredPartitionScan(split, filters) => // TODO: There are other filters that need to be added and translated to Lucene queries if (filters.nonEmpty) { val indexIt = partKeyIndex.partIdsFromFilters(filters, chunkMethod.startTime, chunkMethod.endTime) - new InMemPartitionIterator(indexIt) + IterationResult(new InMemPartitionIterator(indexIt)) } else { - PartitionIterator.fromPartIt(partitions.values.iterator.asScala) + IterationResult(PartitionIterator.fromPartIt(partitions.values.iterator.asScala)) } } def scanPartitions(columnIDs: Seq[Types.ColumnId], partMethod: PartitionScanMethod, chunkMethod: ChunkScanMethod): Observable[ReadablePartition] = { - Observable.fromIterator(iteratePartitions(partMethod, chunkMethod).map { p => + Observable.fromIterator(iteratePartitions(partMethod, chunkMethod).partsInMemoryIter.map { p => shardStats.partitionsQueried.increment() p }) diff --git a/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala b/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala index ef52a9399e..c7c273bfd0 100644 --- a/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala +++ b/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala @@ -82,6 +82,7 @@ class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter w memStore.ingest(dataset1.ref, 0, data) } + memStore.commitIndexForTesting(dataset1.ref) val split = memStore.getScanSplits(dataset1.ref, 1).head val agg1 = memStore.scanRows(dataset1, Seq(1), FilteredPartitionScan(split)).map(_.getDouble(0)).sum agg1 shouldEqual ((1 to 20).map(_.toDouble).sum) @@ -204,6 +205,7 @@ class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter w fut1.futureValue fut2.futureValue + memStore.commitIndexForTesting(dataset1.ref) val splits = memStore.getScanSplits(dataset1.ref, 1) val agg1 = memStore.scanRows(dataset1, Seq(1), FilteredPartitionScan(splits.head)) .map(_.getDouble(0)).sum @@ -242,6 +244,7 @@ class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter w // Two flushes and 3 chunksets have been flushed chunksetsWritten shouldEqual initChunksWritten + 4 + memStore.commitIndexForTesting(dataset1.ref) // Try reading - should be able to read optimized chunks too val splits = memStore.getScanSplits(dataset1.ref, 1) val agg1 = memStore.scanRows(dataset1, Seq(1), FilteredPartitionScan(splits.head)) @@ -336,6 +339,7 @@ class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter w // no flushes chunksetsWritten shouldEqual initChunksWritten + memStore.commitIndexForTesting(dataset1.ref) // Should have less than 50 records ingested // Try reading - should be able to read optimized chunks too val splits = memStore.getScanSplits(dataset1.ref, 1) @@ -401,13 +405,15 @@ class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter w memStore.getShardE(dataset1.ref, 0).evictionWatermark shouldEqual endTime + 1 memStore.getShardE(dataset1.ref, 0).addPartitionsDisabled() shouldEqual false - // Check partitions are now 2 to 21, 0 and 1 got evicted + memStore.commitIndexForTesting(dataset1.ref) val split = memStore.getScanSplits(dataset1.ref, 1).head val parts = memStore.scanPartitions(dataset1, Seq(0, 1), FilteredPartitionScan(split)) .toListL.runAsync .futureValue .asInstanceOf[Seq[TimeSeriesPartition]] - parts.map(_.partID).toSet shouldEqual (2 to 21).toSet + parts.map(_.partID).toSet shouldEqual (2 to 21).toSet ++ Set(0) + // Above query will ODP evicted partition 0 back in, but there is no space for evicted part 1, + // so it will not be returned as part of query :( } it("should be able to ODP/query partitions evicted from memory structures when doing index/tag query") { @@ -511,6 +517,7 @@ class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter w memStore.numPartitions(dataset1.ref, 0) shouldEqual 21 // due to the way the eviction policy works memStore.getShardE(dataset1.ref, 0).evictionWatermark shouldEqual 0 + memStore.commitIndexForTesting(dataset1.ref) // Check partitions are now 0 to 20, 21/22 did not get added val split = memStore.getScanSplits(dataset1.ref, 1).head val parts = memStore.scanPartitions(dataset1, Seq(0, 1), FilteredPartitionScan(split)) From 0779853598ab1c3a2bdeb2e0cc7fbf09e228771b Mon Sep 17 00:00:00 2001 From: sherali42 <43357447+sherali42@users.noreply.github.com> Date: Wed, 27 Mar 2019 13:38:18 -0700 Subject: [PATCH 30/41] bug(query): resets function returns zero instead of NaN for empty window (#298) --- .../filodb/query/exec/rangefn/RangeInstantFunctions.scala | 3 ++- .../scala/filodb/query/exec/rangefn/RateFunctionsSpec.scala | 4 ++++ 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/query/src/main/scala/filodb/query/exec/rangefn/RangeInstantFunctions.scala b/query/src/main/scala/filodb/query/exec/rangefn/RangeInstantFunctions.scala index b6972a3285..e442f79354 100644 --- a/query/src/main/scala/filodb/query/exec/rangefn/RangeInstantFunctions.scala +++ b/query/src/main/scala/filodb/query/exec/rangefn/RangeInstantFunctions.scala @@ -119,11 +119,12 @@ object DerivFunction extends RangeFunction { } class ResetsFunction extends RangeFunction { - var resets = 0 + var resets = Double.NaN // NaN for windows that do not have data def addedToWindow(row: TransientRow, window: Window): Unit = { val size = window.size if (size > 1 && window(size - 2).value > row.value) { + if (resets.isNaN) resets = 0 resets += 1 } } diff --git a/query/src/test/scala/filodb/query/exec/rangefn/RateFunctionsSpec.scala b/query/src/test/scala/filodb/query/exec/rangefn/RateFunctionsSpec.scala index 24f19965c6..457bbb1ec6 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/RateFunctionsSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/RateFunctionsSpec.scala @@ -83,6 +83,10 @@ class RateFunctionsSpec extends FunSpec with Matchers { val q3 = new IndexedArrayQueue[TransientRow]() val gaugeWindowForReset = new QueueBasedWindow(q3) val resetsFunction = new ResetsFunction + + resetsFunction.apply(startTs, endTs, gaugeWindowForReset, toEmit, queryConfig) + assert(toEmit.value.isNaN) // Empty window should return NaN + gaugeSamples.foreach { case (t, v) => val s = new TransientRow(t, v) q3.add(s) From ad413a85b7d41cd93aa0422f1858bd1fccc01c04 Mon Sep 17 00:00:00 2001 From: sherali42 <43357447+sherali42@users.noreply.github.com> Date: Thu, 28 Mar 2019 11:23:11 -0700 Subject: [PATCH 31/41] bug(query): resets function bug fix (#300) bug(query): update resets function spec to capture empty windows and for data with no counter resets --- .../exec/rangefn/RangeInstantFunctions.scala | 2 +- .../exec/rangefn/RateFunctionsSpec.scala | 32 +++++++++++++++++-- 2 files changed, 31 insertions(+), 3 deletions(-) diff --git a/query/src/main/scala/filodb/query/exec/rangefn/RangeInstantFunctions.scala b/query/src/main/scala/filodb/query/exec/rangefn/RangeInstantFunctions.scala index e442f79354..f83fb3222d 100644 --- a/query/src/main/scala/filodb/query/exec/rangefn/RangeInstantFunctions.scala +++ b/query/src/main/scala/filodb/query/exec/rangefn/RangeInstantFunctions.scala @@ -123,8 +123,8 @@ class ResetsFunction extends RangeFunction { def addedToWindow(row: TransientRow, window: Window): Unit = { val size = window.size + if (resets.isNaN && size > 0) resets = 0 if (size > 1 && window(size - 2).value > row.value) { - if (resets.isNaN) resets = 0 resets += 1 } } diff --git a/query/src/test/scala/filodb/query/exec/rangefn/RateFunctionsSpec.scala b/query/src/test/scala/filodb/query/exec/rangefn/RateFunctionsSpec.scala index 457bbb1ec6..c354063712 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/RateFunctionsSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/RateFunctionsSpec.scala @@ -75,18 +75,46 @@ class RateFunctionsSpec extends FunSpec with Matchers { Math.abs(toEmit.value - expected) should be < errorOk } - it ("resets should work when start and end are outside window") { + it ("resets should work with empty windows and no resets data") { val startTs = 8071950L val endTs = 8163070L - val expected = 4.0 val toEmit = new TransientRow val q3 = new IndexedArrayQueue[TransientRow]() val gaugeWindowForReset = new QueueBasedWindow(q3) val resetsFunction = new ResetsFunction + val counterSamples = Seq( 8072000L->1419.00, + 8082100L->2511.00, + 8092196L->3614.00, + 8102215L->4724.00, + 8112223L->5909.00, + 8122388L->6948.00, + 8132570L->7000.00, + 8142822L->8095.00, + 8152858L->9102.00, + 8163000L->9201.00) + resetsFunction.apply(startTs, endTs, gaugeWindowForReset, toEmit, queryConfig) assert(toEmit.value.isNaN) // Empty window should return NaN + counterSamples.foreach { case (t, v) => + val s = new TransientRow(t, v) + q3.add(s) + resetsFunction.addedToWindow(s, gaugeWindowForReset) + } + resetsFunction.apply(startTs, endTs, gaugeWindowForReset, toEmit, queryConfig) + toEmit.value shouldEqual 0 + } + + it ("resets should work when start and end are outside window") { + val startTs = 8071950L + val endTs = 8163070L + val expected = 4.0 + val toEmit = new TransientRow + val q3 = new IndexedArrayQueue[TransientRow]() + val gaugeWindowForReset = new QueueBasedWindow(q3) + val resetsFunction = new ResetsFunction + gaugeSamples.foreach { case (t, v) => val s = new TransientRow(t, v) q3.add(s) From fbb958caa30a5fd2593864cf2fc9ac647deb5329 Mon Sep 17 00:00:00 2001 From: jackson-paul Date: Thu, 28 Mar 2019 15:57:22 -0700 Subject: [PATCH 32/41] Update version to 0.8.4-integration --- version.sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/version.sbt b/version.sbt index 1b0cb1bf45..8682288b4c 100644 --- a/version.sbt +++ b/version.sbt @@ -1 +1 @@ -version in ThisBuild := "0.8.3-integration-SNAPSHOT" +version in ThisBuild := "0.8.4-integration-SNAPSHOT" From 0f2e9e562239b8c823c257cd19aa0d5615c7c261 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Thu, 28 Mar 2019 16:42:37 -0700 Subject: [PATCH 33/41] feat(query): Parallelize scanPartitions and all QE work in query threadpool (#293) --- .../scala/filodb.coordinator/QueryActor.scala | 4 ++- .../NodeCoordinatorActorSpec.scala | 27 +++++++++++++++++ core/src/main/resources/filodb-defaults.conf | 30 ++++++------------- .../filodb.jmh/QueryAndIngestBenchmark.scala | 18 +++++------ .../query/exec/AggrOverRangeVectors.scala | 4 ++- .../scala/filodb/query/exec/ExecPlan.scala | 10 +++++-- .../query/exec/PeriodicSamplesMapper.scala | 4 ++- 7 files changed, 61 insertions(+), 36 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala b/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala index 7f458dc617..d01bb625e6 100644 --- a/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala +++ b/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala @@ -8,6 +8,7 @@ import akka.actor.{ActorRef, ActorSystem, Props} import akka.dispatch.{Envelope, UnboundedStablePriorityMailbox} import com.typesafe.config.Config import kamon.Kamon +import monix.execution.Scheduler import filodb.coordinator.queryengine2.QueryEngine import filodb.core._ @@ -52,11 +53,12 @@ final class QueryActor(memStore: MemStore, import QueryActor._ import client.QueryCommands._ - implicit val scheduler = monix.execution.Scheduler(context.dispatcher) val config = context.system.settings.config val queryEngine2 = new QueryEngine(dataset, shardMapFunc) val queryConfig = new QueryConfig(config.getConfig("filodb.query")) + val numSchedThreads = Math.ceil(config.getDouble("filodb.query.threads-factor") * sys.runtime.availableProcessors) + implicit val scheduler = Scheduler.fixedPool(s"query-${dataset.ref}", numSchedThreads.toInt) private val tags = Map("dataset" -> dataset.ref.toString) private val lpRequests = Kamon.counter("queryactor-logicalPlan-requests").refine(tags) diff --git a/coordinator/src/test/scala/filodb.coordinator/NodeCoordinatorActorSpec.scala b/coordinator/src/test/scala/filodb.coordinator/NodeCoordinatorActorSpec.scala index e2369a1010..011ea417d4 100644 --- a/coordinator/src/test/scala/filodb.coordinator/NodeCoordinatorActorSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/NodeCoordinatorActorSpec.scala @@ -238,6 +238,33 @@ class NodeCoordinatorActorSpec extends ActorTest(NodeCoordinatorActorSpec.getNew } } + it("should parse and execute concurrent LogicalPlan queries") { + val ref = setupTimeSeries() + probe.send(coordinatorActor, IngestRows(ref, 0, records(dataset1, linearMultiSeries().take(40)))) + probe.expectMsg(Ack(0L)) + + memStore.commitIndexForTesting(dataset1.ref) + + val numQueries = 6 + + val series2 = (2 to 4).map(n => s"Series $n").toSet.asInstanceOf[Set[Any]] + val multiFilter = Seq(ColumnFilter("series", Filter.In(series2))) + val q2 = LogicalPlan2Query(ref, + Aggregate(AggregationOperator.Avg, + PeriodicSeries( + RawSeries(AllChunksSelector, multiFilter, Seq("min")), 120000L, 10000L, 130000L)), qOpt) + (0 until numQueries).foreach { i => probe.send(coordinatorActor, q2) } + + (0 until numQueries).foreach { _ => + probe.expectMsgPF() { + case QueryResult(_, schema, vectors) => + schema shouldEqual timeMinSchema + vectors should have length (1) + vectors(0).rows.map(_.getDouble(1)).toSeq shouldEqual Seq(14.0, 24.0) + } + } + } + ignore("should aggregate from multiple shards") { val ref = setupTimeSeries(2) probe.send(coordinatorActor, IngestRows(ref, 0, records(dataset1, linearMultiSeries().take(30)))) diff --git a/core/src/main/resources/filodb-defaults.conf b/core/src/main/resources/filodb-defaults.conf index d4f72108bf..33f753c5c4 100644 --- a/core/src/main/resources/filodb-defaults.conf +++ b/core/src/main/resources/filodb-defaults.conf @@ -38,6 +38,9 @@ filodb { # Minimum step required for a query min-step = 5 seconds + + # Parallelism (query threadpool per dataset) ... ceil(available processors * factor) + threads-factor = 1.0 } shard-manager { @@ -243,6 +246,12 @@ akka { "filodb.query.LogicalPlan" = kryo } + # Reduce the number of threads used by default by the fork-join pool, as it's not really doing much work. + default-dispatcher.fork-join-executor { + parallelism-factor = 2.0 + parallelism-max = 32 + } + # Just the defaults to start with. TODO optimize and pick the executor needed. shard-status-dispatcher { type = Dispatcher @@ -303,27 +312,6 @@ akka { } } - # Just the defaults to start with. TODO optimize and pick the executor needed. - shard-status-dispatcher { - # Dispatcher is the name of the event-based dispatcher - type = Dispatcher - # What kind of ExecutionService to use - executor = "fork-join-executor" - # Configuration for the fork join pool - fork-join-executor { - # Min number of threads to cap factor-based parallelism number to - parallelism-min = 2 - # Parallelism (threads) ... ceil(available processors * factor) - parallelism-factor = 2.0 - # Max number of threads to cap factor-based parallelism number to - parallelism-max = 10 - } - # Throughput defines the maximum number of messages to be - # processed per actor before the thread jumps to the next actor. - # Set to 1 for as fair as possible. - throughput = 100 - } - # Be sure to terminate/exit JVM process after Akka shuts down. This is important for the # custom downing provider's split brain resolution to work properly. Basically, the minority # group will shut down itself and exit the process, helping to bring newer nodes online. diff --git a/jmh/src/main/scala/filodb.jmh/QueryAndIngestBenchmark.scala b/jmh/src/main/scala/filodb.jmh/QueryAndIngestBenchmark.scala index 6f980ee40e..9d7111cb40 100644 --- a/jmh/src/main/scala/filodb.jmh/QueryAndIngestBenchmark.scala +++ b/jmh/src/main/scala/filodb.jmh/QueryAndIngestBenchmark.scala @@ -41,12 +41,12 @@ class QueryAndIngestBenchmark extends StrictLogging { import NodeClusterActor._ val numShards = 2 - val numSamples = 720 // 2 hours * 3600 / 10 sec interval - val numSeries = 100 + val numSamples = 1080 // 3 hours * 3600 / 10 sec interval + val numSeries = 800 var startTime = System.currentTimeMillis - (3600*1000) - val numQueries = 500 // Please make sure this number matches the OperationsPerInvocation below - val queryIntervalMin = 55 // # minutes between start and stop - val queryStep = 60 // # of seconds between each query sample "step" + val numQueries = 200 // Please make sure this number matches the OperationsPerInvocation below + val queryIntervalMin = 180 // # minutes between start and stop + val queryStep = 60 // # of seconds between each query sample "step" val spread = 1 // TODO: move setup and ingestion to another trait @@ -122,14 +122,14 @@ class QueryAndIngestBenchmark extends StrictLogging { * They are designed to match all the time series (common case) under a particular metric and job */ val queries = Seq("heap_usage{_ns=\"App-2\"}", // raw time series - """sum(rate(heap_usage{_ns="App-2"}[5m]))""", + """sum(rate(heap_usage{_ns="App-2"}[1m]))""", """quantile(0.75, heap_usage{_ns="App-2"})""", - """sum_over_time(heap_usage{_ns="App-2"}[5m])""") + """sum_over_time(heap_usage{_ns="App-2"}[1m])""") val queryTime = startTime + (5 * 60 * 1000) // 5 minutes from start until 60 minutes from start val qParams = TimeStepParams(queryTime/1000, queryStep, (queryTime/1000) + queryIntervalMin*60) val logicalPlans = queries.map { q => Parser.queryRangeToLogicalPlan(q, qParams) } val queryCommands = logicalPlans.map { plan => - LogicalPlan2Query(dataset.ref, plan, QueryOptions(1, 20000)) + LogicalPlan2Query(dataset.ref, plan, QueryOptions(1, 1000000)) } private var testProducingFut: Option[Future[Unit]] = None @@ -156,7 +156,7 @@ class QueryAndIngestBenchmark extends StrictLogging { @Benchmark @BenchmarkMode(Array(Mode.Throughput)) @OutputTimeUnit(TimeUnit.SECONDS) - @OperationsPerInvocation(500) + @OperationsPerInvocation(200) def parallelQueries(): Unit = { val futures = (0 until numQueries).map { n => val f = asyncAsk(coordinator, queryCommands(n % queryCommands.length)) diff --git a/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala b/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala index 332a4d7f4b..29e96190e0 100644 --- a/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala +++ b/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala @@ -5,6 +5,7 @@ import java.nio.ByteBuffer import scala.collection.mutable import com.tdunning.math.stats.{ArrayDigest, TDigest} +import com.typesafe.scalalogging.StrictLogging import monix.reactive.Observable import filodb.core.binaryrecord2.RecordBuilder @@ -111,7 +112,7 @@ final case class AggregatePresenter(aggrOp: AggregationOperator, * * This singleton is the facade for the above operations. */ -object RangeVectorAggregator { +object RangeVectorAggregator extends StrictLogging { /** * This method is the facade for map and reduce steps of the aggregation. @@ -146,6 +147,7 @@ object RangeVectorAggregator { rowAgg: RowAggregator, skipMapPhase: Boolean, grouping: RangeVector => RangeVectorKey): Map[RangeVectorKey, Iterator[rowAgg.AggHolderType]] = { + logger.trace(s"mapReduceInternal on ${rvs.size} RangeVectors...") var acc = rowAgg.zero val mapInto = rowAgg.newRowToMapInto rvs.groupBy(grouping).mapValues { rvs => diff --git a/query/src/main/scala/filodb/query/exec/ExecPlan.scala b/query/src/main/scala/filodb/query/exec/ExecPlan.scala index fe4e1be5df..0d4d7ec6b0 100644 --- a/query/src/main/scala/filodb/query/exec/ExecPlan.scala +++ b/query/src/main/scala/filodb/query/exec/ExecPlan.scala @@ -98,7 +98,10 @@ trait ExecPlan extends QueryCommand { queryConfig: QueryConfig) (implicit sched: Scheduler, timeout: FiniteDuration): Task[QueryResponse] = { - try { + // NOTE: we launch the preparatory steps as a Task too. This is important because scanPartitions, + // Lucene index lookup, and On-Demand Paging orchestration work could suck up nontrivial time and + // we don't want these to happen in a single thread. + Task { qLogger.debug(s"queryId: ${id} Started ExecPlan ${getClass.getSimpleName} with $args") val res = doExecute(source, dataset, queryConfig) val schema = schemaOfDoExecute(dataset) @@ -148,9 +151,10 @@ trait ExecPlan extends QueryCommand { qLogger.error(s"queryId: ${id} Exception during execution of query: ${printTree(false)}", ex) QueryError(id, ex) } - } catch { case NonFatal(ex) => + }.flatten + .onErrorRecover { case NonFatal(ex) => qLogger.error(s"queryId: ${id} Exception during orchestration of query: ${printTree(false)}", ex) - Task(QueryError(id, ex)) + QueryError(id, ex) } } diff --git a/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala b/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala index 9a382d1491..de0c9cae83 100644 --- a/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala +++ b/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala @@ -11,6 +11,7 @@ import filodb.core.store.{ChunkSetInfo, WindowedChunkIterator} import filodb.memory.format.{vectors => bv, RowReader} import filodb.query.{BadQueryException, Query, QueryConfig, RangeFunctionId} import filodb.query.exec.rangefn.{ChunkedRangeFunction, RangeFunction, Window} +import filodb.query.Query.qLogger import filodb.query.util.IndexedArrayQueue /** @@ -62,6 +63,7 @@ final case class PeriodicSamplesMapper(start: Long, } 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)) @@ -142,7 +144,7 @@ extends Iterator[R] with StrictLogging { case e: Exception => val timestampVector = nextInfo.vectorPtr(rv.timestampColID) val tsReader = bv.LongBinaryVector(timestampVector) - logger.error(s"addChunks Exception: info.numRows=${nextInfo.numRows} " + + qLogger.error(s"addChunks Exception: info.numRows=${nextInfo.numRows} " + s"info.endTime=${nextInfo.endTime} curWindowEnd=${wit.curWindowEnd} " + s"tsReader=$tsReader timestampVectorLength=${tsReader.length(timestampVector)}") throw e From 283bdc4f7940aaca930e310f24a0eaf22fd02fb3 Mon Sep 17 00:00:00 2001 From: "Brian S. O'Neill" Date: Sun, 31 Mar 2019 12:25:02 -0700 Subject: [PATCH 34/41] bug(coordinator): Use a thread-safe random numnber generator for dispatcher selection. (#303) --- .../filodb.coordinator/queryengine2/QueryEngine.scala | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala b/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala index 5b5ffa1a99..d003c9eda7 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala @@ -1,6 +1,7 @@ package filodb.coordinator.queryengine2 -import java.util.{SplittableRandom, UUID} +import java.util.UUID +import java.util.concurrent.ThreadLocalRandom import scala.concurrent.ExecutionContext import scala.concurrent.duration.FiniteDuration @@ -362,10 +363,7 @@ class QueryEngine(dataset: Dataset, val childTargets = children.map(_.dispatcher) // Above list can contain duplicate dispatchers, and we don't make them distinct. // Those with more shards must be weighed higher - childTargets.iterator.drop(QueryEngine.random.nextInt(childTargets.size)).next + val rnd = ThreadLocalRandom.current() + childTargets.iterator.drop(rnd.nextInt(childTargets.size)).next } } - -object QueryEngine { - val random = new SplittableRandom() -} From a5a178f8a9133d9e534451fd8c73c9594fb77e83 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Mon, 1 Apr 2019 17:07:08 -0700 Subject: [PATCH 35/41] feat(gateway): Add --gen-prom-data and --gen-hist-data to dev gateway; multiple schema support (#279) --- README.md | 9 +- conf/histogram-dev-source.conf | 57 ++++++ .../scala/filodb.core/metadata/Dataset.scala | 3 +- dev-gateway.sh | 8 +- .../scala/filodb/gateway/GatewayServer.scala | 82 +++++++-- .../gateway/conversion/InputRecord.scala | 33 +++- .../timeseries/TestTimeseriesProducer.scala | 164 ++++++++++-------- .../filodb/http/ClusterApiRouteSpec.scala | 23 +-- .../filodb.jmh/PartKeyIndexBenchmark.scala | 12 +- .../filodb.jmh/QueryAndIngestBenchmark.scala | 7 +- .../scala/filodb/kafka/TestConsumer.scala | 31 ++-- project/FiloBuild.scala | 2 +- 12 files changed, 290 insertions(+), 141 deletions(-) create mode 100644 conf/histogram-dev-source.conf diff --git a/README.md b/README.md index 1ed775acfb..f74c72f070 100644 --- a/README.md +++ b/README.md @@ -202,10 +202,10 @@ You can also check the server logs at `logs/filodb-server-N.log`. Now run the time series generator. This will ingest 20 time series (the default) with 100 samples each into the Kafka topic with current timestamps. The required argument is the path to the source config. Use `--help` for all the options. ``` -java -cp gateway/target/scala-2.11/gateway-*-SNAPSHOT filodb.timeseries.TestTimeseriesProducer -c conf/timeseries-dev-source.conf +./dev-gateway.sh --gen-prom-data conf/timeseries-dev-source.conf ``` -NOTE: The `TestTimeseriesProducer` logs to logs/gateway-server.log. +NOTE: Check logs/gateway-server.log for logs. At this point, you should be able to confirm such a message in the server logs: `KAMON counter name=memstore-rows-ingested count=4999` @@ -220,8 +220,9 @@ You can also look at Cassandra to check for persisted data. Look at the tables i If the above does not work, try the following: 1) Delete the Kafka topic and re-create it. Note that Kafka topic deletion might not happen until the server is stopped and restarted +1a) Restart Kafka, this is sometimes necessary. 2) `./filodb-dev-stop.sh` and restart filodb instances like above -3) Re-run the `TestTimeseriesProducer`. You can check consumption via running the `TestConsumer`, like this: `java -Xmx4G -cp standalone/target/scala-2.11/standalone-assembly-0.8-SNAPSHOT.jar filodb.kafka.TestConsumer conf/timeseries-dev-source.conf`. Also, the `memstore_rows_ingested` metric which is logged to `logs/filodb-server-N.log` should become nonzero. +3) Re-run `./dev-gateway.sh --gen-prom-data`. You can check consumption via running the `TestConsumer`, like this: `java -Xmx4G -Dconfig.file=conf/timeseries-filodb-server.conf -cp standalone/target/scala-2.11/standalone-assembly-0.8-SNAPSHOT.jar filodb.kafka.TestConsumer conf/timeseries-dev-source.conf`. Also, the `memstore_rows_ingested` metric which is logged to `logs/filodb-server-N.log` should become nonzero. To stop the dev server. Note that this will stop all the FiloDB servers if multiple are running. ``` @@ -308,7 +309,7 @@ Now if you curl the cluster status you should see 128 shards which are slowly tu Generate records: ``` -java -cp gateway/target/scala-2.11/gateway-*.telemetry-SNAPSHOT filodb.timeseries.TestTimeseriesProducer -c conf/timeseries-128shards-source.conf -p 5000 +./dev-gateway.sh --gen-prom-data -p 5000 conf/timeseries-128shards-source.conf ``` ## Understanding the FiloDB Data Model diff --git a/conf/histogram-dev-source.conf b/conf/histogram-dev-source.conf new file mode 100644 index 0000000000..51b7c6e39c --- /dev/null +++ b/conf/histogram-dev-source.conf @@ -0,0 +1,57 @@ + dataset = "histogram" + num-shards = 4 + min-num-nodes = 2 + sourcefactory = "filodb.kafka.KafkaIngestionStreamFactory" + + sourceconfig { + # Required FiloDB configurations + filo-topic-name = "histogram-dev" + + # Standard kafka configurations, e.g. + # This accepts both the standard kafka value of a comma-separated + # string and a Typesafe list of String values + # EXCEPT: do not populate value.deserializer, as the Kafka format is fixed in FiloDB to be messages of RecordContainer's + bootstrap.servers = "localhost:9092" + group.id = "filo-db-histogram-ingestion" + + # Values controlling in-memory store chunking, flushing, etc. + store { + # Interval it takes to flush ALL time series in a shard. This time is further divided by groups-per-shard + flush-interval = 1h + + # TTL for on-disk / C* data. Data older than this may be purged. + disk-time-to-live = 24 hours + + # amount of time paged chunks should be retained in memory. + # We need to have a minimum of x hours free blocks or else init won't work. + demand-paged-chunk-retention-period = 12 hours + + max-chunks-size = 400 + + # Write buffer size, in bytes, for blob columns (histograms, UTF8Strings). Since these are variable data types, + # we need a maximum size, not a maximum number of items. + max-blob-buffer-size = 15000 + + # Number of bytes of offheap mem to allocate to chunk storage in each shard. Ex. 1000MB, 1G, 2GB + # Assume 5 bytes per sample, should be roughly equal to (# samples per time series) * (# time series) + shard-mem-size = 512MB + + # Number of bytes of offheap mem to allocate to write buffers in each shard. Ex. 1000MB, 1G, 2GB + # Scales with the number of time series a shard should hold + ingestion-buffer-mem-size = 50MB + + # Number of time series to evict at a time. + # num-partitions-to-evict = 1000 + + # Number of subgroups within each shard. Persistence to a ChunkSink occurs one subgroup at a time, as does + # recovery from failure. This many batches of flushes must occur to cover persistence of every partition + groups-per-shard = 20 + + # Use a "MultiPartitionScan" or Cassandra MULTIGET for on-demand paging. Might improve performance. + multi-partition-odp = false + } + downsample { + # can be disabled by setting this flag to false + enabled = false + } + } \ No newline at end of file diff --git a/core/src/main/scala/filodb.core/metadata/Dataset.scala b/core/src/main/scala/filodb.core/metadata/Dataset.scala index b7725dfc69..decc5629d2 100644 --- a/core/src/main/scala/filodb.core/metadata/Dataset.scala +++ b/core/src/main/scala/filodb.core/metadata/Dataset.scala @@ -12,7 +12,7 @@ import filodb.core.downsample.ChunkDownsampler import filodb.core.query.ColumnInfo import filodb.core.store.ChunkSetInfo import filodb.memory.{BinaryRegion, MemFactory} -import filodb.memory.format.{BinaryVector, RowReader, TypedIterator} +import filodb.memory.format.{BinaryVector, RowReader, TypedIterator, ZeroCopyUTF8String => ZCUTF8} /** * A dataset describes the schema (column name & type) and distribution for a stream/set of data. @@ -169,6 +169,7 @@ case class DatasetOptions(shardKeyColumns: Seq[String], val nonMetricShardColumns = shardKeyColumns.filterNot(_ == metricColumn).sorted val nonMetricShardKeyBytes = nonMetricShardColumns.map(_.getBytes).toArray + val nonMetricShardKeyUTF8 = nonMetricShardColumns.map(ZCUTF8.apply).toArray val nonMetricShardKeyHash = nonMetricShardKeyBytes.map(BinaryRegion.hash32) val ignorePartKeyHashTags = ignoreTagsOnPartitionKeyHash.toSet diff --git a/dev-gateway.sh b/dev-gateway.sh index f4d4d45764..bfa98bd9f4 100755 --- a/dev-gateway.sh +++ b/dev-gateway.sh @@ -1,2 +1,8 @@ #!/usr/bin/env bash -java -cp gateway/target/scala-2.11/gateway-* filodb.gateway.GatewayServer conf/timeseries-dev-source.conf & \ No newline at end of file +# +# Starts a local Gateway for ingesting data into FiloDB (run with no options) +# Type --help to see options - options include generating random test data and exiting. +args=${@:-"conf/timeseries-dev-source.conf"} +java -Dconfig.file=conf/timeseries-filodb-server.conf \ + -Dkamon.prometheus.embedded-server.port=9097 \ + -cp gateway/target/scala-2.11/gateway-* filodb.gateway.GatewayServer $args & \ No newline at end of file diff --git a/gateway/src/main/scala/filodb/gateway/GatewayServer.scala b/gateway/src/main/scala/filodb/gateway/GatewayServer.scala index e730b9ddd8..49e5f6712e 100644 --- a/gateway/src/main/scala/filodb/gateway/GatewayServer.scala +++ b/gateway/src/main/scala/filodb/gateway/GatewayServer.scala @@ -4,8 +4,8 @@ import java.net.InetSocketAddress import java.nio.charset.Charset import java.util.concurrent.Executors +import scala.concurrent.{Await, Future} import scala.concurrent.duration._ -import scala.concurrent.Future import scala.util.control.NonFatal import com.typesafe.config.{Config, ConfigFactory} @@ -25,25 +25,42 @@ import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory import org.jboss.netty.handler.ssl.SslContext import org.jboss.netty.handler.ssl.util.SelfSignedCertificate import org.jctools.queues.MpscGrowableArrayQueue +import org.rogach.scallop._ -import filodb.coordinator.{GlobalConfig, ShardMapper} +import filodb.coordinator.{FilodbSettings, GlobalConfig, ShardMapper, StoreFactory} import filodb.core.binaryrecord2.RecordBuilder import filodb.core.metadata.Dataset import filodb.gateway.conversion._ import filodb.memory.MemFactory -import filodb.prometheus.FormatConversion +import filodb.timeseries.TestTimeseriesProducer /** * Gateway server to ingest source streams of data, shard, batch, and write output to Kafka * built using high performance Netty TCP code * - * It takes exactly one arg: the source config file which contains # Kafka partitions/shards and other config - * Also pass in -Dconfig.file=.... as usual + * It usually takes one arg: the source config file which contains # Kafka partitions/shards and other config + * Also pass in -Dconfig.file=.... as usual, with a config that points to the dataset metadata. + * For local setups, simply run `./dev-gateway.sh`. + * For help pass in `--help`. + * + * NOTE: set `kamon.prometheus.embedded-server.port` to avoid conflicting with FiloDB itself. + * + * There are options that can be used to generate test data, such as `--gen-hist-data`. The -n and -p options can + * also be used together to control the # of samples per series and # of time series. + * To generate Histogram schema test data, one must create the following dataset: + * ./filo-cli -Dconfig.file=conf/timeseries-filodb-server.conf --command create --dataset histogram \ + * --dataColumns timestamp:ts,sum:long,count:long,h:hist --partitionColumns metric:string,tags:map \ + * --shardKeyColumns metric --metricColumn metric + * create a Kafka topic: + * kafka-topics --create --zookeeper localhost:2181 --replication-factor 1 --partitions 4 --topic histogram-dev + * and use the `conf/histogram-dev-source.conf` config file. + * Oh, and you have to observe on shards 1 and 3. */ object GatewayServer extends StrictLogging { // Get global configuration using universal FiloDB/Akka-based config val config = GlobalConfig.systemConfig + val storeFactory = StoreFactory(new FilodbSettings(config), Scheduler.io()) // ==== Metrics ==== val numInfluxMessages = Kamon.counter("num-influx-messages") @@ -52,21 +69,29 @@ object GatewayServer extends StrictLogging { val numContainersSent = Kamon.counter("num-containers-sent") val containersSize = Kamon.histogram("containers-size-bytes") + // Most options are for generating test data + class GatewayOptions(args: Seq[String]) extends ScallopConf(args) { + val samplesPerSeries = opt[Int](short = 'n', default = Some(100), + descr="# of samples per time series") + val numSeries = opt[Int](short='p', default = Some(20), descr="# of total time series") + val sourceConfigPath = trailArg[String](descr="Path to source config, eg conf/timeseries-dev-source.conf") + val genHistData = toggle(noshort=true, descrYes="Generate histogram-schema test data and exit") + val genPromData = toggle(noshort=true, descrYes="Generate Prometheus-schema test data and exit") + verify() + } + + //scalastyle:off method.length def main(args: Array[String]): Unit = { Kamon.loadReportersFromConfig() + val userOpts = new GatewayOptions(args) + val numSamples = userOpts.samplesPerSeries() * userOpts.numSeries() + val numSeries = userOpts.numSeries() - if (args.length < 1) { - //scalastyle:off - println("Arguments: [path/to/source-config.conf]") - //scalastyle:on - sys.exit(1) - } - - val sourceConfig = ConfigFactory.parseFile(new java.io.File(args.head)) + val sourceConfig = ConfigFactory.parseFile(new java.io.File(userOpts.sourceConfigPath())) val numShards = sourceConfig.getInt("num-shards") - // TODO: get the dataset from source config and read the definition from Metastore - val dataset = FormatConversion.dataset + val datasetStr = sourceConfig.getString("dataset") + val dataset = Await.result(storeFactory.metaStore.getDataset(datasetStr), 30.seconds) // NOTE: the spread MUST match the default spread used in the HTTP module for consistency between querying // and ingestion sharding @@ -97,8 +122,33 @@ object GatewayServer extends StrictLogging { // TODO: allow configurable sinks, maybe multiple sinks for say writing to multiple Kafka clusters/DCs setupKafkaProducer(sourceConfig, containerStream) - setupTCPService(config, calcShardAndQueueHandler) + + val genHist = userOpts.genHistData.getOrElse(false) + val genProm = userOpts.genPromData.getOrElse(false) + if (genHist || genProm) { + val startTime = System.currentTimeMillis + logger.info(s"Generating $numSamples samples starting at $startTime....") + + val stream = if (genHist) TestTimeseriesProducer.genHistogramData(startTime, dataset, numSeries) + else TestTimeseriesProducer.timeSeriesData(startTime, numSeries) + + stream.take(numSamples).foreach { rec => + val shard = shardMapper.ingestionShard(rec.shardKeyHash, rec.partitionKeyHash, spread) + if (!shardQueues(shard).offer(rec)) { + // Prioritize recent data. This means dropping messages when full, so new data may have a chance. + logger.warn(s"Queue for shard=$shard is full. Dropping data.") + numDroppedMessages.increment + } + } + Thread sleep 10000 + TestTimeseriesProducer.logQueryHelp(numSamples, numSeries, startTime) + logger.info(s"Waited for containers to be sent, exiting...") + sys.exit(0) + } else { + setupTCPService(config, calcShardAndQueueHandler) + } } + //scalastyle:on method.length def setupTCPService(config: Config, handler: ChannelBuffer => Unit): Unit = { val influxPort = config.getInt("gateway.influx-port") diff --git a/gateway/src/main/scala/filodb/gateway/conversion/InputRecord.scala b/gateway/src/main/scala/filodb/gateway/conversion/InputRecord.scala index c6107ca38d..b31afdcaf8 100644 --- a/gateway/src/main/scala/filodb/gateway/conversion/InputRecord.scala +++ b/gateway/src/main/scala/filodb/gateway/conversion/InputRecord.scala @@ -7,6 +7,7 @@ import scalaxy.loops._ import filodb.core.binaryrecord2.RecordBuilder import filodb.core.metadata.Dataset +import filodb.memory.format.{SeqRowReader, ZeroCopyUTF8String => ZCUTF8} /** * An InputRecord represents one "record" of timeseries data for input to FiloDB system. @@ -124,4 +125,34 @@ object PrometheusInputRecord { } tags ++ extraTags } -} \ No newline at end of file +} + +/** + * A generic InputRecord that can serve different data schemas, so long as the partition key consists of: + * - a single metric StringColumn + * - a MapColumn of tags + * + * Can be used to adapt custom dataset/schemas for input into FiloDB using the gateway. + * Not going to be the fastest InputRecord but extremely flexible. + * + * @param values the data column values, first one is probably timestamp + */ +class MetricTagInputRecord(values: Seq[Any], + metric: String, + tags: Map[ZCUTF8, ZCUTF8], + dataset: Dataset) extends InputRecord { + final def shardKeyHash: Int = RecordBuilder.shardKeyHash(nonMetricShardValues, metric) + // NOTE: this is probably not very performant right now. + final def partitionKeyHash: Int = tags.hashCode + + val nonMetricShardValues: Seq[String] = + dataset.options.nonMetricShardKeyUTF8.flatMap(tags.get).map(_.toString).toSeq + final def getMetric: String = metric + + def addToBuilder(builder: RecordBuilder): Unit = { + require(builder.schema == dataset.ingestionSchema) + + val reader = SeqRowReader(values :+ metric :+ tags) + builder.addFromReader(reader) + } +} diff --git a/gateway/src/main/scala/filodb/timeseries/TestTimeseriesProducer.scala b/gateway/src/main/scala/filodb/timeseries/TestTimeseriesProducer.scala index ddcfba0c7d..32ce1f7a1c 100644 --- a/gateway/src/main/scala/filodb/timeseries/TestTimeseriesProducer.scala +++ b/gateway/src/main/scala/filodb/timeseries/TestTimeseriesProducer.scala @@ -3,67 +3,33 @@ package filodb.timeseries import java.net.URLEncoder import java.nio.charset.StandardCharsets -import scala.concurrent.{Await, Future} +import scala.concurrent.Future import scala.concurrent.duration._ import scala.util.Random -import com.typesafe.config.{Config, ConfigFactory} +import com.typesafe.config.Config import com.typesafe.scalalogging.StrictLogging import monix.reactive.Observable -import org.rogach.scallop._ import filodb.coordinator.{GlobalConfig, ShardMapper} -import filodb.core.metadata.Dataset +import filodb.core.metadata.{Column, Dataset} import filodb.gateway.GatewayServer -import filodb.gateway.conversion.PrometheusInputRecord +import filodb.gateway.conversion.{InputRecord, MetricTagInputRecord, PrometheusInputRecord} +import filodb.memory.format.{vectors => bv, ZeroCopyUTF8String => ZCUTF8} import filodb.prometheus.FormatConversion -sealed trait DataOrCommand -final case class DataSample(tags: Map[String, String], - metric: String, - timestamp: Long, - value: Double) extends DataOrCommand -case object FlushCommand extends DataOrCommand - /** - * Simple driver to produce time series data into local kafka similar. Data format is similar to - * prometheus metric sample. - * This is for development testing purposes only. TODO: Later evolve this to accept prometheus formats. - * - * Run as `java -cp classpath filodb.timeseries.TestTimeseriesProducer --help` - * + * Utilities to produce time series data into local Kafka for development testing. + * Please see GatewayServer for the app to run, or README for docs. */ object TestTimeseriesProducer extends StrictLogging { val dataset = FormatConversion.dataset - class ProducerOptions(args: Seq[String]) extends ScallopConf(args) { - val samplesPerSeries = opt[Int](short = 'n', default = Some(100), - descr="# of samples per time series") - val startMinutesAgo = opt[Int](short='t') - val numSeries = opt[Int](short='p', default = Some(20), descr="# of total time series") - val sourceConfigPath = opt[String](required = true, short = 'c', - descr="Path to source conf file eg conf/timeseries-dev-source.conf") - verify() - } - val oneBitMask = 0x1 val twoBitMask = 0x3 val rand = Random // start from a random day in the last 5 years - def main(args: Array[String]): Unit = { - val conf = new ProducerOptions(args) - val sourceConfig = ConfigFactory.parseFile(new java.io.File(conf.sourceConfigPath())) - val numSamples = conf.samplesPerSeries() * conf.numSeries() - val numTimeSeries = conf.numSeries() - // to get default start time, look at numSamples and calculate a startTime that ends generation at current time - val startMinutesAgo = conf.startMinutesAgo.toOption - .getOrElse((numSamples.toDouble / numTimeSeries / 6).ceil.toInt ) // at 6 samples per minute - - Await.result(produceMetrics(sourceConfig, numSamples, numTimeSeries, startMinutesAgo), 1.hour) - sys.exit(0) - } - import scala.concurrent.ExecutionContext.Implicits.global /** @@ -89,36 +55,36 @@ object TestTimeseriesProducer extends StrictLogging { s"from about ${(System.currentTimeMillis() - startTime) / 1000 / 60} minutes ago") producingFut.map { _ => - // Give enough time for the last containers to be sent off successfully to Kafka - Thread sleep 2000 - logger.info(s"Finished producing $numSamples messages into topic $topicName with timestamps " + - s"from about ${(System.currentTimeMillis() - startTime) / 1000 / 60} minutes ago at $startTime") - val startQuery = startTime / 1000 - val endQuery = startQuery + 300 - val query = - s"""./filo-cli '-Dakka.remote.netty.tcp.hostname=127.0.0.1' --host 127.0.0.1 --dataset prometheus """ + - s"""--promql 'heap_usage{dc="DC0",_ns="App-0"}' --start $startQuery --end $endQuery --limit 15""" - logger.info(s"Periodic Samples CLI Query : \n$query") - - val periodQuery = URLEncoder.encode("""heap_usage{dc="DC0",_ns="App-0"}""", StandardCharsets.UTF_8.toString) - val periodicSamplesUrl = s"http://localhost:8080/promql/prometheus/api/v1/query_range?" + - s"query=$periodQuery&start=$startQuery&end=$endQuery&step=15" - logger.info(s"Periodic Samples query URL: \n$periodicSamplesUrl") - - val rawQuery = URLEncoder.encode("""heap_usage{dc="DC0",_ns="App-0"}[2m]""", - StandardCharsets.UTF_8.toString) - val rawSamplesUrl = s"http://localhost:8080/promql/prometheus/api/v1/query?query=$rawQuery&time=$endQuery" - logger.info(s"Raw Samples query URL: \n$rawSamplesUrl") - - val downsampleSumQuery = URLEncoder.encode("""heap_usage{dc="DC0",_ns="App-0",__col__="sum"}[2m]""", - StandardCharsets.UTF_8.toString) - val downsampledSamplesUrl = s"http://localhost:8080/promql/prometheus_ds_1m/api/v1/query?" + - s"query=$downsampleSumQuery&time=$endQuery" - logger.info(s"Downsampled Samples query URL: \n$downsampledSamplesUrl") - + logQueryHelp(numSamples, numTimeSeries, startTime) } } + + def logQueryHelp(numSamples: Int, numTimeSeries: Int, startTime: Long): Unit = { + val samplesDuration = (numSamples.toDouble / numTimeSeries / 6).ceil.toInt * 60L * 1000L + + logger.info(s"Finished producing $numSamples records for ${samplesDuration / 1000} seconds") + val startQuery = startTime / 1000 + val endQuery = startQuery + 300 + val query = + s"""./filo-cli '-Dakka.remote.netty.tcp.hostname=127.0.0.1' --host 127.0.0.1 --dataset prometheus """ + + s"""--promql 'heap_usage{dc="DC0",_ns="App-0"}' --start $startQuery --end $endQuery --limit 15""" + logger.info(s"Periodic Samples CLI Query : \n$query") + + val q = URLEncoder.encode("""heap_usage{dc="DC0",_ns="App-0"}[2m]""", StandardCharsets.UTF_8.toString) + val periodicSamplesUrl = s"http://localhost:8080/promql/prometheus/api/v1/query_range?" + + s"query=$q&start=$startQuery&end=$endQuery&step=15" + logger.info(s"Periodic Samples query URL: \n$periodicSamplesUrl") + + val q2 = URLEncoder.encode("""heap_usage{dc="DC0",_ns="App-0",__col__="sum"}[2m]""", + StandardCharsets.UTF_8.toString) + val rawSamplesUrl = s"http://localhost:8080/promql/prometheus/api/v1/query?query=$q2&time=$endQuery" + logger.info(s"Raw Samples query URL: \n$rawSamplesUrl") + val downsampledSamplesUrl = s"http://localhost:8080/promql/prometheus_ds_1m/api/v1/query?" + + s"query=$q2&time=$endQuery" + logger.info(s"Downsampled Samples query URL: \n$downsampledSamplesUrl") + } + def metricsToContainerStream(startTime: Long, numShards: Int, numTimeSeries: Int, @@ -129,10 +95,9 @@ object TestTimeseriesProducer extends StrictLogging { val (shardQueues, containerStream) = GatewayServer.shardingPipeline(GlobalConfig.systemConfig, numShards, dataset) val producingFut = Future { - timeSeriesData(startTime, numShards, numTimeSeries) + timeSeriesData(startTime, numTimeSeries) .take(numSamples) - .foreach { sample => - val rec = PrometheusInputRecord(sample.tags, sample.metric, dataset, sample.timestamp, sample.value) + .foreach { rec => val shard = shardMapper.ingestionShard(rec.shardKeyHash, rec.partitionKeyHash, spread) while (!shardQueues(shard).offer(rec)) { Thread sleep 50 } } @@ -141,14 +106,13 @@ object TestTimeseriesProducer extends StrictLogging { } /** - * Generate time series data. + * Generate Prometheus-schema time series data. * * @param startTime Start time stamp - * @param numShards the number of shards or Kafka partitions * @param numTimeSeries number of instances or time series * @return stream of a 2-tuple (kafkaParitionId , sampleData) */ - def timeSeriesData(startTime: Long, numShards: Int, numTimeSeries: Int = 16): Stream[DataSample] = { + def timeSeriesData(startTime: Long, numTimeSeries: Int = 16): Stream[InputRecord] = { // TODO For now, generating a (sinusoidal + gaussian) time series. Other generators more // closer to real world data can be added later. Stream.from(0).map { n => @@ -165,7 +129,57 @@ object TestTimeseriesProducer extends StrictLogging { "partition" -> s"partition-$partition", "host" -> s"H$host", "instance" -> s"Instance-$instance") - DataSample(tags, "heap_usage", timestamp, value) + + PrometheusInputRecord(tags, "heap_usage", dataset, timestamp, value) + } + } + + import ZCUTF8._ + import Column.ColumnType._ + + val dcUTF8 = "dc".utf8 + val nsUTF8 = "_ns".utf8 + val partUTF8 = "partition".utf8 + val hostUTF8 = "host".utf8 + val instUTF8 = "instance".utf8 + + /** + * Generate a stream of random Histogram data, with the metric name "http_request_latency" + * Schema: (timestamp:ts, sum:long, count:long, h:hist) for data, plus (metric:string, tags:map) + * The dataset must match the above schema + */ + def genHistogramData(startTime: Long, dataset: Dataset, numTimeSeries: Int = 16): Stream[InputRecord] = { + require(dataset.dataColumns.map(_.columnType) == Seq(TimestampColumn, LongColumn, LongColumn, HistogramColumn)) + val numBuckets = 10 + + val histBucketScheme = bv.GeometricBuckets(2.0, 3.0, numBuckets) + val buckets = new Array[Long](numBuckets) + def updateBuckets(bucketNo: Int): Unit = { + for { b <- bucketNo until numBuckets } { + buckets(b) += 1 + } + } + + Stream.from(0).map { n => + val instance = n % numTimeSeries + val dc = instance & oneBitMask + val partition = (instance >> 1) & twoBitMask + val app = (instance >> 3) & twoBitMask + val host = (instance >> 4) & twoBitMask + val timestamp = startTime + (n.toLong / numTimeSeries) * 10000 // generate 1 sample every 10s for each instance + + updateBuckets(n % numBuckets) + val hist = bv.LongHistogram(histBucketScheme, buckets.map(x => x)) + val count = util.Random.nextInt(100).toLong + val sum = buckets.sum + + val tags = Map(dcUTF8 -> s"DC$dc".utf8, + nsUTF8 -> s"App-$app".utf8, + partUTF8 -> s"partition-$partition".utf8, + hostUTF8 -> s"H$host".utf8, + instUTF8 -> s"Instance-$instance".utf8) + + new MetricTagInputRecord(Seq(timestamp, sum, count, hist), "http_request_latency", tags, dataset) } } } diff --git a/http/src/test/scala/filodb/http/ClusterApiRouteSpec.scala b/http/src/test/scala/filodb/http/ClusterApiRouteSpec.scala index f0e054b0e7..31d0efc46d 100644 --- a/http/src/test/scala/filodb/http/ClusterApiRouteSpec.scala +++ b/http/src/test/scala/filodb/http/ClusterApiRouteSpec.scala @@ -1,7 +1,5 @@ package filodb.http -import scala.concurrent.duration._ - import akka.actor.ActorSystem import akka.http.scaladsl.model.headers.RawHeader import akka.http.scaladsl.model.{StatusCodes, ContentTypes} @@ -83,21 +81,9 @@ class ClusterApiRouteSpec extends FunSpec with ScalatestRouteTest with AsyncTest it("should return shard status after dataset is setup") { setupDataset() - // Repeatedly query cluster status until we know it is OK - var statuses: Seq[ShardStatus] = Nil - do { - probe.send(clusterProxy, NodeClusterActor.GetShardMap(dataset6.ref)) - Thread sleep 500 - statuses = probe.expectMsgPF(3.seconds) { - case CurrentShardSnapshot(_, mapper) => mapper.statuses - } - println(s"Current statuses = $statuses") - info(s"Current statuses = $statuses") - if (statuses.exists(_ == ShardStatusError)) { - info(s"ERROR in status, breaking") - throw new RuntimeException(s"Got error in statuses $statuses") - } - } while (statuses.take(2) != Seq(ShardStatusActive, ShardStatusActive)) + + // Give the coordinator nodes some time to get started + Thread sleep 1000 Get(s"/api/v1/cluster/${dataset6.ref}/status") ~> clusterRoute ~> check { handled shouldBe true @@ -106,7 +92,8 @@ class ClusterApiRouteSpec extends FunSpec with ScalatestRouteTest with AsyncTest val resp = responseAs[HttpList[HttpShardState]] resp.status shouldEqual "success" resp.data should have length 4 - resp.data.map(_.status).filter(_ contains "Active") should have length 2 // Two active nodes + // Exact status of assigned nodes doesn't matter much. This is an HTTP route test, not a sharding test + resp.data.map(_.status).filter(_ contains "Unassigned") should have length 2 // Two unassigned nodes } } diff --git a/jmh/src/main/scala/filodb.jmh/PartKeyIndexBenchmark.scala b/jmh/src/main/scala/filodb.jmh/PartKeyIndexBenchmark.scala index e416ce8983..f52faf427f 100644 --- a/jmh/src/main/scala/filodb.jmh/PartKeyIndexBenchmark.scala +++ b/jmh/src/main/scala/filodb.jmh/PartKeyIndexBenchmark.scala @@ -24,17 +24,9 @@ class PartKeyIndexBenchmark { val dataset = FormatConversion.dataset val partKeyIndex = new PartKeyLuceneIndex(dataset, 0, TestData.storeConf) val numSeries = 1000000 - val partKeyData = TestTimeseriesProducer.timeSeriesData(0, 1, numSeries) take numSeries + val partKeyData = TestTimeseriesProducer.timeSeriesData(0, numSeries) take numSeries val partKeyBuilder = new RecordBuilder(MemFactory.onHeapFactory, dataset.partKeySchema) - partKeyData.foreach { s => - partKeyBuilder.startNewRecord() - partKeyBuilder.startMap() - s.tags.foreach { case (k, v) => - partKeyBuilder.addMapKeyValue(k.getBytes(), v.getBytes()) - } - partKeyBuilder.endMap() - partKeyBuilder.endRecord() - } + partKeyData.foreach(_.addToBuilder(partKeyBuilder)) var partId = 1 val now = System.currentTimeMillis() diff --git a/jmh/src/main/scala/filodb.jmh/QueryAndIngestBenchmark.scala b/jmh/src/main/scala/filodb.jmh/QueryAndIngestBenchmark.scala index 9d7111cb40..8a0df92511 100644 --- a/jmh/src/main/scala/filodb.jmh/QueryAndIngestBenchmark.scala +++ b/jmh/src/main/scala/filodb.jmh/QueryAndIngestBenchmark.scala @@ -100,12 +100,11 @@ class QueryAndIngestBenchmark extends StrictLogging { val shards = (0 until numShards).map { s => memstore.getShardE(dataset.ref, s) } private def ingestSamples(noSamples: Int): Future[Unit] = Future { - TestTimeseriesProducer.timeSeriesData(startTime, numShards, numSeries) + TestTimeseriesProducer.timeSeriesData(startTime, numSeries) .take(noSamples * numSeries) - .foreach { sample => - val rec = PrometheusInputRecord(sample.tags, sample.metric, dataset, sample.timestamp, sample.value) + .foreach { rec => // we shouldn't ingest samples for same timestamps repeatedly. This will also result in out-of-order samples. - startTime = Math.max(startTime, sample.timestamp + 10000) + startTime = Math.max(startTime, rec.asInstanceOf[PrometheusInputRecord].timestamp + 10000) val shard = shardMapper.ingestionShard(rec.shardKeyHash, rec.partitionKeyHash, spread) while (!shardQueues(shard).offer(rec)) { Thread sleep 50 } } diff --git a/kafka/src/main/scala/filodb/kafka/TestConsumer.scala b/kafka/src/main/scala/filodb/kafka/TestConsumer.scala index 9daa4a4d49..4e814abf1d 100644 --- a/kafka/src/main/scala/filodb/kafka/TestConsumer.scala +++ b/kafka/src/main/scala/filodb/kafka/TestConsumer.scala @@ -4,33 +4,44 @@ import scala.concurrent.Await import scala.concurrent.duration._ import com.typesafe.config.ConfigFactory +import monix.execution.Scheduler +import filodb.coordinator.{FilodbSettings, IngestionStreamFactory, StoreFactory} import filodb.core.memstore.SomeData -import filodb.core.metadata.Dataset +import filodb.core.store.IngestionConfig /** - * A simple app which uses the KafkaIngestionStream plus a sourceconfig of your choice to test reading - * data from Kafka and test reading from certain offsets. + * A simple app which uses a sourceconfig of your choice to test reading + * data from Kafka (or whatever configured source factory) and test reading from certain offsets. + * It reads dataset definition from MetaStore, so please pass the server.conf with Cassandra/metastore details. * - * To launch: java -Xmx4G -cp /standalone-assembly-0.7.0.jar filodb.kafka.TestConsumer \ - * my-kafka-sourceconfig.conf + * To launch: java -Xmx4G -Dconfig.file=conf/timeseries-filodb-server.conf \ + * -cp /standalone-assembly-0.7.0.jar filodb.kafka.TestConsumer \ + * my-kafka-sourceconfig.conf * It will read 10 records and then quit, printing out the offsets of each record. * Optional: pass in a second arg which is the offset to seek to. */ object TestConsumer extends App { + val settings = new FilodbSettings() + val storeFactory = StoreFactory(settings, Scheduler.io()) + val sourceConfPath = args(0) val offsetOpt = args.drop(1).headOption.map(_.toLong) + val shard = if (args.length > 1) args(1).toInt else 0 val sourceConf = ConfigFactory.parseFile(new java.io.File(sourceConfPath)) //scalastyle:off - println(s"TestConsumer starting with config $sourceConf\nand offset $offsetOpt") + println(s"TestConsumer starting with shard $shard, config $sourceConf\nand offset $offsetOpt") import monix.execution.Scheduler.Implicits.global - // For now, hard code dataset to a Prometheus like dataset - // TODO: allow specification of dataset, then load from the MetaStore - val dataset = Dataset("prometheus", Seq("tags:map"), Seq("timestamp:long", "value:double")) - val stream = new KafkaIngestionStream(sourceConf, dataset, 0, offsetOpt) + val ingestConf = IngestionConfig(sourceConf, classOf[KafkaIngestionStreamFactory].getClass.getName).get + val dataset = Await.result(storeFactory.metaStore.getDataset(ingestConf.ref), 30.seconds) + + val ctor = Class.forName(ingestConf.streamFactoryClass).getConstructors.head + val streamFactory = ctor.newInstance().asInstanceOf[IngestionStreamFactory] + + val stream = streamFactory.create(sourceConf, dataset, shard, offsetOpt) val fut = stream.get.take(10) .foreach { case SomeData(container, offset) => println(s"\n----- Offset $offset -----") diff --git a/project/FiloBuild.scala b/project/FiloBuild.scala index b6a1607cdf..8ff4f496cb 100644 --- a/project/FiloBuild.scala +++ b/project/FiloBuild.scala @@ -152,7 +152,7 @@ object FiloBuild extends Build { .settings(libraryDependencies ++= gatewayDeps) .settings(gatewayAssemblySettings: _*) .dependsOn(coordinator % "compile->compile; test->test", - prometheus) + prometheus, cassandra) // Zookeeper pulls in slf4j-log4j12 which we DON'T want val excludeZK = ExclusionRule(organization = "org.apache.zookeeper") From 4692efc164913d7e21c0769496172382f3a8b530 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Mon, 1 Apr 2019 22:12:14 -0700 Subject: [PATCH 36/41] feat(core): Optimize index recovery for long term retention (#296) Earlier approach of creating every TSP was ok for 3 day retention given most partitions are ingesting, but does not scale for long term retention where majority of partitions are not ingesting. This commit reduces number of entries in partitions and partSet data structures, thereby reducing memory requirements especially for long term datasets. We pay the cost of a temporary map during recovery that holds all partKeys on heap, but is discarded before ingestion starts. Also optimize the Lucene recovery by scanning through records in reverse order of buckets. This avoids the expensive Lucene upserts - adds become sufficient. --- .../memstore/TimeSeriesShard.scala | 69 +++++++++++++------ .../memstore/TimeSeriesMemStoreSpec.scala | 29 +++++--- .../scala/filodb/query/exec/ExecPlan.scala | 9 ++- 3 files changed, 76 insertions(+), 31 deletions(-) diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala index 75777a42ee..e32073b1b1 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala @@ -17,6 +17,7 @@ import monix.eval.Task import monix.execution.{Scheduler, UncaughtExceptionReporter} import monix.execution.atomic.AtomicBoolean import monix.reactive.Observable +import org.apache.lucene.util.BytesRef import org.jctools.maps.NonBlockingHashMapLong import scalaxy.loops._ @@ -429,16 +430,23 @@ class TimeSeriesShard(val dataset: Dataset, .withTag("dataset", dataset.name) .withTag("shard", shardNum).start() + /* We need this map to track partKey->partId because lucene index cannot be looked up + using partKey efficiently, and more importantly, it is eventually consistent. + The map and contents will be garbage collected after we are done with recovery */ + val partIdMap = debox.Map.empty[BytesRef, Int] + val earliestTimeBucket = Math.max(0, currentIndexTimeBucket - numTimeBucketsToRetain) logger.info(s"Recovering timebuckets $earliestTimeBucket until $currentIndexTimeBucket " + s"for dataset=${dataset.ref} shard=$shardNum ") - val timeBuckets = for { tb <- earliestTimeBucket until currentIndexTimeBucket } yield { + // go through the buckets in reverse order to first one wins and we need not rewrite + // entries in lucene + val timeBuckets = for { tb <- currentIndexTimeBucket until earliestTimeBucket by -1 } yield { colStore.getPartKeyTimeBucket(dataset, shardNum, tb).map { b => new IndexData(tb, b.segmentId, RecordContainer(b.segment.array())) } } val fut = Observable.flatten(timeBuckets: _*) - .foreach(tb => extractTimeBucket(tb))(ingestSched) + .foreach(tb => extractTimeBucket(tb, partIdMap))(ingestSched) .map(_ => completeIndexRecovery()) fut.onComplete(_ => tracer.finish()) fut @@ -450,7 +458,8 @@ class TimeSeriesShard(val dataset: Dataset, logger.info(s"Bootstrapped index for dataset=${dataset.ref} shard=$shardNum") } - private[memstore] def extractTimeBucket(segment: IndexData): Unit = { + // scalastyle:off method.length + private[memstore] def extractTimeBucket(segment: IndexData, partIdMap: debox.Map[BytesRef, Int]): Unit = { var numRecordsProcessed = 0 segment.records.iterate(indexTimeBucketSchema).foreach { row => // read binary record and extract the indexable data fields @@ -459,44 +468,63 @@ class TimeSeriesShard(val dataset: Dataset, val partKeyBaseOnHeap = row.getBlobBase(2).asInstanceOf[Array[Byte]] val partKeyOffset = row.getBlobOffset(2) val partKeyNumBytes = row.getBlobNumBytes(2) + val partKeyBytesRef = new BytesRef(partKeyBaseOnHeap, + PartKeyLuceneIndex.unsafeOffsetToBytesRefOffset(partKeyOffset), + partKeyNumBytes) - // look up partId in partSet if it already exists before assigning new partId. + // look up partKey in partIdMap if it already exists before assigning new partId. // We cant look it up in lucene because we havent flushed index yet - val partId = partSet.getWithPartKeyBR(partKeyBaseOnHeap, partKeyOffset) match { - case None => val group = partKeyGroup(dataset.partKeySchema, partKeyBaseOnHeap, partKeyOffset, numGroups) + if (partIdMap.get(partKeyBytesRef).isEmpty) { + val partId = if (endTime == Long.MaxValue) { + // this is an actively ingesting partition + val group = partKeyGroup(dataset.partKeySchema, partKeyBaseOnHeap, partKeyOffset, numGroups) val part = createNewPartition(partKeyBaseOnHeap, partKeyOffset, group, CREATE_NEW_PARTID, 4) // In theory, we should not get an OutOfMemPartition here since - // it should have occurred before node failed too, and with data sropped, + // it should have occurred before node failed too, and with data stopped, // index would not be updated. But if for some reason we see it, drop data if (part == OutOfMemPartition) { logger.error("Could not accommodate partKey while recovering index. " + "WriteBuffer size may not be configured correctly") - -1 + None } else { val stamp = partSetLock.writeLock() try { partSet.add(part) // createNewPartition doesn't add part to partSet + Some(part.partID) } finally { partSetLock.unlockWrite(stamp) } - part.partID } - case Some(p) => p.partID - } - if (partId != -1) { - // upsert into lucene since there can be multiple records for one partKey, and most recent wins. - partKeyIndex.upsertPartKey(partKeyBaseOnHeap, partId, startTime, endTime, - PartKeyLuceneIndex.unsafeOffsetToBytesRefOffset(partKeyOffset))(partKeyNumBytes) - timeBucketBitmaps.get(segment.timeBucket).set(partId) - activelyIngesting.synchronized { - if (endTime == Long.MaxValue) activelyIngesting.set(partId) else activelyIngesting.clear(partId) + } else { + // partition assign a new partId to non-ingesting partition, + // but no need to create a new TSPartition heap object + val id = nextPartitionID + incrementPartitionID() + Some(id) } + + // add newly assigned partId to lucene index + partId.foreach { partId => + partIdMap(partKeyBytesRef) = partId + partKeyIndex.addPartKey(partKeyBaseOnHeap, partId, startTime, endTime, + PartKeyLuceneIndex.unsafeOffsetToBytesRefOffset(partKeyOffset))(partKeyNumBytes) + timeBucketBitmaps.get(segment.timeBucket).set(partId) + activelyIngesting.synchronized { + if (endTime == Long.MaxValue) activelyIngesting.set(partId) + else activelyIngesting.clear(partId) + } + } + } else { + // partId has already been assigned for this partKey because we previously processed a later record in time. + // Time buckets are processed in reverse order, and given last one wins and is used for index, + // we skip this record and move on. } numRecordsProcessed += 1 } shardStats.indexRecoveryNumRecordsProcessed.increment(numRecordsProcessed) - logger.info(s"Recovered partition keys from timebucket for dataset=${dataset.ref} shard=$shardNum" + - s" timebucket=${segment.timeBucket} segment=${segment.segment} numRecordsProcessed=$numRecordsProcessed") + logger.info(s"Recovered partKeys for dataset=${dataset.ref} shard=$shardNum" + + s" timebucket=${segment.timeBucket} segment=${segment.segment} numRecordsInBucket=$numRecordsProcessed" + + s" numPartsInIndex=${partIdMap.size} numIngestingParts=${partitions.size}") } def indexNames: Iterator[String] = partKeyIndex.indexNames @@ -902,7 +930,6 @@ class TimeSeriesShard(val dataset: Dataset, activelyIngesting.clear(p.partID) } else if (partFlushChunks.nonEmpty && !activelyIngesting.get(p.partID)) { // Partition started re-ingesting. - // TODO: we can do better than this for intermittent time series. Address later. updatePartEndTimeInIndex(p, Long.MaxValue) timeBucketBitmaps.get(timeBucket).set(p.partID) activelyIngesting.set(p.partID) diff --git a/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala b/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala index c7c273bfd0..a59f323bf5 100644 --- a/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala +++ b/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala @@ -6,6 +6,7 @@ import scala.concurrent.duration._ import com.typesafe.config.ConfigFactory import monix.execution.ExecutionModel.BatchedExecution import monix.reactive.Observable +import org.apache.lucene.util.BytesRef import org.scalatest.{BeforeAndAfter, FunSpec, Matchers} import org.scalatest.concurrent.ScalaFutures import org.scalatest.time.{Millis, Seconds, Span} @@ -85,7 +86,7 @@ class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter w memStore.commitIndexForTesting(dataset1.ref) val split = memStore.getScanSplits(dataset1.ref, 1).head val agg1 = memStore.scanRows(dataset1, Seq(1), FilteredPartitionScan(split)).map(_.getDouble(0)).sum - agg1 shouldEqual ((1 to 20).map(_.toDouble).sum) + agg1 shouldEqual (1 to 20).map(_.toDouble).sum } it("should ingest map/tags column as partition key and aggregate") { @@ -272,6 +273,9 @@ class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter w tsShard.timeBucketBitmaps.keySet.asScala.toSeq.sorted shouldEqual 19.to(25) // 6 buckets retained + one for current } + /** + * Tries to write partKeys into time bucket record container and extracts them back into the shard + */ def indexRecoveryTest(dataset: Dataset, partKeys: Seq[Long]): Unit = { memStore.metastore.writeHighestIndexTimeBucket(dataset.ref, 0, 0) memStore.setup(dataset, 0, @@ -282,24 +286,33 @@ class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter w partKeys.zipWithIndex.foreach { case (off, i) => timeBucketRb.startNewRecord() - timeBucketRb.addLong(i + 10) - timeBucketRb.addLong(i + 20) + timeBucketRb.addLong(i + 10) // startTime + timeBucketRb.addLong(if (i%2 == 0) Long.MaxValue else i + 20) // endTime val numBytes = BinaryRegionLarge.numBytes(UnsafeUtils.ZeroPointer, off) - timeBucketRb.addBlob(UnsafeUtils.ZeroPointer, off, numBytes + 4) + timeBucketRb.addBlob(UnsafeUtils.ZeroPointer, off, numBytes + 4) // partKey timeBucketRb.endRecord(false) } tsShard.initTimeBuckets() + val partIdMap = debox.Map.empty[BytesRef, Int] + timeBucketRb.optimalContainerBytes(true).foreach { bytes => - tsShard.extractTimeBucket(new IndexData(1, 0, RecordContainer(bytes))) + tsShard.extractTimeBucket(new IndexData(1, 0, RecordContainer(bytes)), partIdMap) } tsShard.commitPartKeyIndexBlocking() + partIdMap.size shouldEqual partKeys.size partKeys.zipWithIndex.foreach { case (off, i) => val readPartKey = tsShard.partKeyIndex.partKeyFromPartId(i).get val expectedPartKey = dataset1.partKeySchema.asByteArray(UnsafeUtils.ZeroPointer, off) - readPartKey.bytes.drop(readPartKey.offset).take(readPartKey.length) shouldEqual expectedPartKey - tsShard.partitions.get(i).partKeyBytes shouldEqual expectedPartKey - tsShard.partSet.getWithPartKeyBR(UnsafeUtils.ZeroPointer, off).get.partID shouldEqual i + readPartKey.bytes.slice(readPartKey.offset, readPartKey.offset + readPartKey.length) shouldEqual expectedPartKey + if (i%2 == 0) { + tsShard.partSet.getWithPartKeyBR(UnsafeUtils.ZeroPointer, off).get.partID shouldEqual i + tsShard.partitions.containsKey(i) shouldEqual true // since partition is ingesting + } + else { + tsShard.partSet.getWithPartKeyBR(UnsafeUtils.ZeroPointer, off) shouldEqual None + tsShard.partitions.containsKey(i) shouldEqual false // since partition is not ingesting + } } } diff --git a/query/src/main/scala/filodb/query/exec/ExecPlan.scala b/query/src/main/scala/filodb/query/exec/ExecPlan.scala index 0d4d7ec6b0..1d0eb59ac3 100644 --- a/query/src/main/scala/filodb/query/exec/ExecPlan.scala +++ b/query/src/main/scala/filodb/query/exec/ExecPlan.scala @@ -102,16 +102,17 @@ trait ExecPlan extends QueryCommand { // Lucene index lookup, and On-Demand Paging orchestration work could suck up nontrivial time and // we don't want these to happen in a single thread. Task { - qLogger.debug(s"queryId: ${id} Started ExecPlan ${getClass.getSimpleName} with $args") + qLogger.debug(s"queryId: ${id} Setting up ExecPlan ${getClass.getSimpleName} with $args") val res = doExecute(source, dataset, queryConfig) val schema = schemaOfDoExecute(dataset) val finalRes = rangeVectorTransformers.foldLeft((res, schema)) { (acc, transf) => - qLogger.debug(s"queryId: ${id} Started Transformer ${transf.getClass.getSimpleName} with ${transf.args}") + qLogger.debug(s"queryId: ${id} Setting up Transformer ${transf.getClass.getSimpleName} with ${transf.args}") (transf.apply(acc._1, queryConfig, limit, acc._2), transf.schema(dataset, acc._2)) } val recSchema = SerializableRangeVector.toSchema(finalRes._2.columns, finalRes._2.brSchemas) val builder = SerializableRangeVector.toBuilder(recSchema) var numResultSamples = 0 // BEWARE - do not modify concurrently!! + qLogger.debug(s"queryId: ${id} Materializing SRVs from iterators if necessary") finalRes._1 .map { case srv: SerializableRangeVector => @@ -261,6 +262,10 @@ abstract class NonLeafExecPlan extends ExecPlan { /** * Sub-class non-leaf nodes should provide their own implementation of how * to compose the sub-query results here. + * + * @param childResponses observable of a pair. First element of pair is the QueryResponse for + * a child ExecPlan, the second element is the index of the child plan. + * There is one response per child plan. */ protected def compose(dataset: Dataset, childResponses: Observable[(QueryResponse, Int)], From 209b6e67e30340c3eff3f0b29ef37add6fabb973 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Wed, 3 Apr 2019 11:11:19 -0700 Subject: [PATCH 37/41] bug(coord): Remove publish snapshot for RecoveryInProgress (#305) * Don't send snapshot on RecoveryInProgress events since it results in too many events on the cluster singleton causing HTTP timeouts. * Save context.parent on IngestionActor startup and use that instead of context.parent from future thread. --- .../src/main/scala/filodb.coordinator/IngestionActor.scala | 7 ++++--- .../src/main/scala/filodb.coordinator/ShardManager.scala | 3 ++- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/IngestionActor.scala b/coordinator/src/main/scala/filodb.coordinator/IngestionActor.scala index 50cc0e09d2..228225249c 100644 --- a/coordinator/src/main/scala/filodb.coordinator/IngestionActor.scala +++ b/coordinator/src/main/scala/filodb.coordinator/IngestionActor.scala @@ -60,6 +60,7 @@ private[filodb] final class IngestionActor(dataset: Dataset, final val streamSubscriptions = new HashMap[Int, CancelableFuture[Unit]] final val streams = new HashMap[Int, IngestionStream] + final val nodeCoord = context.parent // Params for creating the default memStore flush scheduler private final val numGroups = storeConfig.groupsPerShard @@ -164,7 +165,7 @@ private[filodb] final class IngestionActor(dataset: Dataset, create(shard, offset) map { ingestionStream => val stream = ingestionStream.get logger.info(s"Starting normal/active ingestion for dataset=${dataset.ref} shard=$shard at offset $offset") - statusActor ! IngestionStarted(dataset.ref, shard, context.parent) + statusActor ! IngestionStarted(dataset.ref, shard, nodeCoord) streamSubscriptions(shard) = memStore.ingestStream(dataset.ref, shard, @@ -209,7 +210,7 @@ private[filodb] final class IngestionActor(dataset: Dataset, .withTag("shard", shard.toString) .withTag("dataset", dataset.ref.toString).start() val stream = ingestionStream.get - statusActor ! RecoveryInProgress(dataset.ref, shard, context.parent, 0) + statusActor ! RecoveryInProgress(dataset.ref, shard, nodeCoord, 0) val shardInstance = memStore.asInstanceOf[TimeSeriesMemStore].getShardE(dataset.ref, shard) val fut = memStore.recoverStream(dataset.ref, shard, stream, checkpoints, interval) @@ -218,7 +219,7 @@ private[filodb] final class IngestionActor(dataset: Dataset, else (off - startOffset) * 100 / (endOffset - startOffset) logger.info(s"Recovery of dataset=${dataset.ref} shard=$shard at " + s"$progressPct % - offset $off (target $endOffset)") - statusActor ! RecoveryInProgress(dataset.ref, shard, context.parent, progressPct.toInt) + statusActor ! RecoveryInProgress(dataset.ref, shard, nodeCoord, progressPct.toInt) off } .until(_ >= endOffset) // TODO: move this code to TimeSeriesShard itself. Shard should control the thread diff --git a/coordinator/src/main/scala/filodb.coordinator/ShardManager.scala b/coordinator/src/main/scala/filodb.coordinator/ShardManager.scala index b846929831..6ea9afa123 100644 --- a/coordinator/src/main/scala/filodb.coordinator/ShardManager.scala +++ b/coordinator/src/main/scala/filodb.coordinator/ShardManager.scala @@ -449,7 +449,8 @@ private[coordinator] final class ShardManager(settings: FilodbSettings, // Above condition ensures that we respond to shard events only from the node shard is currently assigned to. // Needed to avoid race conditions where IngestionStopped for an old assignment comes after shard is reassigned. updateFromShardEvent(event) - publishSnapshot(event.ref) + // RecoveryInProgress status results in too many messages that really do not need a publish + if (!event.isInstanceOf[RecoveryInProgress]) publishSnapshot(event.ref) // reassign shard if IngestionError. Exclude previous node since it had error shards. event match { case _: IngestionError => From f6bf77b4e45455b615f6c002c47241d4a1ac2df7 Mon Sep 17 00:00:00 2001 From: Jackson Jeyapaul Date: Wed, 3 Apr 2019 15:02:17 -0700 Subject: [PATCH 38/41] misc(core): Add zipkin configs to filodb-defaults (#306) --- core/src/main/resources/filodb-defaults.conf | 7 ++++++- core/src/main/scala/filodb.core/zipkin/Zipkin.scala | 4 ++-- 2 files changed, 8 insertions(+), 3 deletions(-) diff --git a/core/src/main/resources/filodb-defaults.conf b/core/src/main/resources/filodb-defaults.conf index 33f753c5c4..3b24108270 100644 --- a/core/src/main/resources/filodb-defaults.conf +++ b/core/src/main/resources/filodb-defaults.conf @@ -327,4 +327,9 @@ custom-downing { down-if-in-minority = true shutdown-actor-system-on-resolution = true } -} \ No newline at end of file +} + +kamon.zipkin { + max.requests = 128 + message.max.bytes = 262144 +} diff --git a/core/src/main/scala/filodb.core/zipkin/Zipkin.scala b/core/src/main/scala/filodb.core/zipkin/Zipkin.scala index 3ccc721a1d..b68bf2d85b 100644 --- a/core/src/main/scala/filodb.core/zipkin/Zipkin.scala +++ b/core/src/main/scala/filodb.core/zipkin/Zipkin.scala @@ -139,8 +139,8 @@ class ZipkinReporter extends SpanReporter { OkHttpSender.newBuilder() .encoding(Encoding.JSON) .endpoint(url) - .maxRequests(if (maxRequests > 0) maxRequests else 128) - .messageMaxBytes(if (messageMaxBytes > 0) messageMaxBytes else 1024 * 256) + .maxRequests(maxRequests) + .messageMaxBytes(messageMaxBytes) .build() ) } From 5c0b7e5cd8898b2cf51e5432ecad02cd7aa85b4d Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Thu, 4 Apr 2019 11:06:50 -0700 Subject: [PATCH 39/41] fix(cli): Make --limit actually behave like a user specified output limit (#307) * fix(cli): Make --limit actually behave like a user specified output limit * Change the docs for CLI --- README.md | 3 ++- cli/src/main/scala/filodb.cli/CliMain.scala | 12 ++++++------ 2 files changed, 8 insertions(+), 7 deletions(-) diff --git a/README.md b/README.md index f74c72f070..4f8b95c488 100644 --- a/README.md +++ b/README.md @@ -590,7 +590,8 @@ The `filo-cli` accepts arguments and options as key-value pairs, specified like | minutes | A shortcut to set the start at N minutes ago, and the stop at current time. Should specify a step also. | | chunks | Either "memory" or "buffers" to select either all the in-memory chunks or the write buffers only. Should specify a step also. | | database | Specifies the "database" the dataset should operate in. For Cassandra, this is the keyspace. If not specified, uses config value. | -| limit | The maximum number of samples per time series | +| limit | Limits the number of time series in the output | +| sampleLimit | Maximum number of output samples in the query result. An exception is thrown if the output returns more results than this. | | shards | (EXPERT) overrides the automatic shard calculation by passing in a comma-separated list of specific shards to query. Very useful to debug sharding issues. | | everyNSeconds | Repeats the query every (argument) seconds | | timeoutSeconds | The number of seconds for the network timeout | diff --git a/cli/src/main/scala/filodb.cli/CliMain.scala b/cli/src/main/scala/filodb.cli/CliMain.scala index 790909cfb4..47b153b2f7 100644 --- a/cli/src/main/scala/filodb.cli/CliMain.scala +++ b/cli/src/main/scala/filodb.cli/CliMain.scala @@ -34,9 +34,9 @@ class Arguments extends FieldArgs { var rowKeys: Seq[String] = Seq("timestamp") var partitionKeys: Seq[String] = Nil var select: Option[Seq[String]] = None - // max # query items (vectors or tuples) returned. Don't make it too high. - var limit: Int = 1000 - var sampleLimit: Int = 200 + // max # of RangeVectors returned. Don't make it too high. + var limit: Int = 200 + var sampleLimit: Int = 1000000 var timeoutSeconds: Int = 60 var outfile: Option[String] = None var delimiter: String = "," @@ -364,7 +364,7 @@ object CliMain extends ArgMain[Arguments] with CsvImportExport with FilodbCluste def executeQuery2(client: LocalClient, dataset: String, plan: LogicalPlan, options: QOptions): Unit = { val ref = DatasetRef(dataset) - val qOpts = QueryCommands.QueryOptions(options.spread, options.limit) + val qOpts = QueryCommands.QueryOptions(options.spread, options.sampleLimit) .copy(queryTimeoutSecs = options.timeout.toSeconds.toInt, shardOverrides = options.shardOverrides) println(s"Sending query command to server for $ref with options $qOpts...") @@ -373,7 +373,7 @@ object CliMain extends ArgMain[Arguments] with CsvImportExport with FilodbCluste case Some(intervalSecs) => val fut = Observable.intervalAtFixedRate(intervalSecs.seconds).foreach { n => client.logicalPlan2Query(ref, plan, qOpts) match { - case QueryResult(_, schema, result) => result.foreach(rv => println(rv.prettyPrint())) + case QueryResult(_, schema, result) => result.take(options.limit).foreach(rv => println(rv.prettyPrint())) case err: QueryError => throw new ClientException(err) } }.recover { @@ -385,7 +385,7 @@ object CliMain extends ArgMain[Arguments] with CsvImportExport with FilodbCluste try { client.logicalPlan2Query(ref, plan, qOpts) match { case QueryResult(_, schema, result) => println(s"Number of Range Vectors: ${result.size}") - result.foreach(rv => println(rv.prettyPrint())) + result.take(options.limit).foreach(rv => println(rv.prettyPrint())) case QueryError(_,ex) => println(s"QueryError: ${ex.getClass.getSimpleName} ${ex.getMessage}") } } catch { From a9087270d3de0970b5855bd23b2323acbf4833d0 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Thu, 4 Apr 2019 15:46:57 -0700 Subject: [PATCH 40/41] fix(memory, query): Fix exceptions in Histogram summing (#308) They were caused by some type misalignment. Now everything in aggregations is a HistogramWithBuckets. --- .../filodb.memory/format/vectors/Histogram.scala | 13 ++----------- .../format/vectors/HistogramVector.scala | 4 ++-- .../filodb/query/exec/AggrOverRangeVectors.scala | 9 ++++++--- .../main/scala/filodb/query/exec/TransientRow.scala | 4 ++-- .../query/exec/rangefn/AggrOverTimeFunctions.scala | 2 +- .../filodb/query/exec/rangefn/RangeFunction.scala | 2 +- .../query/exec/AggrOverRangeVectorsSpec.scala | 12 +++++++++++- 7 files changed, 25 insertions(+), 21 deletions(-) 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 c4cd7e0197..ee8ac39214 100644 --- a/memory/src/main/scala/filodb.memory/format/vectors/Histogram.scala +++ b/memory/src/main/scala/filodb.memory/format/vectors/Histogram.scala @@ -116,16 +116,7 @@ trait Histogram extends Ordered[Histogram] { } object Histogram { - val empty = new Histogram { - final def numBuckets: Int = 0 - final def bucketTop(no: Int): Double = ??? - final def bucketValue(no: Int): Double = ??? - final def serialize(intoBuf: Option[MutableDirectBuffer] = None): MutableDirectBuffer = { - val buf = intoBuf.getOrElse(BinaryHistogram.histBuf) - BinaryHistogram.writeNonIncreasing(HistogramBuckets.emptyBuckets, Array[Long](), buf) - buf - } - } + val empty = MutableHistogram(HistogramBuckets.emptyBuckets, Array.empty) } trait HistogramWithBuckets extends Histogram { @@ -176,7 +167,7 @@ final case class MutableHistogram(buckets: HistogramBuckets, values: Array[Doubl /** * Copies this histogram as a new copy so it can be used for aggregation or mutation. Allocates new storage. */ - final def copy: Histogram = MutableHistogram(buckets, values.clone) + final def copy: MutableHistogram = MutableHistogram(buckets, values.clone) /** * Adds the values from another 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 cc23454556..933d4e8bbb 100644 --- a/memory/src/main/scala/filodb.memory/format/vectors/HistogramVector.scala +++ b/memory/src/main/scala/filodb.memory/format/vectors/HistogramVector.scala @@ -279,7 +279,7 @@ class AppendableHistogramVector(factory: MemFactory, trait HistogramReader extends VectorDataReader { def buckets: HistogramBuckets - def apply(index: Int): Histogram + def apply(index: Int): HistogramWithBuckets def sum(start: Int, end: Int): MutableHistogram } @@ -372,7 +372,7 @@ class RowHistogramReader(histVect: Ptr.U8) extends HistogramReader { def length(addr: BinaryVectorPtr): Int = length // WARNING: histogram returned is shared between calls, do not reuse! - final def apply(index: Int): Histogram = { + final def apply(index: Int): HistogramWithBuckets = { require(length > 0) val histPtr = locate(index) val histLen = histPtr.asU16.getU16 diff --git a/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala b/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala index 29e96190e0..e074600949 100644 --- a/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala +++ b/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala @@ -322,7 +322,8 @@ object SumRowAggregator extends RowAggregator { object HistSumRowAggregator extends RowAggregator { import filodb.memory.format.{vectors => bv} - class HistSumHolder(var timestamp: Long = 0L, var h: bv.Histogram = bv.Histogram.empty) extends AggregateHolder { + class HistSumHolder(var timestamp: Long = 0L, + var h: bv.MutableHistogram = bv.Histogram.empty) extends AggregateHolder { val row = new TransientHistRow() def toRowReader: MutableRowReader = { row.setValues(timestamp, h); row } def resetToZero(): Unit = h = bv.Histogram.empty @@ -333,10 +334,12 @@ object HistSumRowAggregator extends RowAggregator { def map(rvk: RangeVectorKey, item: RowReader, mapInto: MutableRowReader): RowReader = item def reduceAggregate(acc: HistSumHolder, aggRes: RowReader): HistSumHolder = { acc.timestamp = aggRes.getLong(0) + val newHist = aggRes.getHistogram(1) acc.h match { // sum is mutable histogram, copy to be sure it's our own copy - case hist if hist.numBuckets == 0 => acc.h = bv.MutableHistogram(aggRes.getHistogram(1)) - case hist: bv.MutableHistogram => hist.add(aggRes.getHistogram(1).asInstanceOf[bv.HistogramWithBuckets]) + case hist if hist.numBuckets == 0 => acc.h = bv.MutableHistogram(newHist) + case h if newHist.numBuckets > 0 => acc.h.add(newHist.asInstanceOf[bv.HistogramWithBuckets]) + case h => } acc } diff --git a/query/src/main/scala/filodb/query/exec/TransientRow.scala b/query/src/main/scala/filodb/query/exec/TransientRow.scala index adf3bd7da5..0a8aeb9b7b 100644 --- a/query/src/main/scala/filodb/query/exec/TransientRow.scala +++ b/query/src/main/scala/filodb/query/exec/TransientRow.scala @@ -56,8 +56,8 @@ final class TransientRow(var timestamp: Long, var value: Double) extends Mutable } final class TransientHistRow(var timestamp: Long = 0L, - var value: bv.Histogram = bv.Histogram.empty) extends MutableRowReader { - def setValues(ts: Long, hist: bv.Histogram): Unit = { + var value: bv.HistogramWithBuckets = bv.Histogram.empty) extends MutableRowReader { + def setValues(ts: Long, hist: bv.HistogramWithBuckets): Unit = { timestamp = ts value = hist } 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 27750e67a9..2d71c8f0c2 100644 --- a/query/src/main/scala/filodb/query/exec/rangefn/AggrOverTimeFunctions.scala +++ b/query/src/main/scala/filodb/query/exec/rangefn/AggrOverTimeFunctions.scala @@ -166,7 +166,7 @@ class SumOverTimeChunkedFunctionL extends SumOverTimeChunkedFunction() with Chun } } -class SumOverTimeChunkedFunctionH(var h: bv.Histogram = bv.Histogram.empty) +class SumOverTimeChunkedFunctionH(var h: bv.MutableHistogram = bv.Histogram.empty) extends ChunkedRangeFunction[TransientHistRow] { override final def reset(): Unit = { h = bv.Histogram.empty } final def apply(endTimestamp: Long, sampleToEmit: TransientHistRow): Unit = { 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 a767c35ea4..2520f5ff1d 100644 --- a/query/src/main/scala/filodb/query/exec/rangefn/RangeFunction.scala +++ b/query/src/main/scala/filodb/query/exec/rangefn/RangeFunction.scala @@ -315,7 +315,7 @@ extends LastSampleChunkedFunction[TransientRow] { } // LastSample function for Histogram columns -class LastSampleChunkedFunctionH(var value: bv.Histogram = bv.Histogram.empty) +class LastSampleChunkedFunctionH(var value: bv.HistogramWithBuckets = bv.Histogram.empty) extends LastSampleChunkedFunction[TransientHistRow] { override final def reset(): Unit = { timestamp = -1L; value = bv.Histogram.empty } final def apply(endTimestamp: Long, sampleToEmit: TransientHistRow): Unit = { diff --git a/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala b/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala index 65c6ee5cae..f926d00118 100644 --- a/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala +++ b/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala @@ -344,7 +344,9 @@ class AggrOverRangeVectorsSpec extends RawDataWindowingSpec with ScalaFutures { val samples: Array[RangeVector] = Array(rv1, rv2) val agg1 = RowAggregator(AggregationOperator.Sum, Nil, ColumnType.HistogramColumn) - val resultObs = RangeVectorAggregator.mapReduce(agg1, false, Observable.fromIterable(samples), noGrouping) + val resultObs1 = RangeVectorAggregator.mapReduce(agg1, false, Observable.fromIterable(samples), noGrouping) + val resultObs = RangeVectorAggregator.mapReduce(agg1, true, resultObs1, rv=>rv.key) + val result = resultObs.toListL.runAsync.futureValue result.size shouldEqual 1 result(0).key shouldEqual noKey @@ -356,6 +358,14 @@ class AggrOverRangeVectorsSpec extends RawDataWindowingSpec with ScalaFutures { }.toList result(0).rows.map(_.getHistogram(1)).toList shouldEqual sums + + // Test mapReduce of empty histogram sums + val agg2 = RowAggregator(AggregationOperator.Sum, Nil, ColumnType.HistogramColumn) + val emptyObs = RangeVectorAggregator.mapReduce(agg2, false, Observable.empty, noGrouping) + val resultObs2 = RangeVectorAggregator.mapReduce(agg2, true, emptyObs ++ resultObs1, rv=>rv.key) + val result2 = resultObs2.toListL.runAsync.futureValue + result2.size shouldEqual 1 + result2(0).key shouldEqual noKey } @tailrec From 418394127e5bcd48be75f3c17ff60b24d16944fb Mon Sep 17 00:00:00 2001 From: "Brian S. O'Neill" Date: Fri, 5 Apr 2019 13:06:04 -0700 Subject: [PATCH 41/41] bug(coordinator): Fix thread safety issue with sharing of ShardMapper instances. (#311) --- .../scala/filodb.coordinator/NodeCoordinatorActor.scala | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/NodeCoordinatorActor.scala b/coordinator/src/main/scala/filodb.coordinator/NodeCoordinatorActor.scala index d96e20fe65..a69291a6c1 100644 --- a/coordinator/src/main/scala/filodb.coordinator/NodeCoordinatorActor.scala +++ b/coordinator/src/main/scala/filodb.coordinator/NodeCoordinatorActor.scala @@ -1,5 +1,7 @@ package filodb.coordinator +import java.util.concurrent.ConcurrentHashMap + import scala.collection.mutable.HashMap import scala.concurrent.duration._ @@ -57,7 +59,7 @@ private[filodb] final class NodeCoordinatorActor(metaStore: MetaStore, val ingesters = new HashMap[DatasetRef, ActorRef] val queryActors = new HashMap[DatasetRef, ActorRef] var clusterActor: Option[ActorRef] = None - val shardMaps = new HashMap[DatasetRef, ShardMapper] + val shardMaps = new ConcurrentHashMap[DatasetRef, ShardMapper] var statusActor: Option[ActorRef] = None private val statusAckTimeout = config.as[FiniteDuration]("tasks.timeouts.status-ack-timeout") @@ -137,7 +139,7 @@ private[filodb] final class NodeCoordinatorActor(metaStore: MetaStore, ingesters(ref) = ingester logger.info(s"Creating QueryActor for dataset $ref") - val queryRef = context.actorOf(QueryActor.props(memStore, dataset, shardMaps(ref)), s"$Query-$ref") + val queryRef = context.actorOf(QueryActor.props(memStore, dataset, shardMaps.get(ref)), s"$Query-$ref") nca.tell(SubscribeShardUpdates(ref), self) queryActors(ref) = queryRef @@ -190,7 +192,7 @@ private[filodb] final class NodeCoordinatorActor(metaStore: MetaStore, case NodeProtocol.ResetState => reset(sender()) case CurrentShardSnapshot(ds, mapper) => logger.debug(s"Received ShardSnapshot $mapper") - shardMaps(ds) = mapper + shardMaps.put(ds, mapper) // NOTE: QueryActor has AtomicRef so no need to forward message to it }