diff --git a/README.md b/README.md
index 4f8b95c488..a7e36d6b0e 100644
--- a/README.md
+++ b/README.md
@@ -52,6 +52,7 @@ and [filodb-discuss](https://groups.google.com/forum/#!forum/filodb-discuss) goo
- [Code Walkthrough](#code-walkthrough)
- [Building and Testing](#building-and-testing)
- [Debugging serialization and queries](#debugging-serialization-and-queries)
+ - [Other debugging tips](#other-debugging-tips)
- [Benchmarking](#benchmarking)
- [You can help!](#you-can-help)
@@ -708,6 +709,13 @@ To dynamically change the log level, you can use the `/admin/loglevel` HTTP API
curl -d 'trace' http://localhost:8080/admin/loglevel/com.esotericsoftware.minlog
+### Other debugging tips
+
+To debug raw record ingestion and data mismatches:
+
+* Set the `trace-filters` source config setting... see `timeseries-dev-source.conf` and `TimeSeriesShard` `tracedPartFilters`. This will log every sample for time series matching criteria set in trace-filters.
+* Use the filtering capability in `filodb.kafka.TestConsumer` to print out raw records received from Kafka.
+
### Benchmarking
To run benchmarks, from within SBT:
diff --git a/cassandra/src/main/scala/filodb.cassandra/columnstore/PartitionIndexTable.scala b/cassandra/src/main/scala/filodb.cassandra/columnstore/PartitionIndexTable.scala
index e108b1d49e..2d028686b9 100644
--- a/cassandra/src/main/scala/filodb.cassandra/columnstore/PartitionIndexTable.scala
+++ b/cassandra/src/main/scala/filodb.cassandra/columnstore/PartitionIndexTable.scala
@@ -6,7 +6,7 @@ import java.nio.ByteBuffer
import scala.collection.JavaConverters._
import scala.concurrent.{ExecutionContext, Future}
-import com.datastax.driver.core.ConsistencyLevel
+import com.datastax.driver.core.{ConsistencyLevel, SimpleStatement}
import monix.reactive.Observable
import filodb.cassandra.FiloCassandraConnector
@@ -40,17 +40,18 @@ sealed class PartitionIndexTable(val dataset: DatasetRef,
|) WITH compression = {
'sstable_compression': '$sstableCompression'}""".stripMargin
- lazy val readCql = session.prepare(s"SELECT segmentid, segment " +
- s"FROM $tableString WHERE shard = ? AND timebucket = ? order by segmentid asc")
+ lazy val readCql = s"SELECT segmentid, segment " +
+ s"FROM $tableString WHERE shard = ? AND timebucket = ? order by segmentid asc"
lazy val writePartitionCql = session.prepare(
s"INSERT INTO $tableString (shard, timebucket, segmentid, segment) VALUES (?, ?, ?, ?) USING TTL ?")
.setConsistencyLevel(writeConsistencyLevel)
def getPartKeySegments(shard: Int, timeBucket: Int): Observable[PartKeyTimeBucketSegment] = {
- val it = session.execute(readCql.bind(shard: JInt, timeBucket: JInt))
+ // fetch size should be low since each row is about an MB. Default fetchSize can result in ReadTimeouts at server
+ val it = session.execute(new SimpleStatement(readCql, shard: JInt, timeBucket: JInt).setFetchSize(15))
.asScala.toIterator.map(row => {
- PartKeyTimeBucketSegment(row.getInt("segmentid"), row.getBytes("segment"))
+ PartKeyTimeBucketSegment(row.getInt("segmentid"), row.getBytes("segment"))
})
Observable.fromIterator(it)
}
diff --git a/cassandra/src/main/scala/filodb.cassandra/metastore/CassandraMetaStore.scala b/cassandra/src/main/scala/filodb.cassandra/metastore/CassandraMetaStore.scala
index c49ee84d98..6038e26ecf 100644
--- a/cassandra/src/main/scala/filodb.cassandra/metastore/CassandraMetaStore.scala
+++ b/cassandra/src/main/scala/filodb.cassandra/metastore/CassandraMetaStore.scala
@@ -59,12 +59,12 @@ class CassandraMetaStore(config: Config, filoSessionProvider: Option[FiloSession
checkpointTable.writeCheckpoint(dataset, shardNum, groupNum, offset)
}
- def readCheckpoints(dataset: DatasetRef, shardNum: Int): Future[Map[Int,Long]] = {
+ def readCheckpoints(dataset: DatasetRef, shardNum: Int): Future[Map[Int, Long]] = {
checkpointTable.readCheckpoints(dataset, shardNum)
}
def readEarliestCheckpoint(dataset: DatasetRef, shardNum: Int): Future[Long] = {
- readCheckpoints(dataset,shardNum) map { m =>
+ readCheckpoints(dataset, shardNum) map { m =>
if (m.values.isEmpty) Long.MinValue else m.values.min
}
}
diff --git a/cassandra/src/main/scala/filodb.cassandra/metastore/CheckpointTable.scala b/cassandra/src/main/scala/filodb.cassandra/metastore/CheckpointTable.scala
index f71eddb328..807a8b00f3 100644
--- a/cassandra/src/main/scala/filodb.cassandra/metastore/CheckpointTable.scala
+++ b/cassandra/src/main/scala/filodb.cassandra/metastore/CheckpointTable.scala
@@ -76,7 +76,7 @@ sealed class CheckpointTable(val config: Config,
dataset.dataset, shardNum: JInt, groupNum: JInt, offset: JLong))
}
- def readCheckpoints(dataset: DatasetRef, shardNum: Int): Future[Map[Int,Long]] = {
+ def readCheckpoints(dataset: DatasetRef, shardNum: Int): Future[Map[Int, Long]] = {
session.executeAsync(readCheckpointCql.bind(dataset.database.getOrElse(""),
dataset.dataset, shardNum: JInt))
.toIterator // future of Iterator
diff --git a/cli/src/main/scala/filodb.cli/CliMain.scala b/cli/src/main/scala/filodb.cli/CliMain.scala
index 47b153b2f7..85930008dc 100644
--- a/cli/src/main/scala/filodb.cli/CliMain.scala
+++ b/cli/src/main/scala/filodb.cli/CliMain.scala
@@ -13,6 +13,7 @@ import monix.reactive.Observable
import filodb.coordinator._
import filodb.coordinator.client._
+import filodb.coordinator.client.QueryCommands.{SpreadChange, SpreadProvider, StaticSpreadProvider}
import filodb.core._
import filodb.core.metadata.{Column, Dataset, DatasetOptions}
import filodb.core.store._
@@ -60,6 +61,7 @@ class Arguments extends FieldArgs {
var ignoreTagsOnPartitionKeyHash: Seq[String] = Nil
var everyNSeconds: Option[String] = None
var shards: Option[Seq[String]] = None
+ var spread: Option[Integer] = None
}
object CliMain extends ArgMain[Arguments] with CsvImportExport with FilodbClusterNode {
@@ -116,7 +118,6 @@ object CliMain extends ArgMain[Arguments] with CsvImportExport with FilodbCluste
}
def main(args: Arguments): Unit = {
- val spread = config.getInt("default-spread")
try {
val timeout = args.timeoutSeconds.seconds
args.command match {
@@ -198,7 +199,7 @@ object CliMain extends ArgMain[Arguments] with CsvImportExport with FilodbCluste
require(args.host.nonEmpty && args.dataset.nonEmpty && args.matcher.nonEmpty, "--host, --dataset and --matcher must be defined")
val remote = Client.standaloneClient(system, args.host.get, args.port)
val options = QOptions(args.limit, args.sampleLimit, args.everyNSeconds.map(_.toInt),
- timeout, args.shards.map(_.map(_.toInt)), spread)
+ timeout, args.shards.map(_.map(_.toInt)), args.spread)
parseTimeSeriesMetadataQuery(remote, args.matcher.get, args.dataset.get,
getQueryRange(args), options)
@@ -206,7 +207,7 @@ object CliMain extends ArgMain[Arguments] with CsvImportExport with FilodbCluste
require(args.host.nonEmpty && args.dataset.nonEmpty && args.labelNames.nonEmpty, "--host, --dataset and --labelName must be defined")
val remote = Client.standaloneClient(system, args.host.get, args.port)
val options = QOptions(args.limit, args.sampleLimit, args.everyNSeconds.map(_.toInt),
- timeout, args.shards.map(_.map(_.toInt)), spread)
+ timeout, args.shards.map(_.map(_.toInt)), args.spread)
parseLabelValuesQuery(remote, args.labelNames, args.labelFilter, args.dataset.get,
getQueryRange(args), options)
@@ -216,7 +217,7 @@ object CliMain extends ArgMain[Arguments] with CsvImportExport with FilodbCluste
require(args.host.nonEmpty && args.dataset.nonEmpty, "--host and --dataset must be defined")
val remote = Client.standaloneClient(system, args.host.get, args.port)
val options = QOptions(args.limit, args.sampleLimit, args.everyNSeconds.map(_.toInt),
- timeout, args.shards.map(_.map(_.toInt)), spread)
+ timeout, args.shards.map(_.map(_.toInt)), args.spread)
parsePromQuery2(remote, query, args.dataset.get, getQueryRange(args), options)
}.orElse {
args.select.map { selectCols =>
@@ -267,13 +268,13 @@ object CliMain extends ArgMain[Arguments] with CsvImportExport with FilodbCluste
ignoreTagsOnPartitionKeyHash: Seq[String],
timeout: FiniteDuration): Unit = {
try {
- val datasetObj = Dataset(dataset.dataset, partitionColumns, dataColumns, rowKeys, downsamplers)
val options = DatasetOptions.DefaultOptions.copy(metricColumn = metricColumn,
shardKeyColumns = shardKeyColumns,
ignoreShardKeyColumnSuffixes = ignoreShardKeyColumnSuffixes,
ignoreTagsOnPartitionKeyHash = ignoreTagsOnPartitionKeyHash)
+ val datasetObj = Dataset(dataset.dataset, partitionColumns, dataColumns, rowKeys, downsamplers, options)
println(s"Creating dataset $dataset with options $options...")
- client.createNewDataset(datasetObj.copy(options = options), dataset.database)
+ client.createNewDataset(datasetObj, dataset.database)
exitCode = 0
} catch {
case b: Dataset.BadSchemaError =>
@@ -339,7 +340,7 @@ object CliMain extends ArgMain[Arguments] with CsvImportExport with FilodbCluste
final case class QOptions(limit: Int, sampleLimit: Int,
everyN: Option[Int], timeout: FiniteDuration,
shardOverrides: Option[Seq[Int]],
- spread: Int)
+ spread: Option[Integer])
def parseTimeSeriesMetadataQuery(client: LocalClient, query: String, dataset: String,
timeParams: TimeRangeParams,
@@ -364,7 +365,8 @@ 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.sampleLimit)
+ val spreadProvider: Option[SpreadProvider] = options.spread.map(s => StaticSpreadProvider(SpreadChange(0, s)))
+ val qOpts = QueryCommands.QueryOptions(spreadProvider, options.sampleLimit)
.copy(queryTimeoutSecs = options.timeout.toSeconds.toInt,
shardOverrides = options.shardOverrides)
println(s"Sending query command to server for $ref with options $qOpts...")
diff --git a/conf/histogram-dev-source.conf b/conf/histogram-dev-source.conf
index 51b7c6e39c..71afdccf1b 100644
--- a/conf/histogram-dev-source.conf
+++ b/conf/histogram-dev-source.conf
@@ -36,9 +36,8 @@
# 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 bytes of offheap mem to allocate to write buffers for all shards. Ex. 1000MB, 1G, 2GB
+ ingestion-buffer-mem-size = 200MB
# Number of time series to evict at a time.
# num-partitions-to-evict = 1000
diff --git a/conf/logback-dev.xml b/conf/logback-dev.xml
index 88b81cdd38..94a9c3fe14 100644
--- a/conf/logback-dev.xml
+++ b/conf/logback-dev.xml
@@ -29,6 +29,5 @@
-
-
\ No newline at end of file
+
diff --git a/conf/timeseries-dev-source.conf b/conf/timeseries-dev-source.conf
index 961f17c2d4..d758f3d334 100644
--- a/conf/timeseries-dev-source.conf
+++ b/conf/timeseries-dev-source.conf
@@ -37,9 +37,12 @@
# 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 bytes of offheap mem to allocate to write buffers for all shards. Ex. 1000MB, 1G, 2GB
+ ingestion-buffer-mem-size = 200MB
+
+ # Maximum numer of write buffers to retain in each shard's WriteBufferPool. Any extra buffers are released
+ # back to native memory, which helps memory reuse.
+ # max-buffer-pool-size = 10000
# Number of time series to evict at a time.
# num-partitions-to-evict = 1000
@@ -72,6 +75,11 @@
# Thats about 6MB
evicted-pk-bloom-filter-capacity = 50000
+ # Uncomment to log at DEBUG ingested samples matching these filters on Partition Key columns
+ # Only works for StringColumn fields in the partition key. Scope may be expanded later.
+ # trace-filters = {
+ # metric = "bad-metric-to-log"
+ # }
}
downsample {
# can be disabled by setting this flag to false
diff --git a/conf/timeseries-ds-1m-dev-source.conf b/conf/timeseries-ds-1m-dev-source.conf
index d6364319ec..a8ac823758 100644
--- a/conf/timeseries-ds-1m-dev-source.conf
+++ b/conf/timeseries-ds-1m-dev-source.conf
@@ -33,9 +33,8 @@
# 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 bytes of offheap mem to allocate to write buffers for all shards. Ex. 1000MB, 1G, 2GB
+ ingestion-buffer-mem-size = 200MB
# Number of time series to evict at a time.
# num-partitions-to-evict = 1000
diff --git a/conf/timeseries-filodb-server.conf b/conf/timeseries-filodb-server.conf
index 17b40d666c..83a1a4afbe 100644
--- a/conf/timeseries-filodb-server.conf
+++ b/conf/timeseries-filodb-server.conf
@@ -5,6 +5,19 @@ filodb {
port = 9042
partition-list-num-groups = 1
}
+ spread-default = 1
+
+ # Override default spread for application using override block which will have non metric shard keys and spread.
+ spread-assignment = [
+ {
+ _ns = App-0,
+ _spread_ = 2
+ },
+ {
+ _ns = App-5,
+ _spread_ = 0
+ }
+ ]
}
kamon {
diff --git a/conf/timeseries-standalonetest-source.conf b/conf/timeseries-standalonetest-source.conf
index d7e13a135d..a71b1e6199 100644
--- a/conf/timeseries-standalonetest-source.conf
+++ b/conf/timeseries-standalonetest-source.conf
@@ -18,7 +18,7 @@
max-chunks-size = 400
shard-mem-size = 512MB
- ingestion-buffer-mem-size = 50MB
+ ingestion-buffer-mem-size = 200MB
groups-per-shard = 20
multi-partition-odp = false
}
diff --git a/coordinator/src/main/scala/filodb.coordinator/IngestionActor.scala b/coordinator/src/main/scala/filodb.coordinator/IngestionActor.scala
index 228225249c..d6c62724ba 100644
--- a/coordinator/src/main/scala/filodb.coordinator/IngestionActor.scala
+++ b/coordinator/src/main/scala/filodb.coordinator/IngestionActor.scala
@@ -1,6 +1,6 @@
package filodb.coordinator
-import scala.collection.mutable.HashMap
+import scala.collection.mutable.{HashMap, HashSet}
import scala.concurrent.Future
import scala.util.{Failure, Success, Try}
import scala.util.control.NonFatal
@@ -36,7 +36,7 @@ object IngestionActor {
/**
* Oversees ingestion and recovery process for a single dataset. The overall process for a single shard:
- * 1. StartShardIngestion command is received and start() called
+ * 1. Shard state command is received and start() called
* 2. MemStore.setup() is called for that shard
* 3. IF no checkpoint data is found, THEN normal ingestion is started
* 4. IF checkpoints are found, then recovery is started from the minimum checkpoint offset
@@ -61,6 +61,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
+ var shardStateVersion: Long = 0
// Params for creating the default memStore flush scheduler
private final val numGroups = storeConfig.groupsPerShard
@@ -83,66 +84,114 @@ private[filodb] final class IngestionActor(dataset: Dataset,
override def postStop(): Unit = {
super.postStop() // <- logs shutting down
logger.info("Cancelling all streams and calling teardown")
- streamSubscriptions.keys.foreach(stop(dataset.ref, _, ActorRef.noSender))
+ streamSubscriptions.keys.foreach(stopIngestion(_))
}
- /** All [[ShardCommand]] tasks are only started if the dataset
- * and shard are valid for this ingester.
- */
def receive: Receive = LoggingReceive {
- case e: StartShardIngestion => start(e, sender())
- case e: IngestRows => ingest(e)
- case GetStatus => status(sender())
- case StopShardIngestion(ds, shard) => stop(ds, shard, sender())
+ case GetStatus => status(sender())
+ case e: IngestRows => ingest(e)
+ case e: ShardIngestionState => resync(e, sender())
}
- /** Guards that only this dataset's commands are acted upon.
- * Handles initial memstore setup of dataset to shard.
- * Also handles recovery process.
+ /**
+ * Compares the given shard mapper snapshot to the current set of shards being ingested and
+ * reconciles any differences. It does so by stopping ingestion for shards that aren't mapped
+ * to this node, and it starts ingestion for those that are.
*/
- private def start(e: StartShardIngestion, origin: ActorRef): Unit =
- if (invalid(e.ref)) handleInvalid(e, Some(origin)) else {
- try memStore.setup(dataset, e.shard, storeConfig, downsample) catch {
- case ShardAlreadySetup(ds, shard) =>
- logger.warn(s"dataset=$ds shard=$shard already setup, skipping....")
- return
- }
+ private def resync(state: ShardIngestionState, origin: ActorRef): Unit = {
+ if (invalid(state.ref)) {
+ logger.error(s"$state is invalid for this ingester '${dataset.ref}'.")
+ return
+ }
- val ingestion = for {
- _ <- memStore.recoverIndex(dataset.ref, e.shard)
- checkpoints <- memStore.metastore.readCheckpoints(dataset.ref, e.shard) }
- yield {
- if (checkpoints.isEmpty) {
- // Start normal ingestion with no recovery checkpoint and flush group 0 first
- normalIngestion(e.shard, None, 0, storeConfig.diskTTLSeconds)
- } else {
- // Figure out recovery end watermark and intervals. The reportingInterval is the interval at which
- // offsets come back from the MemStore for us to report progress.
- val startRecoveryWatermark = checkpoints.values.min + 1
- val endRecoveryWatermark = checkpoints.values.max
- val lastFlushedGroup = checkpoints.find(_._2 == endRecoveryWatermark).get._1
- val reportingInterval = Math.max((endRecoveryWatermark - startRecoveryWatermark) / 20, 1L)
- logger.info(s"Starting recovery for dataset=${dataset.ref} " +
- s"shard=${e.shard}: from $startRecoveryWatermark to $endRecoveryWatermark; " +
- s"last flushed group $lastFlushedGroup")
- logger.info(s"Checkpoints for dataset=${dataset.ref} shard=${e.shard}: $checkpoints")
- for { lastOffset <- doRecovery(e.shard, startRecoveryWatermark, endRecoveryWatermark, reportingInterval,
- checkpoints) }
- yield {
- // Start reading past last offset for normal records; start flushes one group past last group
- normalIngestion(e.shard, Some(lastOffset + 1), (lastFlushedGroup + 1) % numGroups,
- storeConfig.diskTTLSeconds)
+ if (state.version != 0 && state.version <= shardStateVersion) {
+ logger.info(s"Ignoring old ShardIngestionState version: ${state.version} <= $shardStateVersion")
+ return
+ }
+
+ // Start with the full set of all shards being ingested, and remove shards from this set
+ // which must continue being ingested.
+ val shardsToStop = HashSet() ++ streams.keySet
+
+ for (shard <- 0 until state.map.numShards) {
+ if (state.map.coordForShard(shard) == context.parent) {
+ if (state.map.isAnIngestionState(shard)) {
+ if (shardsToStop.contains(shard)) {
+ // Is aready ingesting, and it must not be stopped.
+ shardsToStop.remove(shard)
+ } else {
+ try {
+ // Isn't ingesting, so start it.
+ startIngestion(shard)
+ } catch {
+ case t: Throwable =>
+ logger.error(s"Error occurred during initialization of ingestion for " +
+ s"dataset=${dataset.ref} shard=${shard}", t)
+ handleError(dataset.ref, shard, t)
+ }
}
+ } else {
+ val status = state.map.statuses(shard)
+ logger.info(s"Will stop ingestion of shard $shard due to status ${status}")
}
}
- ingestion.recover {
- case NonFatal(t) =>
- logger.error(s"Error occurred during initialization/execution of ingestion for " +
- s"dataset=${dataset.ref} shard=${e.shard}", t)
- handleError(dataset.ref, e.shard, t)
+ }
+
+ // Stop ingesting the rest.
+ for (shard <- shardsToStop) {
+ stopIngestion(shard)
+ }
+
+ if (state.version != 0) {
+ shardStateVersion = state.version
+ }
+ }
+
+ private def startIngestion(shard: Int): Unit = {
+ try memStore.setup(dataset, shard, storeConfig, downsample) catch {
+ case ShardAlreadySetup(ds, shard) =>
+ logger.warn(s"dataset=$ds shard=$shard already setup, skipping....")
+ return
+ }
+
+ logger.info(s"Initiating ingestion for dataset=${dataset.ref} shard=${shard}")
+ val ingestion = for {
+ _ <- memStore.recoverIndex(dataset.ref, shard)
+ checkpoints <- memStore.metastore.readCheckpoints(dataset.ref, shard) }
+ yield {
+ if (checkpoints.isEmpty) {
+ logger.info(s"No checkpoints were found for dataset=${dataset.ref} shard=${shard} -- skipping kafka recovery")
+ // Start normal ingestion with no recovery checkpoint and flush group 0 first
+ normalIngestion(shard, None, 0, storeConfig.diskTTLSeconds)
+ } else {
+ // Figure out recovery end watermark and intervals. The reportingInterval is the interval at which
+ // offsets come back from the MemStore for us to report progress.
+ val startRecoveryWatermark = checkpoints.values.min + 1
+ val endRecoveryWatermark = checkpoints.values.max
+ val lastFlushedGroup = checkpoints.find(_._2 == endRecoveryWatermark).get._1
+ val reportingInterval = Math.max((endRecoveryWatermark - startRecoveryWatermark) / 20, 1L)
+ logger.info(s"Starting recovery for dataset=${dataset.ref} " +
+ s"shard=${shard} from $startRecoveryWatermark to $endRecoveryWatermark ; " +
+ s"last flushed group $lastFlushedGroup")
+ logger.info(s"Checkpoints for dataset=${dataset.ref} shard=${shard} were $checkpoints")
+ for { lastOffset <- doRecovery(shard, startRecoveryWatermark, endRecoveryWatermark, reportingInterval,
+ checkpoints) }
+ yield {
+ // Start reading past last offset for normal records; start flushes one group past last group
+ normalIngestion(shard, Some(lastOffset.getOrElse(endRecoveryWatermark) + 1),
+ (lastFlushedGroup + 1) % numGroups, storeConfig.diskTTLSeconds)
+ }
}
}
+ ingestion.recover {
+ case NonFatal(t) =>
+ logger.error(s"Error occurred during initialization/execution of ingestion for " +
+ s"dataset=${dataset.ref} shard=${shard}", t)
+ handleError(dataset.ref, shard, t)
+ }
+ }
+
private def flushStream(startGroupNo: Int = 0): Observable[FlushCommand] = {
if (source.config.as[Option[Boolean]]("noflush").getOrElse(false)) {
FlushStream.empty
@@ -180,7 +229,7 @@ private[filodb] final class IngestionActor(dataset: Dataset,
case Failure(x) =>
handleError(dataset.ref, shard, x)
case Success(_) =>
- // We dont release resources when fitite ingestion ends normally.
+ // We dont release resources when finite ingestion ends normally.
// Kafka ingestion is usually infinite and does not end unless canceled.
// Cancel operation is already releasing after cancel is done.
// We also have some tests that validate after finite ingestion is complete
@@ -204,7 +253,7 @@ private[filodb] final class IngestionActor(dataset: Dataset,
* @param interval the interval of reporting progress
*/
private def doRecovery(shard: Int, startOffset: Long, endOffset: Long, interval: Long,
- checkpoints: Map[Int, Long]): Future[Long] = {
+ checkpoints: Map[Int, Long]): Future[Option[Long]] = {
val futTry = create(shard, Some(startOffset)) map { ingestionStream =>
val recoveryTrace = Kamon.buildSpan("ingestion-recovery-trace")
.withTag("shard", shard.toString)
@@ -213,7 +262,7 @@ private[filodb] final class IngestionActor(dataset: Dataset,
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)
+ val fut = memStore.recoverStream(dataset.ref, shard, stream, startOffset, endOffset, checkpoints, interval)
.map { off =>
val progressPct = if (endOffset - startOffset == 0) 100
else (off - startOffset) * 100 / (endOffset - startOffset)
@@ -223,9 +272,10 @@ private[filodb] final class IngestionActor(dataset: Dataset,
off }
.until(_ >= endOffset)
// TODO: move this code to TimeSeriesShard itself. Shard should control the thread
- .lastL.runAsync(shardInstance.ingestSched)
+ .lastOptionL.runAsync(shardInstance.ingestSched)
fut.onComplete {
case Success(_) =>
+ logger.info(s"Finished recovery for dataset=${dataset.ref} shard=$shard")
ingestionStream.teardown()
streams.remove(shard)
recoveryTrace.finish()
@@ -276,21 +326,19 @@ private[filodb] final class IngestionActor(dataset: Dataset,
private def status(origin: ActorRef): Unit =
origin ! IngestionStatus(memStore.numRowsIngested(dataset.ref))
- /** Guards that only this dataset's commands are acted upon. */
- private def stop(ds: DatasetRef, shard: Int, origin: ActorRef): Unit =
- if (invalid(ds)) handleInvalid(StopShardIngestion(ds, shard), Some(origin)) else {
- streamSubscriptions.get(shard).foreach { s =>
- s.onComplete {
- case Success(_) =>
- // release resources when stop is invoked explicitly, not when ingestion ends in non-kafka environments
- removeAndReleaseResources(ds, shard)
- // ingestion stopped event is already handled in the normalIngestion method
- logger.info(s"Stopped streaming ingestion for dataset=${dataset.ref} shard=$shard and released resources")
- case Failure(_) =>
- // release of resources on failure is already handled in the normalIngestion method
- }
+ private def stopIngestion(shard: Int): Unit = {
+ streamSubscriptions.get(shard).foreach { s =>
+ s.onComplete {
+ case Success(_) =>
+ // release resources when stop is invoked explicitly, not when ingestion ends in non-kafka environments
+ removeAndReleaseResources(dataset.ref, shard)
+ // ingestion stopped event is already handled in the normalIngestion method
+ logger.info(s"Stopped streaming ingestion for dataset=${dataset.ref} shard=$shard and released resources")
+ case Failure(_) =>
+ // release of resources on failure is already handled in the normalIngestion method
}
- streamSubscriptions.get(shard).foreach(_.cancel())
+ }
+ streamSubscriptions.get(shard).foreach(_.cancel())
}
private def invalid(ref: DatasetRef): Boolean = ref != dataset.ref
@@ -303,19 +351,16 @@ private[filodb] final class IngestionActor(dataset: Dataset,
logger.error(s"Stopped dataset=${dataset.ref} shard=$shard after error was thrown")
}
- private def handleInvalid(command: ShardCommand, origin: Option[ActorRef]): Unit = {
- logger.error(s"$command is invalid for this ingester '${dataset.ref}'.")
- origin foreach(_ ! InvalidIngestionCommand(command.ref, command.shard))
- }
-
private def removeAndReleaseResources(ref: DatasetRef, shard: Int): Unit = {
- // TODO: Wait for all the queries to stop
- streamSubscriptions.remove(shard).foreach(_.cancel)
- streams.remove(shard).foreach(_.teardown())
- // Release memory for shard in MemStore
- memStore.asInstanceOf[TimeSeriesMemStore].getShard(ref, shard)
- .foreach { shard =>
- shard.shutdown()
- }
+ if (streamSubscriptions.contains(shard)) {
+ // TODO: Wait for all the queries to stop
+ streamSubscriptions.remove(shard).foreach(_.cancel)
+ streams.remove(shard).foreach(_.teardown())
+ // Release memory for shard in MemStore
+ memStore.asInstanceOf[TimeSeriesMemStore].getShard(ref, shard)
+ .foreach { shard =>
+ shard.shutdown()
+ }
+ }
}
-}
\ No newline at end of file
+}
diff --git a/coordinator/src/main/scala/filodb.coordinator/KamonLogger.scala b/coordinator/src/main/scala/filodb.coordinator/KamonLogger.scala
index 7c69d77040..67665a07f0 100644
--- a/coordinator/src/main/scala/filodb.coordinator/KamonLogger.scala
+++ b/coordinator/src/main/scala/filodb.coordinator/KamonLogger.scala
@@ -46,7 +46,7 @@ class KamonMetricsLogReporter extends MetricReporter with StrictLogging {
}
}
- private def formatTags(tags: Map[String, String]) = tags.view.map { case (k,v) => s"$k=$v" }.mkString(" ")
+ private def formatTags(tags: Map[String, String]) = tags.view.map { case (k, v) => s"$k=$v" }.mkString(" ")
private def normalizeLabelName(label: String): String =
label.map(charOrUnderscore)
diff --git a/coordinator/src/main/scala/filodb.coordinator/NodeClusterActor.scala b/coordinator/src/main/scala/filodb.coordinator/NodeClusterActor.scala
index 137850bffb..485f76d4b9 100644
--- a/coordinator/src/main/scala/filodb.coordinator/NodeClusterActor.scala
+++ b/coordinator/src/main/scala/filodb.coordinator/NodeClusterActor.scala
@@ -206,7 +206,6 @@ private[filodb] class NodeClusterActor(settings: FilodbSettings,
val shardManager = new ShardManager(settings, assignmentStrategy)
val localRemoteAddr = RemoteAddressExtension(context.system).address
var everybodyLeftSender: Option[ActorRef] = None
- val shardUpdates = new MutableHashSet[DatasetRef]
// Counter is incremented each time shardmapper snapshot is published.
// value > 0 implies that the node is a ShardManager. For rest of the nodes metric will not be reported.
@@ -258,7 +257,6 @@ private[filodb] class NodeClusterActor(settings: FilodbSettings,
}
private def memberUp(member: Member): Future[Unit] = {
- logger.info(s"Member ${member.status}: ${member.address} with roles ${member.roles}")
val memberCoordActor = nodeCoordinatorPath(member.address)
context.actorSelection(memberCoordActor).resolveOne(ResolveActorTimeout)
.map { ref => self ! AddCoordinator(member.roles, member.address, ref) }
@@ -272,12 +270,14 @@ private[filodb] class NodeClusterActor(settings: FilodbSettings,
def membershipHandler: Receive = LoggingReceive {
case s: CurrentClusterState =>
logger.info(s"Initial Cluster State was: $s")
+ shardManager.logAllMappers("After receiving initial cluster state")
val memberUpFutures = s.members.filter(_.status == MemberStatus.Up).map(memberUp(_))
Future.sequence(memberUpFutures.toSeq).onComplete { _ =>
self ! RemoveStaleCoordinators
}
case MemberUp(member) =>
+ logger.info(s"Member ${member.status}: ${member.address} with roles ${member.roles}")
memberUp(member)
case UnreachableMember(member) =>
@@ -337,6 +337,7 @@ private[filodb] class NodeClusterActor(settings: FilodbSettings,
// never includes downed nodes, which come through cluster.subscribe event replay
mappers foreach { case (ref, map) => shardManager.recoverShards(ref, map) }
shardManager.recoverSubscriptions(subscriptions)
+ shardManager.logAllMappers("After NCA shard state/subscription recovery")
// NOW, subscribe to cluster membership state and then switch to normal receiver
logger.info("Subscribing to cluster events and switching to normalReceive")
@@ -350,7 +351,7 @@ private[filodb] class NodeClusterActor(settings: FilodbSettings,
// handleEventEnvelope() currently acks right away, so there is a chance that this actor dies between receiving
// a new event and the new snapshot is published.
private def scheduleSnapshotPublishes() = {
- pubTask = Some(context.system.scheduler.schedule(1.second, publishInterval, self, PublishSnapshot))
+ pubTask = Some(context.system.scheduler.schedule(1.minute, publishInterval, self, PublishSnapshot))
}
def shardMapHandler: Receive = LoggingReceive {
@@ -364,19 +365,16 @@ private[filodb] class NodeClusterActor(settings: FilodbSettings,
def subscriptionHandler: Receive = LoggingReceive {
case e: ShardEvent => handleShardEvent(e)
case e: StatusActor.EventEnvelope => handleEventEnvelope(e, sender())
- case PublishSnapshot => shardUpdates.foreach(shardManager.publishSnapshot)
+ case PublishSnapshot => datasets.keys.foreach(shardManager.publishSnapshot)
//This counter gets published from ShardManager,
// > 0 means this node is shardmanager
iamShardManager.increment()
- shardUpdates.clear()
case e: SubscribeShardUpdates => subscribe(e.ref, sender())
case SubscribeAll => subscribeAll(sender())
case Terminated(subscriber) => context unwatch subscriber
}
private def handleShardEvent(e: ShardEvent) = {
- logger.debug(s"Received ShardEvent $e from $sender")
- shardUpdates += e.ref
shardManager.updateFromExternalShardEvent(sender(), e)
}
@@ -466,7 +464,6 @@ private[filodb] class NodeClusterActor(settings: FilodbSettings,
logger.info("Resetting all dataset state except membership.")
datasets.clear()
sources.clear()
- shardUpdates.clear()
implicit val timeout: Timeout = DefaultTaskTimeout
shardManager.reset()
diff --git a/coordinator/src/main/scala/filodb.coordinator/NodeCoordinatorActor.scala b/coordinator/src/main/scala/filodb.coordinator/NodeCoordinatorActor.scala
index e0c2e3a8c0..141a5d9683 100644
--- a/coordinator/src/main/scala/filodb.coordinator/NodeCoordinatorActor.scala
+++ b/coordinator/src/main/scala/filodb.coordinator/NodeCoordinatorActor.scala
@@ -114,8 +114,7 @@ private[filodb] final class NodeCoordinatorActor(metaStore: MetaStore,
}
/** Creates a new ingestion actor initialized with the shard actor,
- * and sends it a sequence of `StartShardIngestion` commands created
- * during dataset setup.
+ * and sends it a shard resync command created.
*
* Creates a QueryActor, subscribes it to shard events, keeping
* it decoupled from the shard actor. The QueryActor will receive an
@@ -164,7 +163,6 @@ private[filodb] final class NodeCoordinatorActor(metaStore: MetaStore,
case DatasetSetup(compactDSString, storeConf, source, downsample) =>
val dataset = Dataset.fromCompactString(compactDSString)
if (!(ingesters contains dataset.ref)) { setupDataset(dataset, storeConf, source, downsample, sender()) }
- else { logger.warn(s"Getting redundant DatasetSetup for dataset ${dataset.ref}") }
case IngestRows(dataset, shard, rows) =>
withIngester(sender(), dataset) { _ ! IngestionActor.IngestRows(sender(), shard, rows) }
@@ -185,7 +183,7 @@ private[filodb] final class NodeCoordinatorActor(metaStore: MetaStore,
def coordinatorReceive: Receive = LoggingReceive {
case e: CoordinatorRegistered => registered(e)
- case e: ShardCommand => forward(e, sender())
+ case e: ShardIngestionState => forward(e, e.ref, sender())
case Terminated(memstoreCoord) => terminated(memstoreCoord)
case MiscCommands.GetClusterActor => sender() ! clusterActor
case StatusActor.GetCurrentEvents => statusActor.foreach(_.tell(StatusActor.GetCurrentEvents, sender()))
@@ -209,15 +207,15 @@ private[filodb] final class NodeCoordinatorActor(metaStore: MetaStore,
}
}
- /** Forwards shard commands to the ingester for the given dataset.
+ /** Forwards shard actions to the ingester for the given dataset.
* TODO version match if needed, when > 1, currently only 0.
*/
- private def forward(command: ShardCommand, origin: ActorRef): Unit =
- ingesters.get(command.ref) match {
+ private def forward(action: ShardAction, ref: DatasetRef, origin: ActorRef): Unit =
+ ingesters.get(ref) match {
case Some(actor) =>
- actor.tell(command, origin)
+ actor.tell(action, origin)
case _ =>
- logger.warn(s"No IngestionActor for dataset ${command.ref}")
+ logger.warn(s"No IngestionActor for dataset ${ref}")
}
private def terminated(ingester: ActorRef): Unit = {
@@ -255,4 +253,4 @@ private[filodb] final class NodeCoordinatorActor(metaStore: MetaStore,
// This is a bit heavy handed, it clears out the entire cache, not just for all datasets
memStore.reset()
}
-}
\ No newline at end of file
+}
diff --git a/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala b/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala
index d01bb625e6..5fad8e3350 100644
--- a/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala
+++ b/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala
@@ -2,13 +2,14 @@ package filodb.coordinator
import java.util.concurrent.atomic.AtomicLong
-import scala.util.control.NonFatal
-
import akka.actor.{ActorRef, ActorSystem, Props}
import akka.dispatch.{Envelope, UnboundedStablePriorityMailbox}
import com.typesafe.config.Config
import kamon.Kamon
import monix.execution.Scheduler
+import net.ceedubs.ficus.Ficus._
+import net.ceedubs.ficus.readers.ValueReader
+import scala.util.control.NonFatal
import filodb.coordinator.queryengine2.QueryEngine
import filodb.core._
@@ -55,6 +56,23 @@ final class QueryActor(memStore: MemStore,
val config = context.system.settings.config
+ var filodbSpreadMap = new collection.mutable.HashMap[collection.Map[String, String], Int]
+ val applicationShardKeyName = dataset.options.nonMetricShardColumns(0)
+ val defaultSpread = config.getInt("filodb.spread-default")
+
+ implicit val spreadOverrideReader: ValueReader[SpreadAssignment] = ValueReader.relative { spreadAssignmentConfig =>
+ SpreadAssignment(
+ shardKeysMap = dataset.options.nonMetricShardColumns.map(x =>
+ (x, spreadAssignmentConfig.getString(x))).toMap[String, String],
+ spread = spreadAssignmentConfig.getInt("_spread_")
+ )
+ }
+ val spreadAssignment : List[SpreadAssignment]= config.as[List[SpreadAssignment]]("filodb.spread-assignment")
+ spreadAssignment.foreach{ x => filodbSpreadMap.put(x.shardKeysMap, x.spread)}
+
+ val spreadFunc = QueryOptions.simpleMapSpreadFunc(applicationShardKeyName, filodbSpreadMap, defaultSpread)
+ val functionalSpreadProvider = FunctionalSpreadProvider(spreadFunc)
+
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)
@@ -91,11 +109,15 @@ final class QueryActor(memStore: MemStore,
}
}
+ private def getSpreadProvider(queryOptions: QueryOptions): SpreadProvider = {
+ return queryOptions.spreadProvider.getOrElse(functionalSpreadProvider)
+ }
+
private def processLogicalPlan2Query(q: LogicalPlan2Query, replyTo: ActorRef) = {
// This is for CLI use only. Always prefer clients to materialize logical plan
lpRequests.increment
try {
- val execPlan = queryEngine2.materialize(q.logicalPlan, q.queryOptions)
+ val execPlan = queryEngine2.materialize(q.logicalPlan, q.queryOptions, getSpreadProvider(q.queryOptions))
self forward execPlan
} catch {
case NonFatal(ex) =>
@@ -104,6 +126,17 @@ final class QueryActor(memStore: MemStore,
}
}
+ private def processExplainPlanQuery(q: ExplainPlan2Query, replyTo: ActorRef) = {
+ try {
+ val execPlan = queryEngine2.materialize(q.logicalPlan, q.queryOptions, getSpreadProvider(q.queryOptions))
+ replyTo ! execPlan
+ } catch {
+ case NonFatal(ex) =>
+ logger.error(s"Exception while materializing logical plan", ex)
+ replyTo ! QueryError("unknown", ex)
+ }
+ }
+
private def processIndexValues(g: GetIndexValues, originator: ActorRef): Unit = {
val localShards = memStore.activeShards(g.dataset)
if (localShards contains g.shard) {
@@ -119,8 +152,9 @@ final class QueryActor(memStore: MemStore,
def receive: Receive = {
case q: LogicalPlan2Query => val replyTo = sender()
processLogicalPlan2Query(q, replyTo)
-
- case q: ExecPlan => execPhysicalPlan2(q, sender())
+ case q: ExplainPlan2Query => val replyTo = sender()
+ processExplainPlanQuery(q, replyTo)
+ case q: ExecPlan => execPhysicalPlan2(q, sender())
case GetIndexNames(ref, limit, _) =>
sender() ! memStore.indexNames(ref).take(limit).map(_._1).toBuffer
diff --git a/coordinator/src/main/scala/filodb.coordinator/ShardHealthStats.scala b/coordinator/src/main/scala/filodb.coordinator/ShardHealthStats.scala
index 62057ef248..fdf00b6d3b 100644
--- a/coordinator/src/main/scala/filodb.coordinator/ShardHealthStats.scala
+++ b/coordinator/src/main/scala/filodb.coordinator/ShardHealthStats.scala
@@ -19,15 +19,17 @@ class ShardHealthStats(ref: DatasetRef,
shardMapFunc: => ShardMapper,
reportingInterval: FiniteDuration = 5.seconds) {
- val numActive = Kamon.gauge(s"num-active-shards-$ref")
- val numRecovering = Kamon.gauge(s"num-recovering-shards-$ref")
- val numUnassigned = Kamon.gauge(s"num-unassigned-shards-$ref")
- val numAssigned = Kamon.gauge(s"num-assigned-shards-$ref")
- val numError = Kamon.gauge(s"num-error-shards-$ref")
- val numStopped = Kamon.gauge(s"num-stopped-shards-$ref")
- val numDown = Kamon.gauge(s"num-down-shards-$ref")
- val numErrorReassignmentsDone = Kamon.counter(s"num-error-reassignments-done-$ref")
- val numErrorReassignmentsSkipped = Kamon.counter(s"num-error-reassignments-skipped-$ref")
+ val numActive = Kamon.gauge(s"num-active-shards").refine("dataset", ref.toString)
+ val numRecovering = Kamon.gauge(s"num-recovering-shards").refine("dataset", ref.toString)
+ val numUnassigned = Kamon.gauge(s"num-unassigned-shards").refine("dataset", ref.toString)
+ val numAssigned = Kamon.gauge(s"num-assigned-shards").refine("dataset", ref.toString)
+ val numError = Kamon.gauge(s"num-error-shards").refine("dataset", ref.toString)
+ val numStopped = Kamon.gauge(s"num-stopped-shards").refine("dataset", ref.toString)
+ val numDown = Kamon.gauge(s"num-down-shards").refine("dataset", ref.toString)
+ val numErrorReassignmentsDone = Kamon.counter(s"num-error-reassignments-done")
+ .refine("dataset", ref.toString)
+ val numErrorReassignmentsSkipped = Kamon.counter(s"num-error-reassignments-skipped")
+ .refine("dataset", ref.toString)
def update(mapper: ShardMapper): Unit = {
numActive.set(shardMapFunc.statuses.count(_ == ShardStatusActive))
diff --git a/coordinator/src/main/scala/filodb.coordinator/ShardManager.scala b/coordinator/src/main/scala/filodb.coordinator/ShardManager.scala
index 6ea9afa123..c3f2f7667c 100644
--- a/coordinator/src/main/scala/filodb.coordinator/ShardManager.scala
+++ b/coordinator/src/main/scala/filodb.coordinator/ShardManager.scala
@@ -58,6 +58,12 @@ private[coordinator] final class ShardManager(settings: FilodbSettings,
_shardMappers foreach { case (ref, map) => subscriber ! CurrentShardSnapshot(ref, map) }
}
+ def logAllMappers(msg: String = ""): Unit = {
+ _shardMappers.foreach { case (ref, mapper) =>
+ logger.info(s"$msg dataset=$ref Current mapper state:\n${mapper.prettyPrint}")
+ }
+ }
+
/** If the mapper for the provided `datasetRef` has been added, sends an initial
* current snapshot of partition state, as ingestion will subscribe usually when
* the cluster is already stable.
@@ -71,7 +77,7 @@ private[coordinator] final class ShardManager(settings: FilodbSettings,
* INTERNAL API. Idempotent.
*/
def subscribe(subscriber: ActorRef, dataset: DatasetRef): Unit =
- mapperOpt(dataset) match {
+ mapperCopyOpt(dataset) match {
case Some(current) =>
logger.info(s"Adding $subscriber as a subscriber for dataset=$dataset")
_subscriptions = subscriptions.subscribe(subscriber, dataset)
@@ -117,8 +123,22 @@ private[coordinator] final class ShardManager(settings: FilodbSettings,
* The response is returned directly to the requester.
*/
def sendSnapshot(ref: DatasetRef, origin: ActorRef): Unit =
- origin ! mapperOpt(ref).getOrElse(DatasetUnknown(ref))
+ origin ! mapperCopyOpt(ref).getOrElse(DatasetUnknown(ref))
+ /**
+ * Returns a complete copy of the ShardMapper within a CurrentShardSnapshot, if the dataset
+ * exists. Although a copy of the ShardMapper isn't typically required, it is required for
+ * the tests to work properly. This is because the TestProbe provides access to the local
+ * ShardMapper instance, and so any observation of the snapshot would expose the latest
+ * mappings instead. The complete copy also offers a nice safeguard, in case the ShardMapper
+ * is concurrently modified before the message is sent. This isn't really expected, however.
+ */
+ private def mapperCopyOpt(ref: DatasetRef): Option[CurrentShardSnapshot] =
+ _shardMappers.get(ref).map(m => CurrentShardSnapshot(ref, m.copy()))
+
+ /**
+ * Same as mapperCopyOpt, except it directly references the ShardMapper instance.
+ */
private def mapperOpt(ref: DatasetRef): Option[CurrentShardSnapshot] =
_shardMappers.get(ref).map(m => CurrentShardSnapshot(ref, m))
@@ -134,19 +154,23 @@ private[coordinator] final class ShardManager(settings: FilodbSettings,
for ((dataset, resources, mapper) <- datasetShardMaps) {
val assignable = strategy.shardAssignments(coordinator, dataset, resources, mapper)
- if (assignable.nonEmpty) sendAssignmentMessagesAndEvents(dataset, coordinator, assignable)
+ if (assignable.nonEmpty) {
+ doAssignShards(dataset, coordinator, assignable)
+ publishChanges(dataset)
+ }
}
logger.info(s"Completed addMember for coordinator $coordinator")
}
/** Called on MemberRemoved, new status already updated. */
- def removeMember(address: Address): Option[ActorRef] =
+ def removeMember(address: Address): Option[ActorRef] = {
_coordinators.get(address) map { coordinator =>
- logger.info(s"Initiated removeMember for coordinator on $address")
- _coordinators remove address
- removeCoordinator(coordinator)
- coordinator
+ logger.info(s"Initiated removeMember for coordinator=$coordinator on $address")
+ _coordinators remove address
+ removeCoordinator(coordinator)
+ coordinator
}
+ }
private def updateShardMetrics(): Unit = {
_datasetInfo.foreach { case (dataset, info) =>
@@ -259,7 +283,7 @@ private[coordinator] final class ShardManager(settings: FilodbSettings,
*
* @return - Validates and returns error message on failure and a unit if no validation error
*/
- def validateRequestAndStopShards(shardStopReq: StopShards, ackTo: ActorRef): Unit Or ErrorResponse = {
+ private def validateRequestAndStopShards(shardStopReq: StopShards, ackTo: ActorRef): Unit Or ErrorResponse = {
for {
shardMapper <- validateDataset(shardStopReq.datasetRef)
shards <- validateShardsToStop(shardStopReq.unassignmentConfig.shardList, shardMapper)
@@ -271,14 +295,14 @@ private[coordinator] final class ShardManager(settings: FilodbSettings,
/**
* Shutdown shards from the coordinator where it is running
*/
- def unassignShards(shards: Seq[Int],
- dataset: DatasetRef,
- shardMapper: ShardMapper): Unit = {
+ private def unassignShards(shards: Seq[Int],
+ dataset: DatasetRef,
+ shardMapper: ShardMapper): Unit = {
for { shard <- shards} {
val curCoordinator = shardMapper.coordForShard(shard)
- sendUnassignmentMessagesAndEvents(dataset, curCoordinator,
- shardMapper, Seq(shard))
+ doUnassignShards(dataset, curCoordinator, Seq(shard))
}
+ publishChanges(dataset)
}
/**
@@ -302,9 +326,9 @@ private[coordinator] final class ShardManager(settings: FilodbSettings,
*
* @return - Validates and returns error message on failure and a unit if no validation error
*/
- def validateRequestAndStartShards(dataset: DatasetRef,
- assignmentConfig: AssignShardConfig,
- ackTo: ActorRef): Unit Or ErrorResponse = {
+ private def validateRequestAndStartShards(dataset: DatasetRef,
+ assignmentConfig: AssignShardConfig,
+ ackTo: ActorRef): Unit Or ErrorResponse = {
for {
shardMapper <- validateDataset(dataset)
coordinator <- validateCoordinator(assignmentConfig.address, assignmentConfig.shardList)
@@ -312,7 +336,8 @@ private[coordinator] final class ShardManager(settings: FilodbSettings,
_ <- validateNodeCapacity(shards, shardMapper, dataset,
_datasetInfo(dataset).resources, coordinator)
} yield {
- sendAssignmentMessagesAndEvents(dataset, coordinator, shards)
+ doAssignShards(dataset, coordinator, shards)
+ publishChanges(dataset)
}
}
@@ -327,23 +352,25 @@ private[coordinator] final class ShardManager(settings: FilodbSettings,
(dataset, mapper) <- shardMappers
} yield {
val allRegisteredNodes = mapper.allNodes
- allRegisteredNodes -- coordinators // coordinators is the list of recovered nodes
+ val toRemove = allRegisteredNodes -- coordinators // coordinators is the list of recovered nodes
+ logger.info(s"Cleaning up dataset=$dataset stale coordinators $toRemove after recovery")
+ toRemove
}
for { coord <- nodesToRemove.flatten } {
- logger.info(s"Cleaning up stale coordinator $coord after recovery")
removeCoordinator(coord)
}
updateShardMetrics()
+ logAllMappers("After removing stale coordinators")
}
private def removeCoordinator(coordinator: ActorRef): Unit = {
for ((dataset, resources, mapper) <- datasetShardMaps) {
var shardsToDown = mapper.shardsForCoord(coordinator)
- sendUnassignmentMessagesAndEvents(dataset, coordinator, mapper, shardsToDown)
+ doUnassignShards(dataset, coordinator, shardsToDown)
// try to reassign shards that were unassigned to other nodes that have room.
assignShardsToNodes(dataset, mapper, resources)
+ publishChanges(dataset)
}
- logger.info(s"Completed removeMember for coordinator $coordinator")
}
/**
@@ -353,7 +380,6 @@ private[coordinator] final class ShardManager(settings: FilodbSettings,
def addDataset(setup: SetupDataset,
dataset: Dataset,
ackTo: ActorRef): Map[ActorRef, Seq[Int]] = {
-
logger.info(s"Initiated setup for dataset=${setup.ref}")
val answer: Map[ActorRef, Seq[Int]] = mapperOpt(setup.ref) match {
case Some(_) =>
@@ -380,7 +406,7 @@ private[coordinator] final class ShardManager(settings: FilodbSettings,
ackTo ! DatasetVerified
assignments
}
- updateShardMetrics()
+ publishChanges(setup.ref)
answer
}
@@ -392,7 +418,7 @@ private[coordinator] final class ShardManager(settings: FilodbSettings,
coord <- latestCoords if !excludeCoords.contains(coord) // assign shards on newer nodes first
} yield {
val assignable = strategy.shardAssignments(coord, dataset, resources, mapper)
- if (assignable.nonEmpty) sendAssignmentMessagesAndEvents(dataset, coord, assignable)
+ if (assignable.nonEmpty) doAssignShards(dataset, coord, assignable)
coord -> assignable
}).toMap
}
@@ -403,7 +429,8 @@ private[coordinator] final class ShardManager(settings: FilodbSettings,
(_, coord) <- _coordinators
mapper = _shardMappers(dataset)
shardsToDown = mapper.shardsForCoord(coord)
- } sendUnassignmentMessagesAndEvents(dataset, coord, mapper, shardsToDown)
+ } doUnassignShards(dataset, coord, shardsToDown)
+ publishChanges(dataset)
_datasetInfo remove dataset
_shardMappers remove dataset
_subscriptions = _subscriptions - dataset
@@ -417,7 +444,7 @@ private[coordinator] final class ShardManager(settings: FilodbSettings,
def recoverShards(ref: DatasetRef, map: ShardMapper): Unit = {
logger.info(s"Recovering ShardMap for dataset=$ref ; ShardMap contents: $map")
_shardMappers(ref) = map
- updateShardMetrics()
+ publishChanges(ref)
}
def recoverSubscriptions(subs: ShardSubscriptions): Unit = {
@@ -436,11 +463,10 @@ private[coordinator] final class ShardManager(settings: FilodbSettings,
* the ShardMapper for dataset.
*/
def updateFromExternalShardEvent(sender: ActorRef, event: ShardEvent): Unit = {
- logger.info(s"Received event=$event from sender=$sender for dataset=${event.ref} for shard=${event.shard}")
_shardMappers.get(event.ref) foreach { mapper =>
val currentCoord = mapper.coordForShard(event.shard)
if (currentCoord == ActorRef.noSender) {
- logger.warn(s"Ignoring event=$event from sender=$sender for dataset=${event.ref} since shard=${event.shard} " +
+ logger.debug(s"Ignoring event=$event from sender=$sender for dataset=${event.ref} since shard=${event.shard} " +
s"is not currently assigned. Was $sender the previous owner for a shard that was just unassigned? " +
s"How else could this happen? ")
// Note that this path is not used for an initial shard assignment when currentCoord would indeed be noSender;
@@ -449,8 +475,6 @@ 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)
- // 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 =>
@@ -477,9 +501,10 @@ private[coordinator] final class ShardManager(settings: FilodbSettings,
}
case _ =>
}
- updateShardMetrics()
+ // RecoveryInProgress status results in too many messages that really do not need a publish
+ if (!event.isInstanceOf[RecoveryInProgress]) publishSnapshot(event.ref)
} else {
- logger.warn(s"Ignoring event $event from $sender for dataset=${event.ref} since it does not match current " +
+ logger.debug(s"Ignoring event $event from $sender for dataset=${event.ref} since it does not match current " +
s"owner of shard=${event.shard} which is ${mapper.coordForShard(event.shard)}")
}
}
@@ -504,65 +529,73 @@ private[coordinator] final class ShardManager(settings: FilodbSettings,
case Failure(l) =>
logger.error(s"updateFromShardEvent error for dataset=${event.ref} event $event. Mapper now: $mapper", l)
case Success(r) =>
- logger.info(s"updateFromShardEvent success for dataset=${event.ref} event $event. Mapper now: $mapper")
+ logger.debug(s"updateFromShardEvent success for dataset=${event.ref} event $event. Mapper now: $mapper")
}
}
updateShardMetrics()
}
- /**
- * This method has the shared logic for sending shard assignment messages
- * to the coordinator, updating state for the event and broadcast of the state change to subscribers
- */
- private def sendAssignmentMessagesAndEvents(dataset: DatasetRef,
- coord: ActorRef,
- shards: Seq[Int]): Unit = {
+ private def doAssignShards(dataset: DatasetRef,
+ coord: ActorRef,
+ shards: Seq[Int]): Unit = {
val state = _datasetInfo(dataset)
logger.info(s"Sending setup message for dataset=${state.dataset.ref} to coordinators $coord.")
val setupMsg = client.IngestionCommands.DatasetSetup(state.dataset.asCompactString,
state.storeConfig, state.source, state.downsample)
coord ! setupMsg
+ logger.info(s"Assigning shards for dataset=$dataset to " +
+ s"coordinator $coord for shards $shards")
for { shard <- shards } {
val event = ShardAssignmentStarted(dataset, shard, coord)
updateFromShardEvent(event)
}
- publishSnapshot(dataset)
- /* If no shards are assigned to a coordinator, no commands are sent. */
- logger.info(s"Sending start ingestion message for dataset=$dataset to coordinator $coord for shards $shards")
- for {shard <- shards} coord ! StartShardIngestion(dataset, shard, None)
}
- /**
- * This method has the shared logic for sending shard un-assignment messages
- * to the coordinator, updating state for the event and broadcast of the state change to subscribers
- */
- private def sendUnassignmentMessagesAndEvents(dataset: DatasetRef,
- coordinator: ActorRef,
- mapper: ShardMapper,
- shardsToDown: Seq[Int],
- nodeUp: Boolean = true): Unit = {
- logger.info(s"Sending stop ingestion message for dataset=$dataset to " +
+ private def doUnassignShards(dataset: DatasetRef,
+ coordinator: ActorRef,
+ shardsToDown: Seq[Int]): Unit = {
+ logger.info(s"Unassigning shards for dataset=$dataset to " +
s"coordinator $coordinator for shards $shardsToDown")
for { shard <- shardsToDown } {
val event = ShardDown(dataset, shard, coordinator)
updateFromShardEvent(event)
- if (nodeUp) coordinator ! StopShardIngestion(dataset, shard)
}
- publishSnapshot(dataset)
- logger.debug(s"Unassigned shards $shardsToDown of dataset=$dataset from $coordinator")
+ }
+
+ /**
+ * To be called after making a bunch of changes to the ShardMapper for the given dataset.
+ * Calling this method more often is permitted, but it generates more publish messages
+ * than is necessary.
+ */
+ private def publishChanges(ref: DatasetRef): Unit = {
+ publishSnapshot(ref)
+ updateShardMetrics()
}
/** Publishes a ShardMapper snapshot of given dataset to all subscribers of that dataset. */
- def publishSnapshot(ref: DatasetRef): Unit =
- mapperOpt(ref) match {
- case Some(snapshot) =>
+ def publishSnapshot(ref: DatasetRef): Unit = {
+ mapperCopyOpt(ref) match {
+ case Some(snapshot) => {
for {
subscription <- _subscriptions.subscription(ref)
} subscription.subscribers foreach (_ ! snapshot)
+
+ // Also send a complete ingestion state command to all ingestion actors. Without this,
+ // they won't start or stop ingestion.
+
+ // TODO: Need to provide a globally consistent version, incremented when anything
+ // changes, for any dataset.
+ val resync = ShardIngestionState(0, snapshot.ref, snapshot.map)
+
+ for (coord <- coordinators) {
+ coord ! resync
+ }
+ }
case None =>
logger.warn(s"Cannot publish snapshot which doesn't exist for ref $ref")
}
+ }
private def latestCoords: Seq[ActorRef] =
_coordinators.values.foldLeft(List[ActorRef]())((x, y) => y :: x) // reverses the set
@@ -583,4 +616,4 @@ private[coordinator] object ShardManager {
downsample: DownsampleConfig,
storeConfig: StoreConfig,
dataset: Dataset)
-}
\ No newline at end of file
+}
diff --git a/coordinator/src/main/scala/filodb.coordinator/ShardMapper.scala b/coordinator/src/main/scala/filodb.coordinator/ShardMapper.scala
index f62ef97486..0cb935ad55 100644
--- a/coordinator/src/main/scala/filodb.coordinator/ShardMapper.scala
+++ b/coordinator/src/main/scala/filodb.coordinator/ShardMapper.scala
@@ -164,6 +164,11 @@ class ShardMapper(val numShards: Int) extends Serializable {
def numAssignedShards: Int = numShards - unassignedShards.length
+ def isAnIngestionState(shard: Int): Boolean = statusMap(shard) match {
+ case ShardStatusStopped | ShardStatusDown => false
+ case _ => true
+ }
+
/**
* Find out if a shard is active (Normal or Recovery status) or filter a list of shards
*/
@@ -250,6 +255,23 @@ class ShardMapper(val numShards: Int) extends Serializable {
private[coordinator] def clear(): Unit = {
for { i <- 0 until numShards } { shardMap(i) = ActorRef.noSender }
}
+
+ /**
+ * Gives a pretty grid-view summary of the status of each shard, plus a sorted view of shards owned by each
+ * coordinator.
+ */
+ def prettyPrint: String = {
+ val sortedCoords = allNodes.toSeq.sorted
+ "Status legend: U=Unassigned N=Assigned A=Active E=Error R=Recovery S=Stopped D=Down\n----- Status Map-----\n" +
+ statusMap.toSeq.grouped(16).zipWithIndex.map { case (statGroup, i) =>
+ f" ${i * 16}%4d-${Math.min(i * 16 + 15, numShards)}%4d " +
+ statGroup.grouped(8).map(_.map(statusToLetter).mkString("")).mkString(" ")
+ }.mkString("\n") +
+ "\n----- Coordinators -----\n" +
+ sortedCoords.map { coord =>
+ f" $coord%40s\t${shardsForCoord(coord).mkString(", ")}"
+ }.mkString("\n")
+ }
}
private[filodb] object ShardMapper extends StrictLogging {
@@ -271,4 +293,14 @@ private[filodb] object ShardMapper extends StrictLogging {
final case class ShardError(event: ShardEvent, context: String)
extends Exception(s"$context [shard=${event.shard}, event=$event]")
+
+ def statusToLetter(status: ShardStatus): String = status match {
+ case ShardStatusUnassigned => "."
+ case ShardStatusAssigned => "N"
+ case ShardStatusActive => "A"
+ case ShardStatusError => "E"
+ case s: ShardStatusRecovery => "R"
+ case ShardStatusStopped => "S"
+ case ShardStatusDown => "D"
+ }
}
diff --git a/coordinator/src/main/scala/filodb.coordinator/ShardStatus.scala b/coordinator/src/main/scala/filodb.coordinator/ShardStatus.scala
index cf3375cffb..77814a91be 100644
--- a/coordinator/src/main/scala/filodb.coordinator/ShardStatus.scala
+++ b/coordinator/src/main/scala/filodb.coordinator/ShardStatus.scala
@@ -12,30 +12,12 @@ final case class CurrentShardSnapshot(ref: DatasetRef,
map: ShardMapper) extends ShardAction with Response
/**
- * These commands are sent by the NodeClusterActor to the right nodes upon events or
- * changes to the cluster. For example a new node joins, StartShardIngestion might be sent.
- * Should start with a verb, since these are commands.
+ * Full state of all shards, sent to all ingestion actors. They react by starting/stopping
+ * ingestion for the shards they own or no longer own. The version is expected to be global
+ * and monotonically increasing, but if the version is 0, then the actor should skip the
+ * version check and blindly apply the resync action.
*/
-sealed trait ShardCommand extends ShardAction {
- def ref: DatasetRef
- def shard: Int
-}
-
-final case class StartShardIngestion(ref: DatasetRef, shard: Int, offset: Option[Long]) extends ShardCommand
-
-final case class StopShardIngestion(ref: DatasetRef, shard: Int) extends ShardCommand
-
-/** Direct result of sending an invalid [[ShardCommand]]. It is acked to the
- * sender if the shard command's shard or dataset is not valid based on the
- * projection or shard state. It is located with the shard commands because
- * this is not a potential result of an Ingestion command and flows through
- * a node's coordinator, one of its ingesters, the cluster shard actor and
- * its [[filodb.coordinator.ShardAssignmentStrategy]].
- *
- * Use cases: result of invalid state and injecting failure to the memstore
- * during stream creation in the [[StartShardIngestion]] protocol.
- */
-final case class InvalidIngestionCommand(ref: DatasetRef, shard: Int)
+final case class ShardIngestionState(version: Long, ref: DatasetRef, map: ShardMapper) extends ShardAction
/**
* The events are sent by the IngestionActor on a node when the actual ingestion stream
diff --git a/coordinator/src/main/scala/filodb.coordinator/client/QueryCommands.scala b/coordinator/src/main/scala/filodb.coordinator/client/QueryCommands.scala
index 1628c61d97..bbf52167c2 100644
--- a/coordinator/src/main/scala/filodb.coordinator/client/QueryCommands.scala
+++ b/coordinator/src/main/scala/filodb.coordinator/client/QueryCommands.scala
@@ -45,6 +45,8 @@ object QueryCommands {
}
}
+ case class SpreadAssignment(shardKeysMap: collection.Map[String, String], spread: Int)
+
/**
* Serialize with care! would be based on the provided function.
* @param f a function that would act as the spread provider
@@ -60,37 +62,42 @@ object QueryCommands {
* This class provides general query processing parameters
* @param spreadFunc a function that returns chronologically ordered spread changes for the filter
*/
- final case class QueryOptions(spreadProvider: SpreadProvider = StaticSpreadProvider(),
+ final case class QueryOptions(spreadProvider: Option[SpreadProvider] = None,
parallelism: Int = 16,
queryTimeoutSecs: Int = 30,
sampleLimit: Int = 1000000,
shardOverrides: Option[Seq[Int]] = None)
object QueryOptions {
- def apply(constSpread: Int, sampleLimit: Int): QueryOptions =
- QueryOptions(spreadProvider = StaticSpreadProvider(SpreadChange(0, constSpread)), sampleLimit = sampleLimit)
+ def apply(constSpread: Option[SpreadProvider], sampleLimit: Int): QueryOptions =
+ QueryOptions(spreadProvider = constSpread, sampleLimit = sampleLimit)
/**
* Creates a spreadFunc that looks for a particular filter with keyName Equals a value, and then maps values
* present in the spreadMap to specific spread values, with a default if the filter/value not present in the map
*/
def simpleMapSpreadFunc(keyName: String,
- spreadMap: collection.Map[String, Int],
+ spreadMap: collection.mutable.Map[collection.Map[String, String], Int],
defaultSpread: Int): Seq[ColumnFilter] => Seq[SpreadChange] = {
filters: Seq[ColumnFilter] =>
filters.collectFirst {
case ColumnFilter(key, Filter.Equals(filtVal: String)) if key == keyName => filtVal
}.map { tagValue =>
- Seq(SpreadChange(spread = spreadMap.getOrElse(tagValue, defaultSpread)))
+ Seq(SpreadChange(spread = spreadMap.getOrElse(collection.mutable.Map(keyName->tagValue), defaultSpread)))
}.getOrElse(Seq(SpreadChange(defaultSpread)))
}
import collection.JavaConverters._
def simpleMapSpreadFunc(keyName: String,
- spreadMap: java.util.Map[String, Int],
- defaultSpread: Int): Seq[ColumnFilter] => Seq[SpreadChange] =
- simpleMapSpreadFunc(keyName, spreadMap.asScala, defaultSpread)
+ spreadMap: java.util.Map[java.util.Map[String, String], Integer],
+ defaultSpread: Int): Seq[ColumnFilter] => Seq[SpreadChange] = {
+ val spreadAssignment: collection.mutable.Map[collection.Map[String, String], Int]= spreadMap.asScala.map {
+ case (d, v) => d.asScala -> v.toInt
+ }
+
+ simpleMapSpreadFunc(keyName, spreadAssignment, defaultSpread)
+ }
}
/**
@@ -107,6 +114,10 @@ object QueryCommands {
queryOptions: QueryOptions = QueryOptions(),
submitTime: Long = System.currentTimeMillis()) extends QueryCommand
+ final case class ExplainPlan2Query(dataset: DatasetRef,
+ logicalPlan: LogicalPlan2,
+ queryOptions: QueryOptions = QueryOptions(),
+ submitTime: Long = System.currentTimeMillis()) extends QueryCommand
// Error responses from query
final case class UndefinedColumns(undefined: Set[String]) extends ErrorResponse
final case class BadArgument(msg: String) extends ErrorResponse with QueryResponse
diff --git a/coordinator/src/main/scala/filodb.coordinator/client/Serializer.scala b/coordinator/src/main/scala/filodb.coordinator/client/Serializer.scala
index 07c10d4ec7..530bf4e7c0 100644
--- a/coordinator/src/main/scala/filodb.coordinator/client/Serializer.scala
+++ b/coordinator/src/main/scala/filodb.coordinator/client/Serializer.scala
@@ -30,6 +30,10 @@ import filodb.memory.format.ZeroCopyUTF8String
*/
class KryoInit {
def customize(kryo: Kryo): Unit = {
+ // Default level used by Kryo is 'trace', which is expensive. It always builds the message,
+ // even if it gets filtered out by the logging framework.
+ com.esotericsoftware.minlog.Log.WARN()
+
kryo.addDefaultSerializer(classOf[Column.ColumnType], classOf[ColumnTypeSerializer])
val colTypeSer = new ColumnTypeSerializer
Column.ColumnType.values.zipWithIndex.foreach { case (ct, i) => kryo.register(ct.getClass, colTypeSer, 100 + i) }
diff --git a/coordinator/src/main/scala/filodb.coordinator/queryengine/Utils.scala b/coordinator/src/main/scala/filodb.coordinator/queryengine/Utils.scala
index bb48d1ef92..5687cde7e3 100644
--- a/coordinator/src/main/scala/filodb.coordinator/queryengine/Utils.scala
+++ b/coordinator/src/main/scala/filodb.coordinator/queryengine/Utils.scala
@@ -28,6 +28,7 @@ final case class ChildErrorResponse(source: ActorRef, resp: ErrorResponse) exten
object Utils extends StrictLogging {
import filodb.coordinator.client.QueryCommands._
import TrySugar._
+ import filodb.coordinator.client.QueryCommands._
/**
* Convert column name strings into columnIDs. NOTE: column names should not include row key columns
@@ -49,7 +50,8 @@ object Utils extends StrictLogging {
*/
def validatePartQuery(dataset: Dataset, shardMap: ShardMapper,
partQuery: PartitionQuery,
- options: QueryOptions): Seq[PartitionScanMethod] Or ErrorResponse =
+ options: QueryOptions, spreadProvider: SpreadProvider):
+ Seq[PartitionScanMethod] Or ErrorResponse =
Try(partQuery match {
case SinglePartitionQuery(keyParts) =>
val partKey = dataset.partKey(keyParts: _*)
@@ -75,7 +77,7 @@ object Utils extends StrictLogging {
if (shardCols.length > 0) {
shardHashFromFilters(filters, shardCols, dataset) match {
case Some(shardHash) => shardMap.queryShards(shardHash,
- options.spreadProvider.spreadFunc(filters).last.spread)
+ spreadProvider.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 72d5e54bfa..ba0743be6a 100644
--- a/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala
+++ b/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala
@@ -12,7 +12,7 @@ import kamon.Kamon
import monix.eval.Task
import filodb.coordinator.ShardMapper
-import filodb.coordinator.client.QueryCommands.QueryOptions
+import filodb.coordinator.client.QueryCommands.{QueryOptions, SpreadProvider, StaticSpreadProvider}
import filodb.core.Types
import filodb.core.binaryrecord2.RecordBuilder
import filodb.core.metadata.Dataset
@@ -58,16 +58,19 @@ class QueryEngine(dataset: Dataset,
* Converts a LogicalPlan to the ExecPlan
*/
def materialize(rootLogicalPlan: LogicalPlan,
- options: QueryOptions): ExecPlan = {
+ options: QueryOptions, spreadProvider: SpreadProvider = StaticSpreadProvider()): ExecPlan = {
val queryId = UUID.randomUUID().toString
- val materialized = walkLogicalPlanTree(rootLogicalPlan, queryId, System.currentTimeMillis(), options) match {
+
+ val materialized = walkLogicalPlanTree(rootLogicalPlan, queryId, System.currentTimeMillis(),
+ options, spreadProvider)
+ match {
case PlanResult(Seq(justOne), stitch) =>
if (stitch) justOne.addRangeVectorTransformer(new StitchRvsMapper())
justOne
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 =>
val topPlan = DistConcatExec(queryId, targetActor, many)
@@ -83,7 +86,7 @@ class QueryEngine(dataset: Dataset,
val shardColumns = dataset.options.shardKeyColumns.sorted
private def shardsFromFilters(filters: Seq[ColumnFilter],
- options: QueryOptions): Seq[Int] = {
+ options: QueryOptions, spreadProvider : SpreadProvider): Seq[Int] = {
require(shardColumns.nonEmpty || options.shardOverrides.nonEmpty,
s"Dataset ${dataset.ref} does not have shard columns defined, and shard overrides were not mentioned")
@@ -107,13 +110,16 @@ class QueryEngine(dataset: Dataset,
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.spreadProvider.spreadFunc(filters).last.spread)
+ shardMapperFunc.queryShards(shardHash, spreadProvider.spreadFunc(filters).last.spread)
}
}
private def dispatcherForShard(shard: Int): PlanDispatcher = {
val targetActor = shardMapperFunc.coordForShard(shard)
- if (targetActor == ActorRef.noSender) throw new RuntimeException("Not all shards available") // TODO fix this
+ if (targetActor == ActorRef.noSender) {
+ logger.debug(s"ShardMapper: $shardMapperFunc")
+ throw new RuntimeException(s"Shard: $shard is not available") // TODO fix this
+ }
ActorPlanDispatcher(targetActor)
}
@@ -125,27 +131,35 @@ class QueryEngine(dataset: Dataset,
private def walkLogicalPlanTree(logicalPlan: LogicalPlan,
queryId: String,
submitTime: Long,
- options: QueryOptions): PlanResult = {
+ options: QueryOptions, spreadProvider: SpreadProvider): PlanResult = {
+
logicalPlan match {
- case lp: RawSeries => materializeRawSeries(queryId, submitTime, options, lp)
- case lp: RawChunkMeta => materializeRawChunkMeta(queryId, submitTime, options, lp)
- case lp: PeriodicSeries => materializePeriodicSeries(queryId, submitTime, options, lp)
- case lp: PeriodicSeriesWithWindowing => materializePeriodicSeriesWithWindowing(queryId, submitTime, options, lp)
- case lp: ApplyInstantFunction => materializeApplyInstantFunction(queryId, submitTime, options, lp)
- case lp: Aggregate => materializeAggregate(queryId, submitTime, options, lp)
- case lp: BinaryJoin => materializeBinaryJoin(queryId, submitTime, options, lp)
- 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)
+ case lp: RawSeries => materializeRawSeries(queryId, submitTime, options, lp, spreadProvider)
+ case lp: RawChunkMeta => materializeRawChunkMeta(queryId, submitTime, options, lp, spreadProvider)
+ case lp: PeriodicSeries => materializePeriodicSeries(queryId, submitTime, options, lp,
+ spreadProvider)
+ case lp: PeriodicSeriesWithWindowing => materializePeriodicSeriesWithWindowing(queryId, submitTime, options, lp,
+ spreadProvider)
+ case lp: ApplyInstantFunction => materializeApplyInstantFunction(queryId, submitTime, options, lp,
+ spreadProvider)
+ case lp: Aggregate => materializeAggregate(queryId, submitTime, options, lp, spreadProvider)
+ case lp: BinaryJoin => materializeBinaryJoin(queryId, submitTime, options, lp, spreadProvider)
+ case lp: ScalarVectorBinaryOperation => materializeScalarVectorBinOp(queryId, submitTime, options, lp,
+ spreadProvider)
+ case lp: LabelValues => materializeLabelValues(queryId, submitTime, options, lp, spreadProvider)
+ case lp: SeriesKeysByFilters => materializeSeriesKeysByFilters(queryId, submitTime, options, lp,
+ spreadProvider)
+ case lp: ApplyMiscellaneousFunction => materializeApplyMiscellaneousFunction(queryId, submitTime, options, lp,
+ spreadProvider)
}
}
private def materializeScalarVectorBinOp(queryId: String,
submitTime: Long,
options: QueryOptions,
- lp: ScalarVectorBinaryOperation): PlanResult = {
- val vectors = walkLogicalPlanTree(lp.vector, queryId, submitTime, options)
+ lp: ScalarVectorBinaryOperation,
+ spreadProvider : SpreadProvider): PlanResult = {
+ val vectors = walkLogicalPlanTree(lp.vector, queryId, submitTime, options, spreadProvider)
vectors.plans.foreach(_.addRangeVectorTransformer(ScalarOperationMapper(lp.operator, lp.scalar, lp.scalarIsLhs)))
vectors
}
@@ -153,11 +167,11 @@ class QueryEngine(dataset: Dataset,
private def materializeBinaryJoin(queryId: String,
submitTime: Long,
options: QueryOptions,
- lp: BinaryJoin): PlanResult = {
- val lhs = walkLogicalPlanTree(lp.lhs, queryId, submitTime, options)
+ lp: BinaryJoin, spreadProvider : SpreadProvider): PlanResult = {
+ val lhs = walkLogicalPlanTree(lp.lhs, queryId, submitTime, options, spreadProvider)
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 rhs = walkLogicalPlanTree(lp.rhs, queryId, submitTime, options, spreadProvider)
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.
@@ -175,8 +189,8 @@ class QueryEngine(dataset: Dataset,
private def materializeAggregate(queryId: String,
submitTime: Long,
options: QueryOptions,
- lp: Aggregate): PlanResult = {
- val toReduce = walkLogicalPlanTree(lp.vectors, queryId, submitTime, options)
+ lp: Aggregate, spreadProvider : SpreadProvider): PlanResult = {
+ val toReduce = walkLogicalPlanTree(lp.vectors, queryId, submitTime, options, spreadProvider )
// 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,
@@ -200,8 +214,8 @@ class QueryEngine(dataset: Dataset,
private def materializeApplyInstantFunction(queryId: String,
submitTime: Long,
options: QueryOptions,
- lp: ApplyInstantFunction): PlanResult = {
- val vectors = walkLogicalPlanTree(lp.vectors, queryId, submitTime, options)
+ lp: ApplyInstantFunction, spreadProvider : SpreadProvider): PlanResult = {
+ val vectors = walkLogicalPlanTree(lp.vectors, queryId, submitTime, options, spreadProvider)
vectors.plans.foreach(_.addRangeVectorTransformer(InstantVectorFunctionMapper(lp.function, lp.functionArgs)))
vectors
}
@@ -209,8 +223,9 @@ class QueryEngine(dataset: Dataset,
private def materializePeriodicSeriesWithWindowing(queryId: String,
submitTime: Long,
options: QueryOptions,
- lp: PeriodicSeriesWithWindowing): PlanResult ={
- val rawSeries = walkLogicalPlanTree(lp.rawSeries, queryId, submitTime, options)
+ lp: PeriodicSeriesWithWindowing,
+ spreadProvider: SpreadProvider): PlanResult = {
+ val rawSeries = walkLogicalPlanTree(lp.rawSeries, queryId, submitTime, options, spreadProvider)
rawSeries.plans.foreach(_.addRangeVectorTransformer(PeriodicSamplesMapper(lp.start, lp.step,
lp.end, Some(lp.window), Some(lp.function), lp.functionArgs)))
rawSeries
@@ -219,8 +234,8 @@ class QueryEngine(dataset: Dataset,
private def materializePeriodicSeries(queryId: String,
submitTime: Long,
options: QueryOptions,
- lp: PeriodicSeries): PlanResult = {
- val rawSeries = walkLogicalPlanTree(lp.rawSeries, queryId, submitTime, options)
+ lp: PeriodicSeries, spreadProvider : SpreadProvider): PlanResult = {
+ val rawSeries = walkLogicalPlanTree(lp.rawSeries, queryId, submitTime, options, spreadProvider)
rawSeries.plans.foreach(_.addRangeVectorTransformer(PeriodicSamplesMapper(lp.start, lp.step, lp.end,
None, None, Nil)))
rawSeries
@@ -229,15 +244,15 @@ class QueryEngine(dataset: Dataset,
private def materializeRawSeries(queryId: String,
submitTime: Long,
options: QueryOptions,
- lp: RawSeries): PlanResult = {
+ lp: RawSeries, spreadProvider : SpreadProvider): PlanResult = {
val colIDs = getColumnIDs(dataset, lp.columns)
val renamedFilters = renameMetricFilter(lp.filters)
- val spreadChanges = options.spreadProvider.spreadFunc(renamedFilters)
+ val spreadChanges = spreadProvider.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 execPlans = shardsFromFilters(renamedFilters, options, spreadProvider).map { shard =>
val dispatcher = dispatcherForShard(shard)
SelectRawPartitionsExec(queryId, submitTime, options.sampleLimit, dispatcher, dataset.ref, shard,
renamedFilters, toChunkScanMethod(lp.rangeSelector), colIDs)
@@ -248,7 +263,7 @@ class QueryEngine(dataset: Dataset,
private def materializeLabelValues(queryId: String,
submitTime: Long,
options: QueryOptions,
- lp: LabelValues): PlanResult = {
+ lp: LabelValues, spreadProvider : SpreadProvider): PlanResult = {
val filters = lp.labelConstraints.map { case (k, v) =>
new ColumnFilter(k, Filter.Equals(v))
}.toSeq
@@ -259,7 +274,7 @@ class QueryEngine(dataset: Dataset,
lp.labelNames.updated(metricLabelIndex, dataset.options.metricColumn) else lp.labelNames
val shardsToHit = if (shardColumns.toSet.subsetOf(lp.labelConstraints.keySet)) {
- shardsFromFilters(filters, options)
+ shardsFromFilters(filters, options, spreadProvider)
} else {
mdNoShardKeyFilterRequests.increment()
shardMapperFunc.assignedShards
@@ -275,11 +290,11 @@ class QueryEngine(dataset: Dataset,
private def materializeSeriesKeysByFilters(queryId: String,
submitTime: Long,
options: QueryOptions,
- lp: SeriesKeysByFilters): PlanResult = {
+ lp: SeriesKeysByFilters, spreadProvider : SpreadProvider): PlanResult = {
val renamedFilters = renameMetricFilter(lp.filters)
val filterCols = lp.filters.map(_.column).toSet
val shardsToHit = if (shardColumns.toSet.subsetOf(filterCols)) {
- shardsFromFilters(lp.filters, options)
+ shardsFromFilters(lp.filters, options, spreadProvider)
} else {
mdNoShardKeyFilterRequests.increment()
shardMapperFunc.assignedShards
@@ -295,12 +310,12 @@ class QueryEngine(dataset: Dataset,
private def materializeRawChunkMeta(queryId: String,
submitTime: Long,
options: QueryOptions,
- lp: RawChunkMeta): PlanResult = {
+ lp: RawChunkMeta, spreadProvider : SpreadProvider): 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)
- val metaExec = shardsFromFilters(renamedFilters, options).map { shard =>
+ val metaExec = shardsFromFilters(renamedFilters, options, spreadProvider).map { shard =>
val dispatcher = dispatcherForShard(shard)
SelectChunkInfosExec(queryId, submitTime, options.sampleLimit, dispatcher, dataset.ref, shard,
renamedFilters, toChunkScanMethod(lp.rangeSelector), colID)
@@ -309,10 +324,11 @@ class QueryEngine(dataset: Dataset,
}
private def materializeApplyMiscellaneousFunction(queryId: String,
- submitTime: Long,
- options: QueryOptions,
- lp: ApplyMiscellaneousFunction): PlanResult = {
- val vectors = walkLogicalPlanTree(lp.vectors, queryId, submitTime, options)
+ submitTime: Long,
+ options: QueryOptions,
+ lp: ApplyMiscellaneousFunction,
+ spreadProvider: SpreadProvider): PlanResult = {
+ val vectors = walkLogicalPlanTree(lp.vectors, queryId, submitTime, options, spreadProvider)
vectors.plans.foreach(_.addRangeVectorTransformer(MiscellaneousFunctionMapper(lp.function, lp.functionArgs)))
vectors
}
diff --git a/coordinator/src/test/scala/filodb.coordinator/IngestionStreamSpec.scala b/coordinator/src/test/scala/filodb.coordinator/IngestionStreamSpec.scala
index d4385e6c8f..a867e6f23e 100644
--- a/coordinator/src/test/scala/filodb.coordinator/IngestionStreamSpec.scala
+++ b/coordinator/src/test/scala/filodb.coordinator/IngestionStreamSpec.scala
@@ -118,9 +118,6 @@ class IngestionStreamSpec extends ActorTest(IngestionStreamSpec.getNewSystem) wi
"if incorrect shard is sent for the creation of the stream") {
setup(dataset6.ref, "/GDELT-sample-test.csv", rowsToRead = 5, None)
- val invalidShard = -1
- coordinatorActor ! StartShardIngestion(dataset6.ref, invalidShard, None)
-
// first it becomes active
expectMsgPF(within) {
case CurrentShardSnapshot(dataset6.ref, mapper) =>
@@ -193,6 +190,7 @@ 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
+ /* RecoveryInProgress events don't publish snapshots because of the high message load.
for { i <- 0 until 3} {
expectMsgPF(within) {
case CurrentShardSnapshot(dataset33.ref, mapper) =>
@@ -200,6 +198,7 @@ class IngestionStreamSpec extends ActorTest(IngestionStreamSpec.getNewSystem) wi
mapper.statuses.head.isInstanceOf[ShardStatusRecovery] shouldEqual true
}
}
+ */
expectMsgPF(within) {
case CurrentShardSnapshot(dataset33.ref, mapper) =>
@@ -217,4 +216,4 @@ class IngestionStreamSpec extends ActorTest(IngestionStreamSpec.getNewSystem) wi
coordinatorActor ! GetIngestionStats(dataset33.ref)
expectMsg(IngestionActor.IngestionStatus(85)) // <-- must be rounded to 5, we ingest entire batches
}
-}
\ No newline at end of file
+}
diff --git a/coordinator/src/test/scala/filodb.coordinator/NodeCoordinatorActorSpec.scala b/coordinator/src/test/scala/filodb.coordinator/NodeCoordinatorActorSpec.scala
index 221d1bb5b9..5138883380 100644
--- a/coordinator/src/test/scala/filodb.coordinator/NodeCoordinatorActorSpec.scala
+++ b/coordinator/src/test/scala/filodb.coordinator/NodeCoordinatorActorSpec.scala
@@ -208,7 +208,7 @@ class NodeCoordinatorActorSpec extends ActorTest(NodeCoordinatorActorSpec.getNew
probe.send(coordinatorActor, q2)
probe.expectMsgPF() {
case QueryResult(_, schema, vectors) =>
- schema shouldEqual timeMinSchema
+ schema shouldEqual timeMinSchema.copy(fixedVectorLen = Some(2))
vectors should have length (1)
vectors(0).rows.map(_.getDouble(1)).toSeq shouldEqual Seq(14.0, 24.0)
}
@@ -221,7 +221,7 @@ class NodeCoordinatorActorSpec extends ActorTest(NodeCoordinatorActorSpec.getNew
probe.send(coordinatorActor, q3)
probe.expectMsgPF() {
case QueryResult(_, schema, vectors) =>
- schema shouldEqual countSchema
+ schema shouldEqual countSchema.copy(fixedVectorLen = Some(2))
vectors should have length (1)
vectors(0).rows.map(_.getDouble(1)).toSeq shouldEqual Seq(98.0, 108.0)
}
@@ -235,7 +235,7 @@ class NodeCoordinatorActorSpec extends ActorTest(NodeCoordinatorActorSpec.getNew
probe.send(coordinatorActor, q4)
probe.expectMsgPF() {
case QueryResult(_, schema, vectors) =>
- schema shouldEqual timeMinSchema
+ schema shouldEqual timeMinSchema.copy(fixedVectorLen = Some(2))
vectors should have length (0)
}
}
@@ -260,7 +260,7 @@ class NodeCoordinatorActorSpec extends ActorTest(NodeCoordinatorActorSpec.getNew
(0 until numQueries).foreach { _ =>
probe.expectMsgPF() {
case QueryResult(_, schema, vectors) =>
- schema shouldEqual timeMinSchema
+ schema shouldEqual timeMinSchema.copy(fixedVectorLen = Some(2))
vectors should have length (1)
vectors(0).rows.map(_.getDouble(1)).toSeq shouldEqual Seq(14.0, 24.0)
}
@@ -387,7 +387,7 @@ class NodeCoordinatorActorSpec extends ActorTest(NodeCoordinatorActorSpec.getNew
probe.expectMsgPF() {
case QueryResult(_, schema, vectors) =>
schema shouldEqual ResultSchema(Seq(ColumnInfo("GLOBALEVENTID", LongColumn),
- ColumnInfo("AvgTone", DoubleColumn)), 1)
+ ColumnInfo("AvgTone", DoubleColumn)), 1, fixedVectorLen = Some(10))
vectors should have length (1)
// vectors(0).rows.map(_.getDouble(1)).toSeq shouldEqual Seq(575.24)
// TODO: verify if the expected results are right. They are something....
diff --git a/coordinator/src/test/scala/filodb.coordinator/ReassignShardsSpec.scala b/coordinator/src/test/scala/filodb.coordinator/ReassignShardsSpec.scala
index 12efb0c965..8e4f2b05fd 100644
--- a/coordinator/src/test/scala/filodb.coordinator/ReassignShardsSpec.scala
+++ b/coordinator/src/test/scala/filodb.coordinator/ReassignShardsSpec.scala
@@ -1,5 +1,7 @@
package filodb.coordinator
+import scala.concurrent.duration._
+
import akka.actor.{ActorRef, Address}
import akka.testkit.TestProbe
import com.typesafe.config.ConfigFactory
@@ -23,26 +25,45 @@ class ReassignShardsSpec extends AkkaSpec {
val settings = new FilodbSettings(ConfigFactory.load("application_test.conf"))
protected val shardManager = new ShardManager(settings, DefaultShardAssignmentStrategy)
- val coord1 = TestProbe("coordinator1")
+ private def makeTestProbe(name: String): TestProbe = {
+ val tp = TestProbe(name)
+ // Uncomment to ignore messages of a specific type.
+ //tp.ignoreMsg({case m: Any => m.isInstanceOf[...]})
+ tp
+ }
+
+ val coord1 = makeTestProbe("coordinator1")
val coord1Address = uniqueAddress(coord1.ref)
- val coord2 = TestProbe("coordinator2")
+ val coord2 = makeTestProbe("coordinator2")
val coord2Address = uniqueAddress(coord2.ref)
- val coord3 = TestProbe("coordinator3")
+ val coord3 = makeTestProbe("coordinator3")
val coord3Address = uniqueAddress(coord3.ref)
- val coord4 = TestProbe("coordinator4")
+ val coord4 = makeTestProbe("coordinator4")
val coord4Address = uniqueAddress(coord4.ref)
- val coordInvalid = TestProbe("coordinatorInvalid")
+ val coordInvalid = makeTestProbe("coordinatorInvalid")
val coordInvalidAddress = uniqueAddress(coordInvalid.ref)
- val subscriber = TestProbe("subscriber")
+ val subscriber = makeTestProbe("subscriber")
val noOpSource1 = IngestionSource(classOf[NoOpStreamFactory].getName)
val setupDs1 = SetupDataset(dataset1, resources1, noOpSource1, TestData.storeConf)
+ private def expectDataset(coord: TestProbe, dataset: Dataset): TestProbe = {
+ coord.expectMsgPF() { case ds: DatasetSetup =>
+ ds.compactDatasetStr shouldEqual dataset.asCompactString
+ ds.source shouldEqual noOpSource1
+ }
+ coord
+ }
+
+ private def expectNoMessage(coord: TestProbe): Unit = {
+ coord.expectNoMessage(100.milliseconds)
+ }
+
def uniqueAddress(probe: ActorRef): Address =
probe.path.address.copy(system = s"${probe.path.address.system}-${probe.path.name}")
@@ -52,17 +73,17 @@ class ReassignShardsSpec extends AkkaSpec {
"fail with no datasets" in {
shardManager.subscribeAll(subscriber.ref)
subscriber.expectMsg(ShardSubscriptions(Set.empty, Set(subscriber.ref)))
- subscriber.expectNoMessage() // should not get a CurrentShardSnapshot since there isnt a dataset yet
+ expectNoMessage(subscriber) // should not get a CurrentShardSnapshot since there isnt a dataset yet
shardManager.addMember(coord3Address, coord3.ref)
shardManager.coordinators shouldBe Seq(coord3.ref)
shardManager.datasetInfo.size shouldBe 0
- coord3.expectNoMessage() // since there are no datasets, there should be no assignments
+ expectNoMessage(coord3) // since there are no datasets, there should be no assignments
shardManager.addMember(coord4Address, coord4.ref)
shardManager.coordinators shouldBe Seq(coord3.ref, coord4.ref)
shardManager.datasetInfo.size shouldBe 0
- coord4.expectNoMessage() // since there are no more shards left to assign
+ expectNoMessage(coord4) // since there are no more shards left to assign
val shardAssign1 = AssignShardConfig(coord1Address.toString, Seq(0,1))
shardManager.startShards(NodeClusterActor.StartShards(shardAssign1, dataset1), self)
@@ -80,50 +101,45 @@ class ReassignShardsSpec extends AkkaSpec {
shardManager.datasetInfo.size shouldBe 1
assignments shouldEqual Map(coord4.ref -> Seq(0, 1, 2), coord3.ref -> Seq(3, 4, 5))
expectMsg(DatasetVerified)
- coord4.expectMsgPF() { case ds: DatasetSetup =>
- ds.compactDatasetStr shouldEqual datasetObj1.asCompactString
- ds.source shouldEqual noOpSource1
- }
- // assignments first go to the most recently deployed node
- coord4.expectMsgAllOf(
- StartShardIngestion(dataset1, 0, None),
- StartShardIngestion(dataset1, 1, None),
- StartShardIngestion(dataset1, 2, None))
-
- coord3.expectMsgPF() { case ds: DatasetSetup =>
- ds.compactDatasetStr shouldEqual datasetObj1.asCompactString
- ds.source shouldEqual noOpSource1
- }
- // assignments first go to the most recently deployed node
- coord3.expectMsgAllOf(
- StartShardIngestion(dataset1, 3, None),
- StartShardIngestion(dataset1, 4, None),
- StartShardIngestion(dataset1, 5, None))
-
- // NOTE: because subscriptions do not kick in right away, we don't get new snapshots unitl after
- // ShardSubscriptions message
+
+ for (coord <- Seq(coord3, coord4)) {
+ expectDataset(coord, datasetObj1).expectMsgPF() { case s: ShardIngestionState =>
+ s.ref shouldEqual dataset1
+ s.map.shardsForCoord(coord4.ref) shouldEqual Seq(0, 1, 2)
+ s.map.shardsForCoord(coord3.ref) shouldEqual Seq(3, 4, 5)
+ }
+ expectNoMessage(coord)
+ }
+
+ // NOTE: because subscriptions do not kick in right away, we don't get new snapshots until
+ // after ShardSubscriptions message
+
subscriber.expectMsg(ShardSubscriptions(Set(
ShardSubscription(dataset1, Set(subscriber.ref))), Set(subscriber.ref)))
- subscriber.expectMsgPF() { case s: CurrentShardSnapshot =>
- s.ref shouldEqual dataset1
- s.map.shardsForCoord(coord4.ref) shouldEqual Seq(0, 1, 2)
- s.map.shardsForCoord(coord3.ref) shouldEqual Seq(3, 4, 5)
-// s.map.shardsForCoord(coord2.ref) shouldEqual Seq(6, 7)
-// s.map.shardsForCoord(coord1.ref) shouldEqual Seq()
+
+ for (i <- 1 to 2) {
+ // First is the initial set, the second is generated along with the state.
+ subscriber.expectMsgPF() { case s: CurrentShardSnapshot =>
+ s.ref shouldEqual dataset1
+ s.map.shardsForCoord(coord4.ref) shouldEqual Seq(0, 1, 2)
+ s.map.shardsForCoord(coord3.ref) shouldEqual Seq(3, 4, 5)
+ s.map.shardsForCoord(coord2.ref) shouldEqual Nil
+ s.map.shardsForCoord(coord1.ref) shouldEqual Nil
+ }
}
- subscriber.expectNoMessage()
+ expectNoMessage(subscriber)
val shardAssign1 = AssignShardConfig(coord4Address.toString, Seq(5))
shardManager.startShards(NodeClusterActor.StartShards(shardAssign1, dataset1), self)
expectMsgPF() { case s: BadSchema =>
s.message should startWith(s"Can not start")
}
- subscriber.expectNoMessage()
+ expectNoMessage(subscriber)
val shardAssign2 = AssignShardConfig(coord2Address.toString, Seq(0))
shardManager.startShards(NodeClusterActor.StartShards(shardAssign2, dataset1), self)
expectMsg(BadData(s"${coord2Address.toString} not found"))
- subscriber.expectNoMessage()
+ expectNoMessage(subscriber)
}
"fail with invalid node" in {
@@ -138,13 +154,23 @@ class ReassignShardsSpec extends AkkaSpec {
shardManager.coordinators shouldBe Seq(coord3.ref, coord4.ref, coord2.ref)
shardManager.datasetInfo.size shouldBe 1
- coord2.expectMsgPF() { case ds: DatasetSetup =>
- ds.compactDatasetStr shouldEqual datasetObj1.asCompactString
- ds.source shouldEqual noOpSource1
+ expectDataset(coord2, datasetObj1)
+
+ for (coord <- Seq(coord2, coord3, coord4)) {
+ coord.expectMsgPF() { case s: ShardIngestionState =>
+ s.ref shouldEqual dataset1
+ s.map.shardsForCoord(coord4.ref) shouldEqual Seq(0, 1, 2)
+ s.map.shardsForCoord(coord3.ref) shouldEqual Seq(3, 4, 5)
+ s.map.shardsForCoord(coord2.ref) shouldEqual Seq(6, 7)
+ }
+ expectNoMessage(coord)
+ }
+
+ subscriber.expectMsgPF() { case s: CurrentShardSnapshot =>
+ s.map.shardsForCoord(coord4.ref) shouldEqual Seq(0, 1, 2)
+ s.map.shardsForCoord(coord3.ref) shouldEqual Seq(3, 4, 5)
+ s.map.shardsForCoord(coord2.ref) shouldEqual Seq(6, 7)
}
- coord2.expectMsgAllOf(
- StartShardIngestion(dataset1, 6, None),
- StartShardIngestion(dataset1, 7, None))
val assignments = shardManager.shardMappers(dataset1).shardValues
assignments shouldEqual Array((coord4.ref, ShardStatusAssigned), (coord4.ref, ShardStatusAssigned),
@@ -154,14 +180,45 @@ class ReassignShardsSpec extends AkkaSpec {
val shardAssign1 = AssignShardConfig(coord2Address.toString, Seq(5))
shardManager.stopShards(NodeClusterActor.StopShards(UnassignShardConfig(shardAssign1.shardList), dataset1), self)
expectMsg(Success)
+
+ for (coord <- Seq(coord2, coord3, coord4)) {
+ coord.expectMsgPF() { case s: ShardIngestionState =>
+ s.ref shouldEqual dataset1
+ s.map.shardsForCoord(coord4.ref) shouldEqual Seq(0, 1, 2)
+ s.map.shardsForCoord(coord3.ref) shouldEqual Seq(3, 4)
+ s.map.shardsForCoord(coord2.ref) shouldEqual Seq(6, 7)
+ }
+ expectNoMessage(coord)
+ }
+
+ subscriber.expectMsgPF() { case s: CurrentShardSnapshot =>
+ s.map.shardsForCoord(coord4.ref) shouldEqual Seq(0, 1, 2)
+ s.map.shardsForCoord(coord3.ref) shouldEqual Seq(3, 4)
+ s.map.shardsForCoord(coord2.ref) shouldEqual Seq(6, 7)
+ }
+
shardManager.startShards(NodeClusterActor.StartShards(shardAssign1, dataset1), self)
expectMsg(Success)
+ expectDataset(coord2, datasetObj1)
+
+ for (coord <- Seq(coord2, coord3, coord4)) {
+ coord.expectMsgPF() { case s: ShardIngestionState =>
+ s.ref shouldEqual dataset1
+ s.map.shardsForCoord(coord4.ref) shouldEqual Seq(0, 1, 2)
+ s.map.shardsForCoord(coord3.ref) shouldEqual Seq(3, 4)
+ s.map.shardsForCoord(coord2.ref) shouldEqual Seq(5, 6, 7)
+ }
+ expectNoMessage(coord)
+ }
+
subscriber.expectMsgPF() { case s: CurrentShardSnapshot =>
s.map.shardsForCoord(coord4.ref) shouldEqual Seq(0, 1, 2)
s.map.shardsForCoord(coord3.ref) shouldEqual Seq(3, 4)
s.map.shardsForCoord(coord2.ref) shouldEqual Seq(5, 6, 7)
}
+
+ expectNoMessage(subscriber)
}
"not change after adding spare node" in {
@@ -169,13 +226,17 @@ class ReassignShardsSpec extends AkkaSpec {
shardManager.addMember(coord1Address, coord1.ref)
shardManager.coordinators shouldBe Seq(coord3.ref, coord4.ref, coord2.ref, coord1.ref)
shardManager.datasetInfo.size shouldBe 1
- coord1.expectNoMessage() // since there are no datasets, there should be no assignments
+
+ for (coord <- Seq(coord1, coord2, coord3, coord4)) {
+ expectNoMessage(coord) // since there are no datasets, there should be no assignments
+ }
val assignments = shardManager.shardMappers(dataset1).shardValues
assignments shouldEqual Array((coord4.ref, ShardStatusAssigned), (coord4.ref, ShardStatusAssigned),
(coord4.ref, ShardStatusAssigned), (coord3.ref, ShardStatusAssigned), (coord3.ref, ShardStatusAssigned),
(coord2.ref, ShardStatusAssigned), (coord2.ref, ShardStatusAssigned), (coord2.ref, ShardStatusAssigned))
+ expectNoMessage(subscriber)
}
"fail with invalid datasets" in {
@@ -183,11 +244,7 @@ class ReassignShardsSpec extends AkkaSpec {
shardManager.startShards(NodeClusterActor.StartShards(shardAssign, dataset2), self)
expectMsg(DatasetUnknown(dataset2))
- subscriber.expectMsgPF() { case s: CurrentShardSnapshot =>
- s.map.shardsForCoord(coord4.ref) shouldEqual Seq(0, 1, 2)
- s.map.shardsForCoord(coord3.ref) shouldEqual Seq(3, 4)
- s.map.shardsForCoord(coord2.ref) shouldEqual Seq(5, 6, 7)
- }
+ expectNoMessage(subscriber)
}
"fail with invalid shardNum" in {
@@ -196,12 +253,7 @@ class ReassignShardsSpec extends AkkaSpec {
shardManager.startShards(NodeClusterActor.StartShards(shardAssign1, dataset1), self)
expectMsg(BadSchema(s"Invalid shards found List(8). Valid shards are List()"))
- subscriber.expectMsgPF() { case s: CurrentShardSnapshot =>
- s.map.shardsForCoord(coord4.ref) shouldEqual Seq(0, 1, 2)
- s.map.shardsForCoord(coord3.ref) shouldEqual Seq(3, 4)
- s.map.shardsForCoord(coord2.ref) shouldEqual Seq(5, 6, 7)
- }
-
+ expectNoMessage(subscriber)
}
"fail when assigned to same node" in {
@@ -225,9 +277,41 @@ class ReassignShardsSpec extends AkkaSpec {
val shardAssign1 = AssignShardConfig(coord1Address.toString, Seq(2))
shardManager.stopShards(NodeClusterActor.StopShards(UnassignShardConfig(shardAssign1.shardList), dataset1), self)
expectMsg(Success)
+
+ for (coord <- Seq(coord1, coord2, coord3, coord4)) {
+ coord.expectMsgPF() { case s: ShardIngestionState =>
+ s.ref shouldEqual dataset1
+ s.map.shardsForCoord(coord4.ref) shouldEqual Seq(0, 1)
+ s.map.shardsForCoord(coord3.ref) shouldEqual Seq(3, 4)
+ s.map.shardsForCoord(coord2.ref) shouldEqual Seq(5, 6, 7)
+ s.map.shardsForCoord(coord1.ref) shouldEqual Nil
+ }
+ expectNoMessage(coord)
+ }
+
+ subscriber.expectMsgPF() { case s: CurrentShardSnapshot =>
+ s.map.shardsForCoord(coord4.ref) shouldEqual Seq(0, 1)
+ s.map.shardsForCoord(coord3.ref) shouldEqual Seq(3, 4)
+ s.map.shardsForCoord(coord2.ref) shouldEqual Seq(5, 6, 7)
+ s.map.shardsForCoord(coord1.ref) shouldEqual Nil
+ }
+
shardManager.startShards(NodeClusterActor.StartShards(shardAssign1, dataset1), self)
expectMsg(Success)
+ expectDataset(coord1, datasetObj1)
+
+ for (coord <- Seq(coord1, coord2, coord3, coord4)) {
+ coord.expectMsgPF() { case s: ShardIngestionState =>
+ s.ref shouldEqual dataset1
+ s.map.shardsForCoord(coord4.ref) shouldEqual Seq(0, 1)
+ s.map.shardsForCoord(coord3.ref) shouldEqual Seq(3, 4)
+ s.map.shardsForCoord(coord2.ref) shouldEqual Seq(5, 6, 7)
+ s.map.shardsForCoord(coord1.ref) shouldEqual Seq(2)
+ }
+ expectNoMessage(coord)
+ }
+
subscriber.expectMsgPF() { case s: CurrentShardSnapshot =>
s.map.shardsForCoord(coord4.ref) shouldEqual Seq(0, 1)
s.map.shardsForCoord(coord3.ref) shouldEqual Seq(3, 4)
@@ -238,9 +322,41 @@ class ReassignShardsSpec extends AkkaSpec {
val shardAssign2 = AssignShardConfig(coord3Address.toString, Seq(1))
shardManager.stopShards(NodeClusterActor.StopShards(UnassignShardConfig(shardAssign2.shardList), dataset1), self)
expectMsg(Success)
+
+ for (coord <- Seq(coord1, coord2, coord3, coord4)) {
+ coord.expectMsgPF() { case s: ShardIngestionState =>
+ s.ref shouldEqual dataset1
+ s.map.shardsForCoord(coord4.ref) shouldEqual Seq(0)
+ s.map.shardsForCoord(coord3.ref) shouldEqual Seq(3, 4)
+ s.map.shardsForCoord(coord2.ref) shouldEqual Seq(5, 6, 7)
+ s.map.shardsForCoord(coord1.ref) shouldEqual Seq(2)
+ }
+ expectNoMessage(coord)
+ }
+
+ subscriber.expectMsgPF() { case s: CurrentShardSnapshot =>
+ s.map.shardsForCoord(coord4.ref) shouldEqual Seq(0)
+ s.map.shardsForCoord(coord3.ref) shouldEqual Seq(3, 4)
+ s.map.shardsForCoord(coord2.ref) shouldEqual Seq(5, 6, 7)
+ s.map.shardsForCoord(coord1.ref) shouldEqual Seq(2)
+ }
+
shardManager.startShards(NodeClusterActor.StartShards(shardAssign2, dataset1), self)
expectMsg(Success)
+ expectDataset(coord3, datasetObj1)
+
+ for (coord <- Seq(coord1, coord2, coord3, coord4)) {
+ coord.expectMsgPF() { case s: ShardIngestionState =>
+ s.ref shouldEqual dataset1
+ s.map.shardsForCoord(coord4.ref) shouldEqual Seq(0)
+ s.map.shardsForCoord(coord3.ref) shouldEqual Seq(1, 3, 4)
+ s.map.shardsForCoord(coord2.ref) shouldEqual Seq(5, 6, 7)
+ s.map.shardsForCoord(coord1.ref) shouldEqual Seq(2)
+ }
+ expectNoMessage(coord)
+ }
+
subscriber.expectMsgPF() { case s: CurrentShardSnapshot =>
s.map.shardsForCoord(coord4.ref) shouldEqual Seq(0)
s.map.shardsForCoord(coord3.ref) shouldEqual Seq(1, 3, 4)
@@ -254,11 +370,43 @@ class ReassignShardsSpec extends AkkaSpec {
val shardAssign2 = AssignShardConfig(coord1Address.toString, Seq(0, 7))
shardManager.stopShards(NodeClusterActor.StopShards(UnassignShardConfig(shardAssign2.shardList), dataset1), self)
expectMsg(Success)
+
+ for (coord <- Seq(coord1, coord2, coord3, coord4)) {
+ coord.expectMsgPF() { case s: ShardIngestionState =>
+ s.ref shouldEqual dataset1
+ s.map.shardsForCoord(coord4.ref) shouldEqual Nil
+ s.map.shardsForCoord(coord3.ref) shouldEqual Seq(1, 3, 4)
+ s.map.shardsForCoord(coord2.ref) shouldEqual Seq(5, 6)
+ s.map.shardsForCoord(coord1.ref) shouldEqual Seq(2)
+ }
+ expectNoMessage(coord)
+ }
+
+ subscriber.expectMsgPF() { case s: CurrentShardSnapshot =>
+ s.map.shardsForCoord(coord4.ref) shouldEqual Nil
+ s.map.shardsForCoord(coord3.ref) shouldEqual Seq(1, 3, 4)
+ s.map.shardsForCoord(coord2.ref) shouldEqual Seq(5, 6)
+ s.map.shardsForCoord(coord1.ref) shouldEqual Seq(2)
+ }
+
shardManager.startShards(NodeClusterActor.StartShards(shardAssign2, dataset1), self)
expectMsg(Success)
+ expectDataset(coord1, datasetObj1)
+
+ for (coord <- Seq(coord1, coord2, coord3, coord4)) {
+ coord.expectMsgPF() { case s: ShardIngestionState =>
+ s.ref shouldEqual dataset1
+ s.map.shardsForCoord(coord4.ref) shouldEqual Nil
+ s.map.shardsForCoord(coord3.ref) shouldEqual Seq(1, 3, 4)
+ s.map.shardsForCoord(coord2.ref) shouldEqual Seq(5, 6)
+ s.map.shardsForCoord(coord1.ref) shouldEqual Seq(0, 2, 7)
+ }
+ expectNoMessage(coord)
+ }
+
subscriber.expectMsgPF() { case s: CurrentShardSnapshot =>
- s.map.shardsForCoord(coord4.ref) shouldEqual Seq()
+ s.map.shardsForCoord(coord4.ref) shouldEqual Nil
s.map.shardsForCoord(coord3.ref) shouldEqual Seq(1, 3, 4)
s.map.shardsForCoord(coord2.ref) shouldEqual Seq(5, 6)
s.map.shardsForCoord(coord1.ref) shouldEqual Seq(0, 2, 7)
@@ -271,6 +419,22 @@ class ReassignShardsSpec extends AkkaSpec {
shardManager.coordinators shouldBe Seq(coord4.ref, coord2.ref, coord1.ref)
shardManager.datasetInfo.size shouldBe 1
+ expectDataset(coord4, datasetObj1)
+ expectDataset(coord2, datasetObj1)
+
+ for (coord <- Seq(coord1, coord2, coord4)) {
+ coord.expectMsgPF() { case s: ShardIngestionState =>
+ s.ref shouldEqual dataset1
+ s.map.shardsForCoord(coord4.ref) shouldEqual Seq(3, 4)
+ s.map.shardsForCoord(coord3.ref) shouldEqual Nil
+ s.map.shardsForCoord(coord2.ref) shouldEqual Seq(1, 5, 6)
+ s.map.shardsForCoord(coord1.ref) shouldEqual Seq(0, 2, 7)
+ }
+ expectNoMessage(coord)
+ }
+
+ expectNoMessage(coord3)
+
subscriber.expectMsgPF() { case s: CurrentShardSnapshot if s.ref == dataset1 =>
s.map.shardsForCoord(coord4.ref) shouldEqual Seq(3, 4)
s.map.shardsForCoord(coord2.ref) shouldEqual Seq(1, 5, 6)
@@ -284,9 +448,40 @@ class ReassignShardsSpec extends AkkaSpec {
val shardAssign2 = AssignShardConfig(coord4Address.toString, Seq(2))
shardManager.stopShards(NodeClusterActor.StopShards(UnassignShardConfig(shardAssign2.shardList), dataset1), self)
expectMsg(Success)
+
+ for (coord <- Seq(coord1, coord2, coord4)) {
+ coord.expectMsgPF() { case s: ShardIngestionState =>
+ s.ref shouldEqual dataset1
+ s.map.shardsForCoord(coord4.ref) shouldEqual Seq(3, 4)
+ s.map.shardsForCoord(coord3.ref) shouldEqual Nil
+ s.map.shardsForCoord(coord2.ref) shouldEqual Seq(1, 5, 6)
+ s.map.shardsForCoord(coord1.ref) shouldEqual Seq(0, 7)
+ }
+ expectNoMessage(coord)
+ }
+
+ subscriber.expectMsgPF() { case s: CurrentShardSnapshot =>
+ s.map.shardsForCoord(coord4.ref) shouldEqual Seq(3, 4)
+ s.map.shardsForCoord(coord2.ref) shouldEqual Seq(1, 5, 6)
+ s.map.shardsForCoord(coord1.ref) shouldEqual Seq(0, 7)
+ }
+
shardManager.startShards(NodeClusterActor.StartShards(shardAssign2, dataset1), self)
expectMsg(Success)
+ expectDataset(coord4, datasetObj1)
+
+ for (coord <- Seq(coord1, coord2, coord4)) {
+ coord.expectMsgPF() { case s: ShardIngestionState =>
+ s.ref shouldEqual dataset1
+ s.map.shardsForCoord(coord4.ref) shouldEqual Seq(2, 3, 4)
+ s.map.shardsForCoord(coord3.ref) shouldEqual Nil
+ s.map.shardsForCoord(coord2.ref) shouldEqual Seq(1, 5, 6)
+ s.map.shardsForCoord(coord1.ref) shouldEqual Seq(0, 7)
+ }
+ expectNoMessage(coord)
+ }
+
subscriber.expectMsgPF() { case s: CurrentShardSnapshot =>
s.map.shardsForCoord(coord4.ref) shouldEqual Seq(2, 3, 4)
s.map.shardsForCoord(coord2.ref) shouldEqual Seq(1, 5, 6)
@@ -299,10 +494,28 @@ class ReassignShardsSpec extends AkkaSpec {
shardManager.removeDataset(dataset1)
shardManager.datasetInfo.size shouldBe 0
+ for (coord <- Seq(coord1, coord2, coord4)) {
+ coord.expectMsgPF() { case s: ShardIngestionState =>
+ s.ref shouldEqual dataset1
+ s.map.shardsForCoord(coord4.ref) shouldEqual Nil
+ s.map.shardsForCoord(coord3.ref) shouldEqual Nil
+ s.map.shardsForCoord(coord2.ref) shouldEqual Nil
+ s.map.shardsForCoord(coord1.ref) shouldEqual Nil
+ }
+ expectNoMessage(coord)
+ }
+
+ subscriber.expectMsgPF() { case s: CurrentShardSnapshot =>
+ s.map.shardsForCoord(coord4.ref) shouldEqual Nil
+ s.map.shardsForCoord(coord2.ref) shouldEqual Nil
+ s.map.shardsForCoord(coord1.ref) shouldEqual Nil
+ }
+
val shardAssign1 = AssignShardConfig(coord1Address.toString, Seq(0,1))
shardManager.stopShards(NodeClusterActor.StopShards(UnassignShardConfig(shardAssign1.shardList), dataset1), self)
expectMsg(DatasetUnknown(dataset1)) // since there are no datasets
+ expectNoMessage(subscriber)
}
"succeed after adding dataset back" in {
@@ -313,9 +526,53 @@ class ReassignShardsSpec extends AkkaSpec {
assignments shouldEqual Map(coord1.ref -> Seq(0, 1, 2), coord2.ref -> Seq(3, 4, 5), coord4.ref -> Seq(6, 7))
expectMsg(DatasetVerified)
+ expectDataset(coord4, datasetObj1)
+ expectDataset(coord2, datasetObj1)
+ expectDataset(coord1, datasetObj1)
+
+ for (coord <- Seq(coord1, coord2, coord4)) {
+ coord.expectMsgPF() { case s: ShardIngestionState =>
+ s.ref shouldEqual dataset1
+ s.map.shardsForCoord(coord4.ref) shouldEqual Seq(6, 7)
+ s.map.shardsForCoord(coord3.ref) shouldEqual Nil
+ s.map.shardsForCoord(coord2.ref) shouldEqual Seq(3, 4, 5)
+ s.map.shardsForCoord(coord1.ref) shouldEqual Seq(0, 1, 2)
+ }
+ expectNoMessage(coord)
+ }
+
+ subscriber.expectMsg(ShardSubscriptions(Set(
+ ShardSubscription(dataset1, Set(subscriber.ref))), Set(subscriber.ref)))
+
+ for (i <- 1 to 2) {
+ subscriber.expectMsgPF() { case s: CurrentShardSnapshot =>
+ s.map.shardsForCoord(coord4.ref) shouldEqual Seq(6, 7)
+ s.map.shardsForCoord(coord2.ref) shouldEqual Seq(3, 4, 5)
+ s.map.shardsForCoord(coord1.ref) shouldEqual Seq(0, 1, 2)
+ }
+ }
+
val shardAssign1 = AssignShardConfig(coord4Address.toString, Seq(5))
shardManager.stopShards(NodeClusterActor.StopShards(UnassignShardConfig(shardAssign1.shardList), dataset1), self)
expectMsg(Success)
+
+ for (coord <- Seq(coord1, coord2, coord4)) {
+ coord.expectMsgPF() { case s: ShardIngestionState =>
+ s.ref shouldEqual dataset1
+ s.map.shardsForCoord(coord4.ref) shouldEqual Seq(6, 7)
+ s.map.shardsForCoord(coord3.ref) shouldEqual Nil
+ s.map.shardsForCoord(coord2.ref) shouldEqual Seq(3, 4)
+ s.map.shardsForCoord(coord1.ref) shouldEqual Seq(0, 1, 2)
+ }
+ expectNoMessage(coord)
+ }
+
+ subscriber.expectMsgPF() { case s: CurrentShardSnapshot =>
+ s.map.shardsForCoord(coord4.ref) shouldEqual Seq(6, 7)
+ s.map.shardsForCoord(coord2.ref) shouldEqual Seq(3, 4)
+ s.map.shardsForCoord(coord1.ref) shouldEqual Seq(0, 1, 2)
+ }
+
shardManager.startShards(NodeClusterActor.StartShards(shardAssign1, dataset1), self)
expectMsg(Success)
@@ -324,6 +581,26 @@ class ReassignShardsSpec extends AkkaSpec {
(coord1.ref, ShardStatusAssigned), (coord2.ref, ShardStatusAssigned), (coord2.ref, ShardStatusAssigned),
(coord4.ref, ShardStatusAssigned), (coord4.ref, ShardStatusAssigned), (coord4.ref, ShardStatusAssigned))
+ expectDataset(coord4, datasetObj1)
+
+ for (coord <- Seq(coord1, coord2, coord4)) {
+ coord.expectMsgPF() { case s: ShardIngestionState =>
+ s.ref shouldEqual dataset1
+ s.map.shardsForCoord(coord4.ref) shouldEqual Seq(5, 6, 7)
+ s.map.shardsForCoord(coord3.ref) shouldEqual Nil
+ s.map.shardsForCoord(coord2.ref) shouldEqual Seq(3, 4)
+ s.map.shardsForCoord(coord1.ref) shouldEqual Seq(0, 1, 2)
+ }
+ expectNoMessage(coord)
+ }
+
+ subscriber.expectMsgPF() { case s: CurrentShardSnapshot =>
+ s.map.shardsForCoord(coord4.ref) shouldEqual Seq(5, 6, 7)
+ s.map.shardsForCoord(coord2.ref) shouldEqual Seq(3, 4)
+ s.map.shardsForCoord(coord1.ref) shouldEqual Seq(0, 1, 2)
+ }
+
+ expectNoMessage(subscriber)
}
}
-}
\ No newline at end of file
+}
diff --git a/coordinator/src/test/scala/filodb.coordinator/ShardManagerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/ShardManagerSpec.scala
index 7033d5afb4..c0683b6d93 100644
--- a/coordinator/src/test/scala/filodb.coordinator/ShardManagerSpec.scala
+++ b/coordinator/src/test/scala/filodb.coordinator/ShardManagerSpec.scala
@@ -1,5 +1,7 @@
package filodb.coordinator
+import scala.concurrent.duration._
+
import akka.actor.{ActorRef, Address}
import akka.testkit.TestProbe
import com.typesafe.config.ConfigFactory
@@ -9,7 +11,7 @@ import filodb.core.metadata.Dataset
class ShardManagerSpec extends AkkaSpec {
import NodeClusterActor.{DatasetResourceSpec, DatasetVerified, IngestionSource, SetupDataset}
- import client.IngestionCommands.DatasetSetup
+ import filodb.coordinator.client.IngestionCommands.DatasetSetup
protected val dataset1 = DatasetRef("one")
protected val datasetObj1 = Dataset(dataset1.dataset, Seq("seg:int"), Seq("timestamp:long"))
@@ -22,19 +24,26 @@ class ShardManagerSpec extends AkkaSpec {
val settings = new FilodbSettings(ConfigFactory.load("application_test.conf"))
protected val shardManager = new ShardManager(settings, DefaultShardAssignmentStrategy)
- val coord1 = TestProbe("coordinator1")
+ private def makeTestProbe(name: String): TestProbe = {
+ val tp = TestProbe(name)
+ // Uncomment to ignore messages of a specific type.
+ //tp.ignoreMsg({case m: Any => m.isInstanceOf[...]})
+ tp
+ }
+
+ val coord1 = makeTestProbe("coordinator1")
val coord1Address = uniqueAddress(coord1.ref)
- val coord2 = TestProbe("coordinator2")
+ val coord2 = makeTestProbe("coordinator2")
val coord2Address = uniqueAddress(coord2.ref)
- val coord3 = TestProbe("coordinator3")
+ val coord3 = makeTestProbe("coordinator3")
val coord3Address = uniqueAddress(coord3.ref)
- val coord4 = TestProbe("coordinator4")
+ val coord4 = makeTestProbe("coordinator4")
val coord4Address = uniqueAddress(coord4.ref)
- val subscriber = TestProbe("subscriber")
+ val subscriber = makeTestProbe("subscriber")
val noOpSource1 = IngestionSource(classOf[NoOpStreamFactory].getName)
val setupDs1 = SetupDataset(dataset1, resources1, noOpSource1, TestData.storeConf)
@@ -42,6 +51,17 @@ class ShardManagerSpec extends AkkaSpec {
val noOpSource2 = IngestionSource(classOf[NoOpStreamFactory].getName)
val setupDs2 = SetupDataset(dataset2, resources2, noOpSource2, TestData.storeConf)
+ private def expectDataset(coord: TestProbe, dataset: Dataset): TestProbe = {
+ coord.expectMsgPF() { case ds: DatasetSetup =>
+ ds.compactDatasetStr shouldEqual dataset.asCompactString
+ ds.source shouldEqual noOpSource1
+ }
+ coord
+ }
+
+ private def expectNoMessage(coord: TestProbe): Unit = {
+ coord.expectNoMessage(100.milliseconds)
+ }
def uniqueAddress(probe: ActorRef): Address =
probe.path.address.copy(system = s"${probe.path.address.system}-${probe.path.name}")
@@ -52,35 +72,35 @@ class ShardManagerSpec extends AkkaSpec {
"allow subscription of self for shard events on all datasets" in {
shardManager.subscribeAll(subscriber.ref)
subscriber.expectMsg(ShardSubscriptions(Set.empty, Set(subscriber.ref)))
- subscriber.expectNoMessage() // should not get a CurrentShardSnapshot since there isnt a dataset yet
+ expectNoMessage(subscriber) // should not get a CurrentShardSnapshot since there isnt a dataset yet
}
"change state for addition of first coordinator without datasets" in {
shardManager.addMember(coord1Address, coord1.ref)
shardManager.coordinators shouldBe Seq(coord1.ref)
shardManager.datasetInfo.size shouldBe 0
- coord1.expectNoMessage() // since there are no datasets, there should be no assignments
+ expectNoMessage(coord1) // since there are no datasets, there should be no assignments
}
"change state for addition of second coordinator without datasets" in {
shardManager.addMember(coord2Address, coord2.ref)
shardManager.coordinators shouldBe Seq(coord1.ref, coord2.ref)
shardManager.datasetInfo.size shouldBe 0
- coord2.expectNoMessage() // since there are no datasets, there should be no assignments
+ expectNoMessage(coord2) // since there are no datasets, there should be no assignments
}
"change state for addition of third coordinator without datasets" in {
shardManager.addMember(coord3Address, coord3.ref)
shardManager.coordinators shouldBe Seq(coord1.ref, coord2.ref, coord3.ref)
shardManager.datasetInfo.size shouldBe 0
- coord3.expectNoMessage() // since there are no datasets, there should be no assignments
+ expectNoMessage(coord3) // since there are no datasets, there should be no assignments
}
"change state for removal of coordinator without datasets" in {
shardManager.removeMember(coord2Address)
shardManager.coordinators shouldBe Seq(coord1.ref, coord3.ref)
shardManager.datasetInfo.size shouldBe 0
- coord2.expectNoMessage() // since there are no datasets, there should be no assignments
+ expectNoMessage(coord2) // since there are no datasets, there should be no assignments
}
"change state for addition of new dataset" in {
@@ -89,57 +109,57 @@ class ShardManagerSpec extends AkkaSpec {
assignments shouldEqual Map(coord3.ref -> Seq(0,1,2), coord1.ref -> Seq(3,4,5))
expectMsg(DatasetVerified)
- coord3.expectMsgPF() { case ds: DatasetSetup =>
- ds.compactDatasetStr shouldEqual datasetObj1.asCompactString
- ds.source shouldEqual noOpSource1
- }
- // assignments first go to the most recently deployed node
- coord3.expectMsgAllOf(
- StartShardIngestion(dataset1, 0, None),
- StartShardIngestion(dataset1, 1, None),
- StartShardIngestion(dataset1, 2, None))
-
- coord1.expectMsgPF() { case ds: DatasetSetup =>
- ds.compactDatasetStr shouldEqual datasetObj1.asCompactString
- ds.source shouldEqual noOpSource1
+
+ for (coord <- Seq(coord1, coord3)) {
+ expectDataset(coord, datasetObj1).expectMsgPF() { case s: ShardIngestionState =>
+ s.ref shouldEqual dataset1
+ s.map.shardsForCoord(coord1.ref) shouldEqual Seq(3, 4, 5)
+ s.map.shardsForCoord(coord2.ref) shouldEqual Nil
+ s.map.shardsForCoord(coord3.ref) shouldEqual Seq(0, 1, 2)
+ }
+ expectNoMessage(coord)
}
- coord1.expectMsgAllOf(
- StartShardIngestion(dataset1, 3, None),
- StartShardIngestion(dataset1, 4, None),
- StartShardIngestion(dataset1, 5, None))
- // NOTE: because subscriptions do not kick in right away, we don't get new snapshots unitl after
- // ShardSubscriptions message
+ // NOTE: because subscriptions do not kick in right away, we don't get new snapshots until
+ // after ShardSubscriptions message
}
"send shard subscribers updates on shard events as a result of dataset addition" in {
subscriber.expectMsg(ShardSubscriptions(Set(
ShardSubscription(dataset1, Set(subscriber.ref))), Set(subscriber.ref)))
- subscriber.expectMsgPF() { case s: CurrentShardSnapshot =>
- s.ref shouldEqual dataset1
- s.map.shardsForCoord(coord3.ref) shouldEqual Seq(0, 1, 2)
- s.map.shardsForCoord(coord1.ref) shouldEqual Seq(3, 4, 5)
- }
- subscriber.expectNoMessage()
+ for (i <- 1 to 2) {
+ // First is the initial set, the second is generated along with the state.
+ subscriber.expectMsgPF() { case s: CurrentShardSnapshot =>
+ s.ref shouldEqual dataset1
+ s.map.shardsForCoord(coord1.ref) shouldEqual Seq(3, 4, 5)
+ s.map.shardsForCoord(coord3.ref) shouldEqual Seq(0, 1, 2)
+ }
+ }
+ expectNoMessage(subscriber)
}
"change state for addition of coordinator when there are datasets" in {
shardManager.addMember(coord2Address, coord2.ref)
shardManager.coordinators shouldBe Seq(coord1.ref, coord3.ref, coord2.ref)
- coord2.expectMsgPF() { case ds: DatasetSetup =>
- ds.compactDatasetStr shouldEqual datasetObj1.asCompactString
- ds.source shouldEqual noOpSource1
- }
- coord2.expectMsgAllOf(
- StartShardIngestion(dataset1, 6, None),
- StartShardIngestion(dataset1, 7, None))
-
// Should receive new snapshot with shards 6,7 for coord2
subscriber.expectMsgPF() { case s: CurrentShardSnapshot =>
- s.map.shardsForCoord(coord3.ref) shouldEqual Seq(0, 1, 2)
s.map.shardsForCoord(coord1.ref) shouldEqual Seq(3, 4, 5)
s.map.shardsForCoord(coord2.ref) shouldEqual Seq(6, 7)
+ s.map.shardsForCoord(coord3.ref) shouldEqual Seq(0, 1, 2)
+ }
+
+ expectDataset(coord2, datasetObj1)
+
+ // All should see the changes.
+ for (coord <- Seq(coord1, coord2, coord3)) {
+ coord.expectMsgPF() { case s: ShardIngestionState =>
+ s.ref shouldEqual dataset1
+ s.map.shardsForCoord(coord1.ref) shouldEqual Seq(3, 4, 5)
+ s.map.shardsForCoord(coord2.ref) shouldEqual Seq(6, 7)
+ s.map.shardsForCoord(coord3.ref) shouldEqual Seq(0, 1, 2)
+ }
+ expectNoMessage(coord)
}
}
@@ -147,8 +167,12 @@ class ShardManagerSpec extends AkkaSpec {
shardManager.addMember(coord4Address, coord4.ref)
shardManager.coordinators shouldBe Seq(coord1.ref, coord3.ref, coord2.ref, coord4.ref)
shardManager.datasetInfo.size shouldBe 1
- coord4.expectNoMessage() // since this is a spare node, there should be no assignments
- subscriber.expectNoMessage()
+ expectNoMessage(coord4) // since this is a spare node, there should be no assignments
+ expectNoMessage(subscriber)
+
+ for (coord <- Seq(coord1, coord2, coord3, coord3)) {
+ expectNoMessage(coord)
+ }
}
"change state for removal of coordinator when there are datasets and spare nodes" in {
@@ -156,34 +180,29 @@ class ShardManagerSpec extends AkkaSpec {
shardManager.coordinators shouldBe Seq(coord3.ref, coord2.ref, coord4.ref)
shardManager.datasetInfo.size shouldBe 1
- // first ingestion should be stopped on downed node
- coord1.expectMsgAllOf(
- StopShardIngestion(dataset1, 3),
- StopShardIngestion(dataset1, 4),
- StopShardIngestion(dataset1, 5))
+ expectDataset(coord4, datasetObj1)
- subscriber.expectMsgPF() { case s: CurrentShardSnapshot if s.ref == dataset1 =>
- s.map.shardsForCoord(coord3.ref) shouldEqual Seq(0, 1, 2)
- s.map.shardsForCoord(coord1.ref) shouldEqual Nil
- s.map.shardsForCoord(coord2.ref) shouldEqual Seq(6, 7)
+ // All coordinators should see a state message, even the removed one.
+ for (coord <- Seq(coord2, coord3, coord4)) {
+ //expectDataset(coord, datasetObj1).expectMsgPF() { case s: ShardIngestionState =>
+ coord.expectMsgPF() { case s: ShardIngestionState =>
+ s.ref shouldEqual dataset1
+ s.map.shardsForCoord(coord1.ref) shouldEqual Nil // coord1 is gone
+ s.map.shardsForCoord(coord2.ref) shouldEqual Seq(6, 7)
+ s.map.shardsForCoord(coord3.ref) shouldEqual Seq(0, 1, 2)
+ s.map.shardsForCoord(coord4.ref) shouldEqual Seq(3, 4, 5) // coord4 takes over
+ }
+ expectNoMessage(coord)
}
- // spare coord4 should take over the shards
- coord4.expectMsgPF() { case ds: DatasetSetup =>
- ds.compactDatasetStr shouldEqual datasetObj1.asCompactString
- ds.source shouldEqual noOpSource1
- }
- coord4.expectMsgAllOf(
- StartShardIngestion(dataset1, 3, None),
- StartShardIngestion(dataset1, 4, None),
- StartShardIngestion(dataset1, 5, None))
-
subscriber.expectMsgPF() { case s: CurrentShardSnapshot if s.ref == dataset1 =>
+ s.map.shardsForCoord(coord1.ref) shouldEqual Nil
+ s.map.shardsForCoord(coord2.ref) shouldEqual Seq(6, 7)
s.map.shardsForCoord(coord3.ref) shouldEqual Seq(0, 1, 2)
s.map.shardsForCoord(coord4.ref) shouldEqual Seq(3, 4, 5)
- s.map.shardsForCoord(coord2.ref) shouldEqual Seq(6, 7)
}
- subscriber.expectNoMessage()
+
+ expectNoMessage(subscriber)
}
"reassign shards where additional room available on removal of coordinator when there are no spare nodes" in {
@@ -191,64 +210,59 @@ class ShardManagerSpec extends AkkaSpec {
shardManager.coordinators shouldBe Seq(coord3.ref, coord2.ref)
shardManager.datasetInfo.size shouldBe 1
- // ingestion should be stopped on downed node
- coord4.expectMsgAllOf(
- StopShardIngestion(dataset1, 3),
- StopShardIngestion(dataset1, 4),
- StopShardIngestion(dataset1, 5))
-
+ expectDataset(coord2, datasetObj1)
- subscriber.expectMsgPF() { case s: CurrentShardSnapshot if s.ref == dataset1 =>
- s.map.shardsForCoord(coord3.ref) shouldEqual Seq(0, 1, 2)
- // s.map.shardsForCoord(coord2.ref) shouldEqual Seq(6, 7)
- // NOTE: you would expect shard2 to have 6 & 7 only in the first snapshot which is after removing coord4.
- // The problem is that ShardMapper is mutable, and since this is a local transfer, the actor message
- // points to the ShardMapper object which by this point already has the shard 3 added. :(
- s.map.shardsForCoord(coord2.ref) shouldEqual Seq(3, 6, 7)
- s.map.shardsForCoord(coord4.ref) shouldEqual Nil
- s.map.numAssignedShards shouldEqual 6
+ // Ingestion should be stopped on downed node, and one shard reassigned.
+ for (coord <- Seq(coord2, coord3)) {
+ coord.expectMsgPF() { case s: ShardIngestionState =>
+ s.ref shouldEqual dataset1
+ s.map.shardsForCoord(coord1.ref) shouldEqual Nil
+ s.map.shardsForCoord(coord2.ref) shouldEqual Seq(3, 6, 7)
+ s.map.shardsForCoord(coord3.ref) shouldEqual Seq(0, 1, 2)
+ s.map.shardsForCoord(coord4.ref) shouldEqual Nil
+ }
+ expectNoMessage(coord)
}
- // coord2 has room for one more
- coord2.expectMsgPF() { case ds: DatasetSetup =>
- ds.compactDatasetStr shouldEqual datasetObj1.asCompactString
- ds.source shouldEqual noOpSource1
+ for (coord <- Seq(coord1, coord2, coord3, coord3)) {
+ expectNoMessage(coord)
}
- coord2.expectMsgAllOf(
- StartShardIngestion(dataset1, 3, None))
-
- // since there are no spare nodes now, other coords (which are "down") should not get any message
- coord1.expectNoMessage()
- coord3.expectNoMessage()
subscriber.expectMsgPF() { case s: CurrentShardSnapshot if s.ref == dataset1 =>
- s.map.shardsForCoord(coord3.ref) shouldEqual Seq(0, 1, 2)
+ s.map.shardsForCoord(coord1.ref) shouldEqual Nil
s.map.shardsForCoord(coord2.ref) shouldEqual Seq(3, 6, 7)
- s.map.unassignedShards shouldEqual Seq(4, 5)
+ s.map.shardsForCoord(coord3.ref) shouldEqual Seq(0, 1, 2)
+ s.map.shardsForCoord(coord4.ref) shouldEqual Nil
s.map.numAssignedShards shouldEqual 6
}
- subscriber.expectNoMessage()
+
+ expectNoMessage(subscriber)
}
"reassign remaining unassigned shards when a replacement node comes back" in {
shardManager.addMember(coord4.ref.path.address, coord4.ref)
shardManager.coordinators shouldBe Seq(coord3.ref, coord2.ref, coord4.ref)
- coord4.expectMsgPF() { case ds: DatasetSetup =>
- ds.compactDatasetStr shouldEqual datasetObj1.asCompactString
- ds.source shouldEqual noOpSource1
+ expectDataset(coord4, datasetObj1)
+
+ for (coord <- Seq(coord2, coord3, coord4)) {
+ coord.expectMsgPF() { case s: ShardIngestionState =>
+ s.ref shouldEqual dataset1
+ s.map.shardsForCoord(coord1.ref) shouldEqual Nil
+ s.map.shardsForCoord(coord2.ref) shouldEqual Seq(3, 6, 7)
+ s.map.shardsForCoord(coord3.ref) shouldEqual Seq(0, 1, 2)
+ s.map.shardsForCoord(coord4.ref) shouldEqual Seq(4, 5)
+ }
+ expectNoMessage(coord)
}
- coord4.expectMsgAllOf(
- StartShardIngestion(dataset1, 4, None),
- StartShardIngestion(dataset1, 5, None))
subscriber.expectMsgPF() { case s: CurrentShardSnapshot =>
- s.map.shardsForCoord(coord3.ref) shouldEqual Seq(0, 1, 2)
s.map.shardsForCoord(coord2.ref) shouldEqual Seq(3, 6, 7)
+ s.map.shardsForCoord(coord3.ref) shouldEqual Seq(0, 1, 2)
s.map.shardsForCoord(coord4.ref) shouldEqual Seq(4, 5)
s.map.unassignedShards shouldEqual Nil
}
- subscriber.expectNoMessage()
+ expectNoMessage(subscriber)
}
"ingestion error on a shard should reassign shard to another node" in {
@@ -256,81 +270,76 @@ class ShardManagerSpec extends AkkaSpec {
shardManager.updateFromExternalShardEvent(subscriber.ref,
IngestionError(dataset1, 0, new IllegalStateException("simulated")))
- coord4.expectMsgPF() { case ds: DatasetSetup =>
- ds.compactDatasetStr shouldEqual datasetObj1.asCompactString
- ds.source shouldEqual noOpSource1
- }
- 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
+ expectDataset(coord4, datasetObj1)
+
+ for (coord <- Seq(coord2, coord3, coord4)) {
+ coord.expectMsgPF() { case s: ShardIngestionState =>
+ s.ref shouldEqual dataset1
+ s.map.shardsForCoord(coord1.ref) shouldEqual Nil
+ s.map.shardsForCoord(coord2.ref) shouldEqual Seq(3, 6, 7)
+ s.map.shardsForCoord(coord3.ref) shouldEqual Seq(1, 2)
+ s.map.shardsForCoord(coord4.ref) shouldEqual Seq(0, 4, 5)
+ }
+ expectNoMessage(coord)
}
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(coord3.ref) shouldEqual Seq(1, 2)
s.map.shardsForCoord(coord4.ref) shouldEqual Seq(0, 4, 5)
s.map.unassignedShards shouldEqual Nil
}
- subscriber.expectNoMessage()
+ expectNoMessage(subscriber)
}
"continual ingestion error on a shard should not reassign shard to another node" in {
shardManager.coordinators shouldEqual Seq(coord3.ref, coord2.ref, coord4.ref)
shardManager.updateFromExternalShardEvent(subscriber.ref,
IngestionError(dataset1, 0, new IllegalStateException("simulated")))
- coord3.expectNoMessage()
- coord2.expectNoMessage()
- coord4.expectNoMessage()
+
+ // Shard 0 is "bad" and so it's unassigned now.
+
+ for (coord <- Seq(coord2, coord3, coord4)) {
+ coord.expectMsgPF() { case s: ShardIngestionState =>
+ s.ref shouldEqual dataset1
+ s.map.shardsForCoord(coord1.ref) shouldEqual Nil
+ s.map.shardsForCoord(coord2.ref) shouldEqual Seq(3, 6, 7)
+ s.map.shardsForCoord(coord3.ref) shouldEqual Seq(1, 2)
+ s.map.shardsForCoord(coord4.ref) shouldEqual Seq(4, 5)
+ }
+ expectNoMessage(coord)
+ }
+
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(coord3.ref) shouldEqual Seq(1, 2)
s.map.shardsForCoord(coord4.ref) shouldEqual Seq(4, 5)
s.map.unassignedShards shouldEqual Seq(0)
}
- subscriber.expectNoMessage()
+ expectNoMessage(subscriber)
}
"change state for removal of dataset" in {
shardManager.removeDataset(dataset1)
shardManager.datasetInfo.size shouldBe 0
- coord3.expectMsgAllOf(
- StopShardIngestion(dataset1, 1),
- StopShardIngestion(dataset1, 2))
-
- coord4.expectMsgAllOf(
- StopShardIngestion(dataset1, 4), // shard 0 is in error state
- StopShardIngestion(dataset1, 5))
-
- coord2.expectMsgAllOf(
- StopShardIngestion(dataset1, 3),
- StopShardIngestion(dataset1, 6),
- StopShardIngestion(dataset1, 7))
+ for (coord <- Seq(coord2, coord3, coord4)) {
+ coord.expectMsgPF() { case s: ShardIngestionState =>
+ s.ref shouldEqual dataset1
+ s.map.shardsForCoord(coord1.ref) shouldEqual Nil
+ s.map.shardsForCoord(coord2.ref) shouldEqual Nil
+ s.map.shardsForCoord(coord3.ref) shouldEqual Nil
+ s.map.shardsForCoord(coord4.ref) shouldEqual Nil
+ }
+ expectNoMessage(coord)
+ }
shardManager.subscriptions.subscriptions.size shouldBe 0
- // 3 snapshots one for each coord
- subscriber.expectMsgPF() { case s: CurrentShardSnapshot =>
- s.map.unassignedShards shouldEqual (0 to 7)
- }
subscriber.expectMsgPF() { case s: CurrentShardSnapshot =>
s.map.unassignedShards shouldEqual (0 to 7)
}
- subscriber.expectMsgPF() { case s: CurrentShardSnapshot =>
- s.map.unassignedShards shouldEqual (0 to 7)
- }
- subscriber.expectNoMessage()
+ expectNoMessage(subscriber)
}
"change state for addition of multiple datasets" in {
@@ -341,50 +350,41 @@ class ShardManagerSpec extends AkkaSpec {
val assignments1 = shardManager.addDataset(setupDs1, datasetObj1, self)
shardManager.datasetInfo.size shouldBe 1
assignments1 shouldEqual Map(coord1.ref -> Seq(0, 1, 2),
- coord4.ref -> Seq(3, 4, 5),
coord2.ref -> Seq(6, 7),
- coord3.ref -> Seq())
-
- coord1.expectMsgPF() { case ds: DatasetSetup =>
- ds.compactDatasetStr shouldEqual datasetObj1.asCompactString
- ds.source shouldEqual noOpSource1
- }
- // assignments first go to the most recently deployed node
- coord1.expectMsgAllOf(
- StartShardIngestion(dataset1, 0, None),
- StartShardIngestion(dataset1, 1, None),
- StartShardIngestion(dataset1, 2, None))
+ coord3.ref -> Seq(),
+ coord4.ref -> Seq(3, 4, 5))
- // No CurrentShardSnapshot yet. We have to get subscription first.
+ expectDataset(coord1, datasetObj1)
+ expectDataset(coord2, datasetObj1)
+ expectDataset(coord4, datasetObj1)
- coord4.expectMsgPF() { case ds: DatasetSetup =>
- ds.compactDatasetStr shouldEqual datasetObj1.asCompactString
- ds.source shouldEqual noOpSource1
+ for (coord <- Seq(coord1, coord2, coord3, coord4)) {
+ coord.expectMsgPF() { case s: ShardIngestionState =>
+ s.ref shouldEqual dataset1
+ s.map.shardsForCoord(coord1.ref) shouldEqual Seq(0, 1, 2)
+ s.map.shardsForCoord(coord2.ref) shouldEqual Seq(6, 7)
+ s.map.shardsForCoord(coord3.ref) shouldEqual Nil
+ s.map.shardsForCoord(coord4.ref) shouldEqual Seq(3, 4, 5)
+ }
+ expectNoMessage(coord)
}
- coord4.expectMsgAllOf(
- StartShardIngestion(dataset1, 3, None),
- StartShardIngestion(dataset1, 4, None),
- StartShardIngestion(dataset1, 5, None))
-
- coord2.expectMsgPF() { case ds: DatasetSetup =>
- ds.compactDatasetStr shouldEqual datasetObj1.asCompactString
- ds.source shouldEqual noOpSource1
- }
- coord2.expectMsgAllOf(
- StartShardIngestion(dataset1, 6, None),
- StartShardIngestion(dataset1, 7, None))
-
- coord3.expectNoMessage() // coord3 is spare node for dataset1
// addition of dataset results in snapshot/subscriptions broadcast
subscriber.expectMsg(
ShardSubscriptions(Set(ShardSubscription(dataset1, Set(subscriber.ref))), Set(subscriber.ref)))
- subscriber.expectMsgPF() { case s: CurrentShardSnapshot if s.ref == dataset1 =>
- s.map.shardsForCoord(coord1.ref) shouldEqual Seq(0, 1, 2)
- s.map.shardsForCoord(coord4.ref) shouldEqual Seq(3, 4, 5)
- s.map.shardsForCoord(coord2.ref) shouldEqual Seq(6, 7)
- s.map.unassignedShards shouldEqual Nil
+ for (i <- 1 to 2) {
+ // First is the initial set, the second is generated along with the state.
+ subscriber.expectMsgPF() { case s: CurrentShardSnapshot if s.ref == dataset1 =>
+ s.map.shardsForCoord(coord1.ref) shouldEqual Seq(0, 1, 2)
+ s.map.shardsForCoord(coord2.ref) shouldEqual Seq(6, 7)
+ s.map.shardsForCoord(coord4.ref) shouldEqual Seq(3, 4, 5)
+ s.map.unassignedShards shouldEqual Nil
+ }
}
+ expectNoMessage(subscriber)
+
+ // Assignments first go to the most recently deployed node,
+ // coord2 and coord3 are spare nodes for dataset2.
val assignments2 = shardManager.addDataset(setupDs2, datasetObj2, self)
shardManager.datasetInfo.size shouldBe 2
@@ -394,24 +394,19 @@ class ShardManagerSpec extends AkkaSpec {
coord2.ref -> Seq.empty,
coord3.ref -> Seq.empty)
- coord1.expectMsgPF() { case ds: DatasetSetup =>
- ds.compactDatasetStr shouldEqual datasetObj2.asCompactString
- ds.source shouldEqual noOpSource2
- }
- // assignments first go to the most recently deployed node
- val msgs1 = coord1.receiveWhile(messages = 8) { case m: StartShardIngestion if m.ref == dataset2 => m }
- msgs1 should have length (8)
+ expectDataset(coord1, datasetObj2)
+ expectDataset(coord4, datasetObj2)
- coord4.expectMsgPF() { case ds: DatasetSetup =>
- ds.compactDatasetStr shouldEqual datasetObj2.asCompactString
- ds.source shouldEqual noOpSource2
+ for (coord <- Seq(coord1, coord2, coord3, coord4)) {
+ coord.expectMsgPF() { case s: ShardIngestionState =>
+ s.ref shouldEqual dataset2
+ s.map.shardsForCoord(coord1.ref) shouldEqual Range(0, 8)
+ s.map.shardsForCoord(coord2.ref) shouldEqual Nil
+ s.map.shardsForCoord(coord3.ref) shouldEqual Nil
+ s.map.shardsForCoord(coord4.ref) shouldEqual Range(8, 16)
+ }
+ expectNoMessage(coord)
}
- val msgs2 = coord4.receiveWhile(messages = 8) { case m: StartShardIngestion if m.ref == dataset2 => m }
- msgs2 should have length (8)
-
- // coord2 and coord3 are spare nodes for dataset2
- coord2.expectNoMessage()
- coord3.expectNoMessage()
// shard subscriptions should work with multiple datasets
subscriber.expectMsg(
@@ -419,14 +414,16 @@ class ShardManagerSpec extends AkkaSpec {
ShardSubscription(dataset2,Set(subscriber.ref))),
Set(subscriber.ref)))
- subscriber.expectMsgPF() { case s: CurrentShardSnapshot if s.ref == dataset2 =>
- s.map.shardsForCoord(coord1.ref) shouldEqual Range(0, 8)
- s.map.shardsForCoord(coord4.ref) shouldEqual Range(8, 16)
- s.map.shardsForCoord(coord2.ref) shouldEqual Seq.empty
- s.map.shardsForCoord(coord3.ref) shouldEqual Seq.empty
+ for (i <- 1 to 2) {
+ // First is the initial set, the second is generated along with the state.
+ subscriber.expectMsgPF() { case s: CurrentShardSnapshot if s.ref == dataset2 =>
+ s.map.shardsForCoord(coord1.ref) shouldEqual Range(0, 8)
+ s.map.shardsForCoord(coord2.ref) shouldEqual Seq.empty
+ s.map.shardsForCoord(coord3.ref) shouldEqual Seq.empty
+ s.map.shardsForCoord(coord4.ref) shouldEqual Range(8, 16)
+ }
}
-
- subscriber.expectNoMessage()
+ expectNoMessage(subscriber)
}
"recover state on a failed over node " in {
@@ -455,19 +452,12 @@ class ShardManagerSpec extends AkkaSpec {
shardManager2.coordinators shouldBe Seq(coord3.ref, coord2.ref, coord1.ref)
shardManager2.datasetInfo.size shouldBe 2
- // dataset2: reassign shards 8-16 to coord2. Will get two snapshots during move
+ // dataset2: reassign shards 8-16 to coord2.
subscriber.expectMsgPF() { case s: CurrentShardSnapshot if s.ref == dataset2 =>
s.map.shardsForCoord(coord1.ref) shouldEqual Range(0, 8)
- s.map.shardsForCoord(coord4.ref) shouldEqual Nil
s.map.shardsForCoord(coord2.ref) shouldEqual (8 until 16)
s.map.shardsForCoord(coord3.ref) shouldEqual Nil
- }
-
- subscriber.expectMsgPF() { case s: CurrentShardSnapshot if s.ref == dataset2 =>
- s.map.shardsForCoord(coord1.ref) shouldEqual Range(0, 8)
s.map.shardsForCoord(coord4.ref) shouldEqual Nil
- s.map.shardsForCoord(coord2.ref) shouldEqual (8 until 16)
- s.map.shardsForCoord(coord3.ref) shouldEqual Nil
}
// dataset1: reassign shards 3,4,5 to coord2 and coord3
@@ -478,40 +468,83 @@ class ShardManagerSpec extends AkkaSpec {
s.map.shardsForCoord(coord4.ref) shouldEqual Nil
}
- subscriber.expectMsgPF() { case s: CurrentShardSnapshot if s.ref == dataset1 =>
- s.map.shardsForCoord(coord1.ref) shouldEqual Seq(0, 1, 2)
- s.map.shardsForCoord(coord2.ref) shouldEqual Seq(3, 6, 7)
- s.map.shardsForCoord(coord3.ref) shouldEqual Seq(4, 5)
- s.map.shardsForCoord(coord4.ref) shouldEqual Nil
- }
+ expectNoMessage(subscriber)
// ingestion should be stopped on downed node for 8 + 3 shards
- coord4.receiveWhile(messages = 11) { case m: StopShardIngestion => m } should have length (11)
-
- // ingestion should failover to coord2 for dataset2 first, and get 8 StartShardIngestion messages
- coord2.expectMsgPF() { case ds: DatasetSetup =>
- ds.compactDatasetStr shouldEqual datasetObj2.asCompactString
- ds.source shouldEqual noOpSource2
- }
- coord2.receiveWhile(messages = 8) { case m: StartShardIngestion => m } should have length (8)
-
- // then failover to coord2 for dataset1, with 1 shard
- coord2.expectMsgPF() { case ds: DatasetSetup =>
- ds.compactDatasetStr shouldEqual datasetObj1.asCompactString
- ds.source shouldEqual noOpSource1
- }
- coord2.expectMsg(StartShardIngestion(dataset1, 3, None))
-
- // ingestion should failover to coord3 for dataset1
- coord3.expectMsgPF() { case ds: DatasetSetup =>
- ds.compactDatasetStr shouldEqual datasetObj1.asCompactString
- ds.source shouldEqual noOpSource1
- }
- // 3 shards for dataset1
- coord3.expectMsgAllOf(
- StartShardIngestion(dataset1, 4, None),
- StartShardIngestion(dataset1, 5, None))
+ { // coord1
+ coord1.expectMsgPF() {
+ case s: ShardIngestionState =>
+ s.ref shouldEqual dataset2
+ s.map.shardsForCoord(coord1.ref) shouldEqual Range(0, 8)
+ s.map.shardsForCoord(coord2.ref) shouldEqual (8 until 16)
+ s.map.shardsForCoord(coord3.ref) shouldEqual Nil
+ s.map.shardsForCoord(coord4.ref) shouldEqual Nil
+ }
+
+ coord1.expectMsgPF() {
+ case s: ShardIngestionState =>
+ s.ref shouldEqual dataset1
+ s.map.shardsForCoord(coord1.ref) shouldEqual Seq(0, 1, 2)
+ s.map.shardsForCoord(coord2.ref) shouldEqual Seq(3, 6, 7)
+ s.map.shardsForCoord(coord3.ref) shouldEqual Seq(4, 5)
+ s.map.shardsForCoord(coord4.ref) shouldEqual Nil
+ }
+
+ expectNoMessage(coord1)
+ }
+
+ { // coord2
+ expectDataset(coord2, datasetObj2)
+
+ coord2.expectMsgPF() {
+ case s: ShardIngestionState =>
+ s.ref shouldEqual dataset2
+ s.map.shardsForCoord(coord1.ref) shouldEqual Range(0, 8)
+ s.map.shardsForCoord(coord2.ref) shouldEqual (8 until 16)
+ s.map.shardsForCoord(coord3.ref) shouldEqual Nil
+ s.map.shardsForCoord(coord4.ref) shouldEqual Nil
+ }
+
+ expectDataset(coord2, datasetObj1)
+
+ coord2.expectMsgPF() {
+ case s: ShardIngestionState =>
+ s.ref shouldEqual dataset1
+ s.map.shardsForCoord(coord1.ref) shouldEqual Seq(0, 1, 2)
+ s.map.shardsForCoord(coord2.ref) shouldEqual Seq(3, 6, 7)
+ s.map.shardsForCoord(coord3.ref) shouldEqual Seq(4, 5)
+ s.map.shardsForCoord(coord4.ref) shouldEqual Nil
+ }
+
+ expectNoMessage(coord2)
+ }
+
+ { // coord3
+ coord3.expectMsgPF() {
+ case s: ShardIngestionState =>
+ s.ref shouldEqual dataset2
+ s.map.shardsForCoord(coord1.ref) shouldEqual Range(0, 8)
+ s.map.shardsForCoord(coord2.ref) shouldEqual (8 until 16)
+ s.map.shardsForCoord(coord3.ref) shouldEqual Nil
+ s.map.shardsForCoord(coord4.ref) shouldEqual Nil
+ }
+
+ expectDataset(coord3, datasetObj1)
+
+ coord3.expectMsgPF() {
+ case s: ShardIngestionState =>
+ s.ref shouldEqual dataset1
+ s.map.shardsForCoord(coord1.ref) shouldEqual Seq(0, 1, 2)
+ s.map.shardsForCoord(coord2.ref) shouldEqual Seq(3, 6, 7)
+ s.map.shardsForCoord(coord3.ref) shouldEqual Seq(4, 5)
+ s.map.shardsForCoord(coord4.ref) shouldEqual Nil
+ }
+
+ expectNoMessage(coord3)
+ }
+
+ expectNoMessage(coord4)
}
}
}
diff --git a/coordinator/src/test/scala/filodb.coordinator/ShardMapperSpec.scala b/coordinator/src/test/scala/filodb.coordinator/ShardMapperSpec.scala
index e0490aa48f..bcc64c7a09 100644
--- a/coordinator/src/test/scala/filodb.coordinator/ShardMapperSpec.scala
+++ b/coordinator/src/test/scala/filodb.coordinator/ShardMapperSpec.scala
@@ -118,6 +118,7 @@ class ShardMapperSpec extends ActorTest(ShardMapperSpec.getNewSystem) {
mapper1.updateFromEvent(IngestionStarted(dataset, 3, ref2)).isSuccess shouldEqual true
mapper1.activeShards(Seq(1, 2, 3, 4)) shouldEqual Seq(2, 3, 4)
mapper1.numAssignedShards shouldEqual 3
+ println(mapper1.prettyPrint)
mapper1.updateFromEvent(ShardDown(dataset, 4, ref1)).isSuccess shouldEqual true
mapper1.activeShards(Seq(1, 2, 3, 4)) shouldEqual Seq(2, 3)
diff --git a/coordinator/src/test/scala/filodb.coordinator/client/SerializationSpec.scala b/coordinator/src/test/scala/filodb.coordinator/client/SerializationSpec.scala
index 482b1bf206..6ec7a90921 100644
--- a/coordinator/src/test/scala/filodb.coordinator/client/SerializationSpec.scala
+++ b/coordinator/src/test/scala/filodb.coordinator/client/SerializationSpec.scala
@@ -201,7 +201,8 @@ class SerializationSpec extends ActorTest(SerializationSpecConfig.getNewSystem)
val windowed2 = PeriodicSeriesWithWindowing(raw2, from, 1000, to, 5000, RangeFunctionId.Rate)
val summed2 = Aggregate(AggregationOperator.Sum, windowed2, Nil, Seq("job"))
val logicalPlan = BinaryJoin(summed1, BinaryOperator.DIV, Cardinality.OneToOne, summed2)
- val execPlan = engine.materialize(logicalPlan, QueryOptions(0, 100))
+ val execPlan = engine.materialize(logicalPlan, QueryOptions(Some(StaticSpreadProvider(SpreadChange(0, 0))),
+ 100), new StaticSpreadProvider(SpreadChange(0, 0)))
roundTrip(execPlan) shouldEqual execPlan
}
@@ -219,7 +220,8 @@ class SerializationSpec extends ActorTest(SerializationSpecConfig.getNewSystem)
val logicalPlan1 = Parser.queryRangeToLogicalPlan(
"sum(rate(http_request_duration_seconds_bucket{job=\"prometheus\"}[20s])) by (handler)",
qParams)
- val execPlan1 = engine.materialize(logicalPlan1, QueryOptions(0, 100))
+ val execPlan1 = engine.materialize(logicalPlan1, QueryOptions(Some(new StaticSpreadProvider(SpreadChange(0, 0))),
+ 100), new StaticSpreadProvider(SpreadChange(0, 0)))
roundTrip(execPlan1) shouldEqual execPlan1
// scalastyle:off
@@ -227,7 +229,7 @@ class SerializationSpec extends ActorTest(SerializationSpecConfig.getNewSystem)
"sum(rate(http_request_duration_microseconds_sum{job=\"prometheus\"}[5m])) by (handler) / sum(rate(http_request_duration_microseconds_count{job=\"prometheus\"}[5m])) by (handler)",
qParams)
// scalastyle:on
- val execPlan2 = engine.materialize(logicalPlan2, QueryOptions(0, 100))
+ val execPlan2 = engine.materialize(logicalPlan2, QueryOptions(Some(new StaticSpreadProvider(SpreadChange(0, 0))), 100), new StaticSpreadProvider(SpreadChange(0, 0)))
roundTrip(execPlan2) shouldEqual execPlan2
}
@@ -249,7 +251,8 @@ class SerializationSpec extends ActorTest(SerializationSpecConfig.getNewSystem)
val logicalPlan1 = Parser.metadataQueryToLogicalPlan(
"http_request_duration_seconds_bucket{job=\"prometheus\"}",
qParams)
- val execPlan1 = engine.materialize(logicalPlan1, QueryOptions(0, 100))
+ val execPlan1 = engine.materialize(logicalPlan1, QueryOptions(Some(
+ new StaticSpreadProvider(SpreadChange(0, 0))), 100), new StaticSpreadProvider(SpreadChange(0, 0)))
val partKeysExec = execPlan1.asInstanceOf[PartKeysExec] // will be dispatched to single shard
roundTrip(partKeysExec) shouldEqual partKeysExec
@@ -257,7 +260,8 @@ class SerializationSpec extends ActorTest(SerializationSpecConfig.getNewSystem)
val logicalPlan2 = Parser.metadataQueryToLogicalPlan(
"http_request_duration_seconds_bucket",
qParams)
- val execPlan2 = engine.materialize(logicalPlan2, QueryOptions(0, 100))
+ val execPlan2 = engine.materialize(logicalPlan2, QueryOptions(
+ Some(new StaticSpreadProvider(SpreadChange(0, 0))), 100))
val partKeysDistConcatExec = execPlan2.asInstanceOf[PartKeysDistConcatExec]
// will be dispatched to all active shards since no shard column filters in the query
diff --git a/coordinator/src/test/scala/filodb.coordinator/queryengine2/QueryEngineSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryengine2/QueryEngineSpec.scala
index 72ee6b9761..9ef329c53b 100644
--- a/coordinator/src/test/scala/filodb.coordinator/queryengine2/QueryEngineSpec.scala
+++ b/coordinator/src/test/scala/filodb.coordinator/queryengine2/QueryEngineSpec.scala
@@ -3,7 +3,6 @@ package filodb.coordinator.queryengine2
import akka.actor.ActorSystem
import akka.testkit.TestProbe
import org.scalatest.{FunSpec, Matchers}
-
import filodb.coordinator.ShardMapper
import filodb.coordinator.client.QueryCommands.{FunctionalSpreadProvider, QueryOptions, SpreadChange}
import filodb.core.MetricsTestData
@@ -117,7 +116,6 @@ class QueryEngineSpec extends FunSpec with Matchers {
// materialized exec plan
val execPlan = engine2.materialize(raw2, QueryOptions())
- // println(execPlan.printTree())
execPlan.isInstanceOf[DistConcatExec] shouldEqual true
execPlan.children.foreach { l1 =>
l1.isInstanceOf[SelectRawPartitionsExec] shouldEqual true
@@ -127,13 +125,17 @@ class QueryEngineSpec extends FunSpec with Matchers {
}
it("should use spread function to change/override spread and generate ExecPlan with appropriate shards") {
- val spreadFunc = QueryOptions.simpleMapSpreadFunc("job", Map("myService" -> 2), 1)
+ var filodbSpreadMap = new collection.mutable.HashMap[collection.Map[String, String], Int]
+ filodbSpreadMap.put(collection.Map(("job" -> "myService")), 2)
+
+ val spreadFunc = QueryOptions.simpleMapSpreadFunc("job", filodbSpreadMap, 1)
// final logical plan
val logicalPlan = BinaryJoin(summed1, BinaryOperator.DIV, Cardinality.OneToOne, summed2)
// materialized exec plan
- val execPlan = engine.materialize(logicalPlan, QueryOptions(FunctionalSpreadProvider(spreadFunc)))
+ val execPlan = engine.materialize(logicalPlan, QueryOptions(), FunctionalSpreadProvider(spreadFunc))
+ execPlan.printTree()
execPlan.isInstanceOf[BinaryJoinExec] shouldEqual true
execPlan.children should have length (2)
@@ -148,7 +150,7 @@ class QueryEngineSpec extends FunSpec with Matchers {
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(FunctionalSpreadProvider(spread)))
+ val execPlan = engine.materialize(lp, QueryOptions(), FunctionalSpreadProvider(spread))
execPlan.rangeVectorTransformers.head.isInstanceOf[StitchRvsMapper] shouldEqual true
}
@@ -157,7 +159,7 @@ class QueryEngineSpec extends FunSpec with Matchers {
def spread(filter: Seq[ColumnFilter]): Seq[SpreadChange] = {
Seq(SpreadChange(0, 1), SpreadChange(35000000, 2))
}
- val execPlan = engine.materialize(lp, QueryOptions(FunctionalSpreadProvider(spread)))
+ val execPlan = engine.materialize(lp, QueryOptions(), FunctionalSpreadProvider(spread))
execPlan.rangeVectorTransformers.isEmpty shouldEqual true
}
@@ -167,7 +169,7 @@ class QueryEngineSpec extends FunSpec with Matchers {
def spread(filter: Seq[ColumnFilter]): Seq[SpreadChange] = {
Seq(SpreadChange(0, 1), SpreadChange(25000000, 2))
}
- val execPlan = engine.materialize(lp, QueryOptions(FunctionalSpreadProvider(spread)))
+ val execPlan = engine.materialize(lp, QueryOptions(), FunctionalSpreadProvider(spread))
val binaryJoinNode = execPlan.children(0)
binaryJoinNode.isInstanceOf[BinaryJoinExec] shouldEqual true
binaryJoinNode.children.size shouldEqual 2
@@ -180,7 +182,7 @@ class QueryEngineSpec extends FunSpec with Matchers {
def spread(filter: Seq[ColumnFilter]): Seq[SpreadChange] = {
Seq(SpreadChange(0, 1), SpreadChange(35000000, 2))
}
- val execPlan = engine.materialize(lp, QueryOptions(FunctionalSpreadProvider(spread)))
+ val execPlan = engine.materialize(lp, QueryOptions(), FunctionalSpreadProvider(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/resources/filodb-defaults.conf b/core/src/main/resources/filodb-defaults.conf
index 3b24108270..0acdfdfa04 100644
--- a/core/src/main/resources/filodb-defaults.conf
+++ b/core/src/main/resources/filodb-defaults.conf
@@ -25,7 +25,11 @@ filodb {
}
# # of shards each application/metric is spread out to = 2^spread
- default-spread = 1
+ spread-default = 1
+ # default spread can be overriden for a specific sharding key combination.
+ # Eg: If "__name__, _ns" are your sharding key, for a _ns "App-Test001" the spread can be overriden as follows:
+ # spread-assignment = [ { _ns = App-Test001, _spread_ = 5 } ]
+ spread-assignment = []
query {
# Timeout for query engine subtree/ExecPlans for requests to sub nodes
@@ -41,6 +45,11 @@ filodb {
# Parallelism (query threadpool per dataset) ... ceil(available processors * factor)
threads-factor = 1.0
+
+ # Maximum number of steps/windows to use the RangeVectorAggregator.fastReduce aggregators. This aggregator
+ # uses memory proportional to the # of windows, rather than the # of time series aggregated; it can speed up
+ # high cardinality aggregations in particular.
+ fastreduce-max-windows = 50
}
shard-manager {
@@ -202,8 +211,8 @@ akka {
# For details of kryo section see https://github.com/romix/akka-kryo-serialization
kryo {
- # TODO: turn this off once finished debugging Kryo classes for serialization
- implicit-registration-logging = "true"
+ implicit-registration-logging = "false"
+ kryo-trace = "false"
kryo-custom-serializer-init = "filodb.coordinator.client.KryoInit"
@@ -333,3 +342,56 @@ kamon.zipkin {
max.requests = 128
message.max.bytes = 262144
}
+
+kamon {
+ prometheus.buckets {
+ # Have more buckets, better resolution really helps esp with heatmaps
+ default-buckets = [
+ 4,
+ 8,
+ 16,
+ 32,
+ 64,
+ 128,
+ 256,
+ 512,
+ 1024,
+ 2048,
+ 4096,
+ 8192,
+ 16384,
+ 32768,
+ 65536,
+ 131072,
+ 262144,
+ 524288
+ ]
+
+ # Start at 0.01ms so we can measure smaller timings
+ time-buckets = [
+ 0.00001,
+ 0.000025,
+ 0.00005,
+ 0.0001,
+ 0.00025,
+ 0.0005,
+ 0.001,
+ 0.0025,
+ 0.005,
+ 0.01,
+ 0.025,
+ 0.05,
+ 0.1,
+ 0.25,
+ 0.5,
+ 1,
+ 2.5,
+ 5,
+ 10,
+ 25,
+ 50,
+ 100,
+ 250
+ ]
+ }
+}
diff --git a/core/src/main/scala/filodb.core/Iterators.scala b/core/src/main/scala/filodb.core/Iterators.scala
index 2523c2d64a..d7fb8ad6b3 100644
--- a/core/src/main/scala/filodb.core/Iterators.scala
+++ b/core/src/main/scala/filodb.core/Iterators.scala
@@ -30,7 +30,7 @@ object Iterators extends StrictLogging {
def next: (B, Iterator[T]) = {
val first = iter.next()
val firstValue = func(first)
- val (i1,i2) = iter.span(el => func(el) == firstValue)
+ val (i1, i2) = iter.span(el => func(el) == firstValue)
iter = i2
(firstValue, Iterator.single(first) ++ i1)
}
diff --git a/core/src/main/scala/filodb.core/binaryrecord2/RecordContainer.scala b/core/src/main/scala/filodb.core/binaryrecord2/RecordContainer.scala
index 95e6ce77e5..5732610db2 100644
--- a/core/src/main/scala/filodb.core/binaryrecord2/RecordContainer.scala
+++ b/core/src/main/scala/filodb.core/binaryrecord2/RecordContainer.scala
@@ -27,7 +27,7 @@ final class RecordContainer(val base: Any, val offset: Long, maxLength: Int,
var numRecords: Int = 0) {
import RecordBuilder._
- @inline final def numBytes: Int = UnsafeUtils.getInt(base, offset)
+ @inline final def numBytes: Int = UnsafeUtils.getInt(base, offset)
@inline final def isEmpty: Boolean = numBytes <= 4
/**
diff --git a/core/src/main/scala/filodb.core/binaryrecord2/RecordSchema.scala b/core/src/main/scala/filodb.core/binaryrecord2/RecordSchema.scala
index a06c58131e..fb1108e78f 100644
--- a/core/src/main/scala/filodb.core/binaryrecord2/RecordSchema.scala
+++ b/core/src/main/scala/filodb.core/binaryrecord2/RecordSchema.scala
@@ -198,15 +198,15 @@ final class RecordSchema(val columns: Seq[ColumnInfo],
import Column.ColumnType._
val result = new ArrayBuffer[String]()
columnTypes.zipWithIndex.map {
- case (IntColumn, i) => result += s"${colNames(i)}= ${getInt(base, offset, i)}"
- case (LongColumn, i) => result += s"${colNames(i)}= ${getLong(base, offset, i)}"
- 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 (IntColumn, i) => result += s"${colNames(i)}=${getInt(base, offset, i)}"
+ case (LongColumn, i) => result += s"${colNames(i)}=${getLong(base, offset, i)}"
+ 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 (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)}"
+ 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))}"
}
diff --git a/core/src/main/scala/filodb.core/memstore/MemStore.scala b/core/src/main/scala/filodb.core/memstore/MemStore.scala
index 29261d29c8..0aa53e37f6 100644
--- a/core/src/main/scala/filodb.core/memstore/MemStore.scala
+++ b/core/src/main/scala/filodb.core/memstore/MemStore.scala
@@ -127,6 +127,8 @@ trait MemStore extends ChunkSource {
def recoverStream(dataset: DatasetRef,
shard: Int,
stream: Observable[SomeData],
+ startOffset: Long,
+ endOffset: Long,
checkpoints: Map[Int, Long],
reportingInterval: Long): Observable[Long]
diff --git a/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala b/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala
index 5787ee7996..56e3a9d937 100644
--- a/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala
+++ b/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala
@@ -15,6 +15,7 @@ import filodb.core.downsample.{DownsampleConfig, DownsamplePublisher}
import filodb.core.metadata.Dataset
import filodb.core.store._
import filodb.memory.BinaryRegionLarge
+import filodb.memory.MemFactory
import filodb.memory.format.UnsafeUtils
/**
@@ -24,13 +25,14 @@ import filodb.memory.format.UnsafeUtils
class OnDemandPagingShard(dataset: Dataset,
storeConfig: StoreConfig,
shardNum: Int,
+ bufferMemoryManager: MemFactory,
rawStore: ColumnStore,
metastore: MetaStore,
evictionPolicy: PartitionEvictionPolicy,
downsampleConfig: DownsampleConfig,
downsamplePublisher: DownsamplePublisher)
(implicit ec: ExecutionContext) extends
-TimeSeriesShard(dataset, storeConfig, shardNum, rawStore, metastore, evictionPolicy,
+TimeSeriesShard(dataset, storeConfig, shardNum, bufferMemoryManager, rawStore, metastore, evictionPolicy,
downsampleConfig, downsamplePublisher)(ec) {
import TimeSeriesShard._
@@ -80,7 +82,7 @@ TimeSeriesShard(dataset, storeConfig, shardNum, rawStore, metastore, evictionPol
} 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} " +
+ logger.warn(s"Skipped ODP of partId=$pId in dataset=${dataset.ref} " +
s"shard=$shardNum since we are very likely thrashing")
}
}
@@ -168,6 +170,7 @@ TimeSeriesShard(dataset, storeConfig, shardNum, rawStore, metastore, evictionPol
part <- Option(createNewPartition(partKeyBytesRef.bytes, unsafeKeyOffset, group, id, 4)) } yield {
val stamp = partSetLock.writeLock()
try {
+ part.ingesting = false
partSet.add(part)
} finally {
partSetLock.unlockWrite(stamp)
diff --git a/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala b/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala
index c9a943d9ad..1039d1cead 100644
--- a/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala
+++ b/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala
@@ -8,6 +8,7 @@ import scala.collection.immutable.HashSet
import com.googlecode.javaewah.{EWAHCompressedBitmap, IntIterator}
import com.typesafe.scalalogging.StrictLogging
+import java.util
import kamon.Kamon
import org.apache.lucene.analysis.standard.StandardAnalyzer
import org.apache.lucene.document._
@@ -31,10 +32,10 @@ import filodb.memory.{BinaryRegionLarge, UTF8StringMedium}
import filodb.memory.format.{UnsafeUtils, ZeroCopyUTF8String => UTF8Str}
object PartKeyLuceneIndex {
- final val PART_ID = "__partId__"
+ final val PART_ID = "__partId__"
final val START_TIME = "__startTime__"
- final val END_TIME = "__endTime__"
- final val PART_KEY = "__partKey__"
+ final val END_TIME = "__endTime__"
+ final val PART_KEY = "__partKey__"
final val ignoreIndexNames = HashSet(START_TIME, PART_KEY, END_TIME, PART_ID)
@@ -147,21 +148,28 @@ class PartKeyLuceneIndex(dataset: Dataset,
logger.info(s"Started flush thread for lucene index on dataset=${dataset.ref} shard=$shardNum")
}
- def removeEntries(prunedPartitions: EWAHCompressedBitmap): Unit = {
- val deleteQuery = IntPoint.newSetQuery(PartKeyLuceneIndex.PART_ID, prunedPartitions.toList)
- indexWriter.deleteDocuments(deleteQuery)
- }
-
/**
- * Use to delete partitions that were ingesting before retention period
- * @return partIds of deleted partitions
+ * Find partitions that ended ingesting before a given timestamp. Used to identify partitions that can be purged.
+ * @return matching partIds
*/
- def removePartKeysEndedBefore(endedBefore: Long): IntIterator = {
+ def partIdsEndedBefore(endedBefore: Long): EWAHCompressedBitmap = {
val collector = new PartIdCollector()
val deleteQuery = LongPoint.newRangeQuery(PartKeyLuceneIndex.END_TIME, 0, endedBefore)
searcherManager.acquire().search(deleteQuery, collector)
- indexWriter.deleteDocuments(deleteQuery)
- collector.intIterator()
+ collector.result
+ }
+
+ /**
+ * Delete partitions with given partIds
+ */
+ def removePartKeys(partIds: EWAHCompressedBitmap): Unit = {
+ val terms = new util.ArrayList[BytesRef]()
+ val iter = partIds.intIterator()
+ while (iter.hasNext) {
+ val pId = iter.next()
+ terms.add(new BytesRef(pId.toString.getBytes))
+ }
+ indexWriter.deleteDocuments(new TermInSetQuery(PART_ID, terms))
}
def indexRamBytes: Long = indexWriter.ramBytesUsed()
@@ -265,10 +273,10 @@ class PartKeyLuceneIndex(dataset: Dataset,
partKeyOnHeapBytes: Array[Byte],
partKeyBytesRefOffset: Int = 0): String = {
//scalastyle:off
- s"sh$shardNum-pId$partId[${
+ s"shard=$shardNum partId=$partId [${
TimeSeriesPartition
.partKeyString(dataset, partKeyOnHeapBytes, bytesRefToUnsafeOffset(partKeyBytesRefOffset))
- }"
+ }]"
//scalastyle:on
}
@@ -330,13 +338,13 @@ class PartKeyLuceneIndex(dataset: Dataset,
.withTag("shard", shardNum)
.start()
val collector = new PartIdStartTimeCollector()
- partIds.grouped(512).foreach { batch => // limit on query clause count is 1024, hence batch
- val booleanQuery = new BooleanQuery.Builder
- batch.foreach { pId =>
- booleanQuery.add(new TermQuery(new Term(PART_ID, pId.toString)), Occur.SHOULD)
- }
- searcherManager.acquire().search(booleanQuery.build(), collector)
+ val terms = new util.ArrayList[BytesRef]()
+ partIds.foreach { pId =>
+ terms.add(new BytesRef(pId.toString.getBytes))
}
+ // dont use BooleanQuery which will hit the 1024 term limit. Instead use TermInSetQuery which is
+ // more efficient within Lucene
+ searcherManager.acquire().search(new TermInSetQuery(PART_ID, terms), collector)
span.finish()
collector.startTimes
}
@@ -378,8 +386,8 @@ class PartKeyLuceneIndex(dataset: Dataset,
var startTime = startTimeFromPartId(partId) // look up index for old start time
if (startTime == NOT_FOUND) {
startTime = System.currentTimeMillis() - storeConfig.demandPagedRetentionPeriod.toMillis
- logger.warn(s"Could not find in Lucene startTime for partId $partId. Using $startTime instead.",
- new IllegalStateException()) // assume this time series started retention period ago
+ logger.warn(s"Could not find in Lucene startTime for partId=$partId in dataset=${dataset.ref}. Using " +
+ s"$startTime instead.", new IllegalStateException()) // assume this time series started retention period ago
}
val updatedDoc = makeDocument(partKeyOnHeapBytes, partKeyBytesRefOffset, partKeyNumBytes,
partId, startTime, endTime)
@@ -394,6 +402,7 @@ class PartKeyLuceneIndex(dataset: Dataset,
*/
def commitBlocking(): Unit = {
searcherManager.maybeRefreshBlocking()
+ logger.info("Refreshed index searchers to make reads consistent")
}
private def leafFilter(column: String, filter: Filter): Query = {
diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala
index cdf573c5b1..6a44b96fc7 100644
--- a/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala
+++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala
@@ -14,7 +14,9 @@ import filodb.core.downsample.{DownsampleConfig, DownsamplePublisher}
import filodb.core.metadata.Dataset
import filodb.core.query.ColumnFilter
import filodb.core.store._
-import filodb.memory.format.ZeroCopyUTF8String
+import filodb.memory.MemFactory
+import filodb.memory.NativeMemoryManager
+import filodb.memory.format.{UnsafeUtils, ZeroCopyUTF8String}
class TimeSeriesMemStore(config: Config, val store: ColumnStore, val metastore: MetaStore,
evictionPolicy: Option[PartitionEvictionPolicy] = None)
@@ -24,6 +26,7 @@ extends MemStore with StrictLogging {
type Shards = NonBlockingHashMapLong[TimeSeriesShard]
private val datasets = new HashMap[DatasetRef, Shards]
+ private val datasetMemFactories = new HashMap[DatasetRef, MemFactory]
/**
* The Downsample Publisher is per dataset on the memstore and is shared among all shards of the dataset
@@ -51,8 +54,15 @@ extends MemStore with StrictLogging {
if (shards.containsKey(shard)) {
throw ShardAlreadySetup(dataset.ref, shard)
} else {
+ val memFactory = datasetMemFactories.getOrElseUpdate(dataset.ref, {
+ val bufferMemorySize = storeConf.ingestionBufferMemSize
+ logger.info(s"Allocating $bufferMemorySize bytes for WriteBufferPool/PartitionKeys for dataset=${dataset.ref}")
+ val tags = Map("dataset" -> dataset.ref.toString)
+ new NativeMemoryManager(bufferMemorySize, tags)
+ })
+
val publisher = downsamplePublishers.getOrElseUpdate(dataset.ref, makeAndStartPublisher(downsample))
- val tsdb = new OnDemandPagingShard(dataset, storeConf, shard, store, metastore,
+ val tsdb = new OnDemandPagingShard(dataset, storeConf, shard, memFactory, store, metastore,
partEvictionPolicy, downsample, publisher)
shards.put(shard, tsdb)
}
@@ -130,15 +140,22 @@ extends MemStore with StrictLogging {
def recoverStream(dataset: DatasetRef,
shardNum: Int,
stream: Observable[SomeData],
+ startOffset: Long,
+ endOffset: Long,
checkpoints: Map[Int, Long],
reportingInterval: Long): Observable[Long] = {
val shard = getShardE(dataset, shardNum)
shard.setGroupWatermarks(checkpoints)
- var targetOffset = checkpoints.values.min + reportingInterval
- stream.map(shard.ingest(_)).collect {
- case offset: Long if offset > targetOffset =>
- targetOffset += reportingInterval
- offset
+ if (endOffset < startOffset) Observable.empty
+ else {
+ var targetOffset = startOffset + reportingInterval
+ stream.map(shard.ingest(_)).collect {
+ case offset: Long if offset >= endOffset => // last offset reached
+ offset
+ case offset: Long if offset > targetOffset => // reporting interval reached
+ targetOffset += reportingInterval
+ offset
+ }
}
}
@@ -174,8 +191,15 @@ extends MemStore with StrictLogging {
def scanPartitions(dataset: Dataset,
columnIDs: Seq[Types.ColumnId],
partMethod: PartitionScanMethod,
- chunkMethod: ChunkScanMethod = AllChunkScan): Observable[ReadablePartition] =
- datasets(dataset.ref).get(partMethod.shard).scanPartitions(columnIDs, partMethod, chunkMethod)
+ chunkMethod: ChunkScanMethod = AllChunkScan): Observable[ReadablePartition] = {
+ val shard = datasets(dataset.ref).get(partMethod.shard)
+
+ if (shard == UnsafeUtils.ZeroPointer) {
+ throw new IllegalArgumentException(s"Shard ${partMethod.shard} of dataset ${dataset.ref} is not assigned to " +
+ s"this node. Was it was recently reassigned to another node? Prolonged occurrence indicates an issue.")
+ }
+ shard.scanPartitions(columnIDs, partMethod, chunkMethod)
+ }
def numRowsIngested(dataset: DatasetRef, shard: Int): Long =
getShard(dataset, shard).map(_.numRowsIngested).getOrElse(-1L)
diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala
index 34bd7ce270..24c2fc23b4 100644
--- a/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala
+++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala
@@ -78,8 +78,15 @@ extends ChunkMap(memFactory, initMapSize) with ReadablePartition {
* and new chunk is added to the partition.
* Note that if this is not NULL, then it is always the most recent element of infoMap.
*/
- private var currentChunks = nullChunks
- private var currentInfo = nullInfo
+ protected var currentChunks = nullChunks
+ protected var currentInfo = nullInfo
+
+ /**
+ * True if partition is actively ingesting.
+ * This flag is maintained in addition to the activelyIngesting bitmap maintained in
+ * TimeSeriesShard because this flag is checked for each sample and bitmap.get is not fast.
+ */
+ @volatile var ingesting: Boolean = false
/**
* The newest ChunkID that has been flushed or encoded. You can think of the progression of chunks like this,
@@ -109,7 +116,7 @@ extends ChunkMap(memFactory, initMapSize) with ReadablePartition {
*
* @param blockHolder the BlockMemFactory to use for encoding chunks in case of WriteBuffer overflow
*/
- final def ingest(row: RowReader, blockHolder: BlockMemFactory): Unit = {
+ def ingest(row: RowReader, blockHolder: BlockMemFactory): Unit = {
// NOTE: lastTime is not persisted for recovery. Thus the first sample after recovery might still be out of order.
val ts = dataset.timestamp(row)
if (ts < timestampOfLatestSample) {
@@ -118,17 +125,7 @@ extends ChunkMap(memFactory, initMapSize) with ReadablePartition {
}
val newChunk = currentChunks == nullChunks
- if (newChunk) {
- // First row of a chunk, set the start time to it
- val (infoAddr, newAppenders) = bufferPool.obtain()
- val currentChunkID = newChunkID(ts)
- ChunkSetInfo.setChunkID(infoAddr, currentChunkID)
- ChunkSetInfo.resetNumRows(infoAddr) // Must reset # rows otherwise it keeps increasing!
- ChunkSetInfo.setStartTime(infoAddr, ts)
- currentInfo = ChunkSetInfo(infoAddr)
- currentChunks = newAppenders
- // Don't publish the new chunk just yet. Wait until it has one row.
- }
+ if (newChunk) initNewChunk(ts)
for { col <- 0 until numColumns optimized } {
currentChunks(col).addFromReaderNoNA(row, col) match {
@@ -137,7 +134,7 @@ extends ChunkMap(memFactory, initMapSize) with ReadablePartition {
// 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) }
+ 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
@@ -155,6 +152,18 @@ extends ChunkMap(memFactory, initMapSize) with ReadablePartition {
}
}
+ protected def initNewChunk(ts: Long): Unit = {
+ // First row of a chunk, set the start time to it
+ val (infoAddr, newAppenders) = bufferPool.obtain()
+ val currentChunkID = chunkID(ts)
+ ChunkSetInfo.setChunkID(infoAddr, currentChunkID)
+ ChunkSetInfo.resetNumRows(infoAddr) // Must reset # rows otherwise it keeps increasing!
+ ChunkSetInfo.setStartTime(infoAddr, ts)
+ currentInfo = ChunkSetInfo(infoAddr)
+ currentChunks = newAppenders
+ // Don't publish the new chunk just yet. Wait until it has one row.
+ }
+
private def nonEmptyWriteBuffers: Boolean = currentInfo != nullInfo && currentInfo.numRows > 0
/**
@@ -165,7 +174,7 @@ 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): Boolean =
+ def switchBuffers(blockHolder: BlockMemFactory, encode: Boolean = false): Boolean =
nonEmptyWriteBuffers && {
val oldInfo = currentInfo
val oldAppenders = currentChunks
@@ -364,7 +373,7 @@ extends ChunkMap(memFactory, initMapSize) with ReadablePartition {
// Atomic and multi-thread safe; only mutates state if chunkID not present
final def addChunkInfoIfAbsent(id: ChunkID, infoAddr: BinaryRegion.NativePointer): Boolean = {
chunkmapWithExclusive({
- val inserted = chunkmapDoPutIfAbsent(infoAddr)
+ val inserted = chunkmapDoPutIfAbsent(infoAddr, newestFlushedID)
// Make sure to update newestFlushedID so that flushes work correctly and don't try to flush these chunksets
if (inserted) updateFlushedID(infoGet(id))
inserted
@@ -382,7 +391,7 @@ extends ChunkMap(memFactory, initMapSize) with ReadablePartition {
private[core] def infoLast(): ChunkSetInfo = ChunkSetInfo(chunkmapDoGetLast)
private def infoPut(info: ChunkSetInfo): Unit = {
- chunkmapWithExclusive(chunkmapDoPut(info.infoAddr))
+ chunkmapWithExclusive(chunkmapDoPut(info.infoAddr, newestFlushedID))
}
// Free memory (esp offheap) attached to this TSPartition and return buffers to common pool
@@ -392,6 +401,50 @@ extends ChunkMap(memFactory, initMapSize) with ReadablePartition {
}
}
+/**
+ * A variant of the above which logs every sample ingested and buffer switching/encoding event,
+ * for debugging purposes. See the trace-filters StoreConfig / ingestion config setting.
+ *
+ * NOTE(velvia): The reason why I used inheritance was not so much memory but just ease of implementation.
+ * With composition we'd need to add in tons of methods and clutter things up quite a bit. If it simply
+ * implemented ReadablePartition that might break things in a bunch of places.
+ * So best way to keep changes small and balance out different needs
+ */
+class TracingTimeSeriesPartition(partID: Int,
+ dataset: Dataset,
+ partitionKey: BinaryRegion.NativePointer,
+ shard: Int,
+ bufferPool: WriteBufferPool,
+ shardStats: TimeSeriesShardStats,
+ memFactory: MemFactory,
+ initMapSize: Int) extends
+TimeSeriesPartition(partID, dataset, partitionKey, shard, bufferPool, shardStats, memFactory, initMapSize) {
+ import TimeSeriesPartition._
+
+ _log.debug(s"Creating TracingTimeSeriesPartition: dataset=${dataset.ref} partId=$partID $stringPartition")
+
+ override def ingest(row: RowReader, blockHolder: BlockMemFactory): Unit = {
+ val ts = dataset.timestamp(row)
+ _log.debug(s"dataset=${dataset.ref} shard=$shard partId=$partID $stringPartition - ingesting ts=$ts " +
+ (1 until dataset.dataColumns.length).map(row.getAny).mkString("[", ",", "]"))
+ super.ingest(row, blockHolder)
+ }
+
+ override def switchBuffers(blockHolder: BlockMemFactory, encode: Boolean = false): Boolean = {
+ _log.debug(s"dataset=${dataset.ref} shard=$shard partId=$partID $stringPartition - switchBuffers, encode=$encode" +
+ s" for currentChunk ${currentInfo.debugString}")
+ super.switchBuffers(blockHolder, encode)
+ }
+
+ override protected def initNewChunk(ts: Long): Unit = {
+ _log.debug(s"dataset=${dataset.ref} shard=$shard partId=$partID $stringPartition - initNewChunk($ts)")
+ super.initNewChunk(ts)
+ _log.debug(s"dataset=${dataset.ref} shard=$shard partId=$partID $stringPartition - newly created ChunkInfo " +
+ s"${currentInfo.debugString}")
+ }
+}
+
+
final case class PartKeyRowReader(records: Iterator[TimeSeriesPartition]) extends Iterator[RowReader] {
var currVal: TimeSeriesPartition = _
diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala
index e32073b1b1..848e8bd73e 100644
--- a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala
+++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala
@@ -169,6 +169,8 @@ object PartitionIterator {
* for that group, up to what offset incoming records for that group has been persisted. At recovery time, records
* that fall below the watermark for that group will be skipped (since they can be recovered from disk).
*
+ * @param bufferMemoryManager Unencoded/unoptimized ingested data is stored in buffers that are allocated from this
+ * memory pool. This pool is also used to store partition keys.
* @param storeConfig the store portion of the sourceconfig, not the global FiloDB application config
* @param downsampleConfig configuration for downsample operations
* @param downsamplePublisher is shared among all shards of the dataset on the node
@@ -176,6 +178,7 @@ object PartitionIterator {
class TimeSeriesShard(val dataset: Dataset,
val storeConfig: StoreConfig,
val shardNum: Int,
+ val bufferMemoryManager: MemFactory,
colStore: ColumnStore,
metastore: MetaStore,
evictionPolicy: PartitionEvictionPolicy,
@@ -236,7 +239,6 @@ class TimeSeriesShard(val dataset: Dataset,
private val blockMemorySize = storeConfig.shardMemSize
protected val numGroups = storeConfig.groupsPerShard
- private val bufferMemorySize = storeConfig.ingestionBufferMemSize
private val chunkRetentionHours = (storeConfig.demandPagedRetentionPeriod.toSeconds / 3600).toInt
val pagingEnabled = storeConfig.demandPagingEnabled
@@ -262,19 +264,19 @@ class TimeSeriesShard(val dataset: Dataset,
private[core] val overflowBlockFactory = new BlockMemFactory(blockStore, None, dataset.blockMetaSize, true)
val partitionMaker = new DemandPagedChunkStore(this, blockStore, chunkRetentionHours)
- /**
- * Unencoded/unoptimized ingested data is stored in buffers that are allocated from this off-heap pool
- * Note that this pool is also used to store partition keys.
- */
- logger.info(s"Allocating $bufferMemorySize bytes for WriteBufferPool/PartitionKeys for " +
- s"dataset=${dataset.ref} shard=$shardNum")
- protected val bufferMemoryManager = new NativeMemoryManager(bufferMemorySize)
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)
+ private[memstore] val bufferPool = new WriteBufferPool(bufferMemoryManager, dataset, storeConfig)
private final val partitionGroups = Array.fill(numGroups)(new EWAHCompressedBitmap)
+
+ /**
+ * Bitmap to track actively ingesting partitions.
+ * This bitmap is maintained in addition to the ingesting flag per partition.
+ * TSP.ingesting is MUCH faster than bit.get(i) but we need the bitmap for faster operations
+ * for all partitions of shard (like ingesting cardinality counting, rollover of time buckets etc).
+ */
private final val activelyIngesting = new EWAHCompressedBitmap
private final val numTimeBucketsToRetain = Math.ceil(chunkRetentionHours.hours / storeConfig.flushInterval).toInt
@@ -331,6 +333,16 @@ class TimeSeriesShard(val dataset: Dataset,
dataset.partKeySchema.partitionHash(from.base, from.offset)
}
})
+ private var evictedPartKeysDisposed = false
+
+ /**
+ * Detailed filtered ingestion record logging. See "trace-filters" StoreConfig setting. Warning: may blow up
+ * logs, use at your own risk.
+ */
+ val tracedPartFilters =
+ storeConfig.traceFilters.toSeq
+ .map { case (k, v) => (dataset.partitionColumns.indexWhere(_.name == k), v) }
+ .filter { case (i, v) => i >= 0 && dataset.partitionColumns(i).columnType == ColumnType.StringColumn }
case class InMemPartitionIterator(intIt: IntIterator) extends PartitionIterator {
var nextPart = UnsafeUtils.ZeroPointer.asInstanceOf[TimeSeriesPartition]
@@ -388,7 +400,7 @@ class TimeSeriesShard(val dataset: Dataset,
// TODO:
// explore aligning index time buckets with chunks, and we can then
// remove this partition existence check per sample.
- val part: FiloPartition = getOrAddPartition(recBase, recOffset, group, ingestOffset)
+ val part: FiloPartition = getOrAddPartitionForIngestion(recBase, recOffset, group, ingestOffset)
if (part == OutOfMemPartition) { disableAddPartitions() }
} catch {
case e: OutOfOffheapMemoryException => disableAddPartitions()
@@ -436,11 +448,12 @@ class TimeSeriesShard(val dataset: Dataset,
val partIdMap = debox.Map.empty[BytesRef, Int]
val earliestTimeBucket = Math.max(0, currentIndexTimeBucket - numTimeBucketsToRetain)
- logger.info(s"Recovering timebuckets $earliestTimeBucket until $currentIndexTimeBucket " +
+ logger.info(s"Recovering timebuckets $earliestTimeBucket to ${currentIndexTimeBucket - 1} " +
s"for dataset=${dataset.ref} shard=$shardNum ")
// 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 {
+ // no need to go into currentIndexTimeBucket since it is not present in cass
+ val timeBuckets = for { tb <- currentIndexTimeBucket-1 to earliestTimeBucket by -1 } yield {
colStore.getPartKeyTimeBucket(dataset, shardNum, tb).map { b =>
new IndexData(tb, b.segmentId, RecordContainer(b.segment.array()))
}
@@ -490,6 +503,7 @@ class TimeSeriesShard(val dataset: Dataset,
val stamp = partSetLock.writeLock()
try {
partSet.add(part) // createNewPartition doesn't add part to partSet
+ part.ingesting = true
Some(part.partID)
} finally {
partSetLock.unlockWrite(stamp)
@@ -498,9 +512,12 @@ class TimeSeriesShard(val dataset: Dataset,
} 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)
+ // instead add the partition to evictedPArtKeys bloom filter so that it can be found if necessary
+ evictedPartKeys.synchronized {
+ require(!evictedPartKeysDisposed)
+ evictedPartKeys.add(PartKey(partKeyBaseOnHeap, partKeyOffset))
+ }
+ Some(createPartitionID())
}
// add newly assigned partId to lucene index
@@ -612,7 +629,7 @@ class TimeSeriesShard(val dataset: Dataset,
private def getPartitionFromPartId(partId: Int): TimeSeriesPartition = {
val nextPart = partitions.get(partId)
if (nextPart == UnsafeUtils.ZeroPointer)
- logger.warn(s"PartId $partId was not found in memory and was not included in metadata query result. ")
+ logger.warn(s"PartId=$partId was not found in memory and was not included in metadata query result. ")
// TODO Revisit this code for evicted partitions
/*if (nextPart == UnsafeUtils.ZeroPointer) {
val partKey = partKeyIndex.partKeyFromPartId(partId)
@@ -671,21 +688,27 @@ class TimeSeriesShard(val dataset: Dataset,
}
private def purgeExpiredPartitions(): Unit = ingestSched.executeTrampolined { () =>
- val deletedParts = partKeyIndex.removePartKeysEndedBefore(
+ val partsToPurge = partKeyIndex.partIdsEndedBefore(
System.currentTimeMillis() - storeConfig.demandPagedRetentionPeriod.toMillis)
var numDeleted = 0
- InMemPartitionIterator(deletedParts).foreach { p =>
- logger.debug(s"Purging partition with partId ${p.partID} from memory in dataset=${dataset.ref} shard=$shardNum")
- removePartition(p)
- numDeleted += 1
+ val removedParts = new EWAHCompressedBitmap()
+ InMemPartitionIterator(partsToPurge.intIterator()).foreach { p =>
+ if (!p.ingesting) {
+ logger.debug(s"Purging partition with partId=${p.partID} from memory in dataset=${dataset.ref} shard=$shardNum")
+ removePartition(p)
+ removedParts.set(p.partID)
+ numDeleted += 1
+ }
}
+ if (!removedParts.isEmpty) partKeyIndex.removePartKeys(removedParts)
if (numDeleted > 0) logger.info(s"Purged $numDeleted partitions from memory and " +
- s"index from dataset=${dataset.ref} shard=$shardNum")
+ s"index from dataset=${dataset.ref} shard=$shardNum")
shardStats.purgedPartitions.increment(numDeleted)
}
def createFlushTask(flushGroup: FlushGroup): Task[Response] = {
- val partitionIt = InMemPartitionIterator(partitionGroups(flushGroup.groupNum).intIterator)
+ // clone the bitmap so that reads on the flush thread do not conflict with writes on ingestion thread
+ val partitionIt = InMemPartitionIterator(partitionGroups(flushGroup.groupNum).clone().intIterator)
doFlushSteps(flushGroup, partitionIt)
}
@@ -696,13 +719,17 @@ class TimeSeriesShard(val dataset: Dataset,
shardStats.numPartitions.set(numActivePartitions)
val cardinality = activelyIngesting.synchronized { activelyIngesting.cardinality() }
shardStats.numActivelyIngestingParts.set(cardinality)
+
+ // Also publish MemFactory stats. Instance is expected to be shared, but no harm in
+ // publishing a little more often than necessary.
+ bufferMemoryManager.updateStats()
}
- private def addPartKeyToTimebucketRb(indexRb: RecordBuilder, p: TimeSeriesPartition) = {
+ private def addPartKeyToTimebucketRb(timebucketNum: Int, 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 == Long.MaxValue) startTime = 0 // if for any reason we cant find the startTime, use 0
- val endTime = if (isActivelyIngesting(p.partID)) {
+ val endTime = if (p.ingesting) {
Long.MaxValue
} else {
val et = p.timestampOfLatestSample // -1 can be returned if no sample after reboot
@@ -713,15 +740,11 @@ 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 RB partId ${p.partID} in dataset=${dataset.ref} shard=$shardNum " +
- s"partKey[${p.stringPartition}] with startTime=$startTime endTime=$endTime")
+ logger.debug(s"Added entry into timebucket=${timebucketNum} partId=${p.partID} in dataset=${dataset.ref} " +
+ s"shard=$shardNum partKey[${p.stringPartition}] with startTime=$startTime endTime=$endTime")
indexRb.endRecord(false)
}
- private def isActivelyIngesting(partID: Integer): Boolean = {
- activelyIngesting.synchronized { activelyIngesting.get(partID) }
- }
-
// scalastyle:off method.length
private def doFlushSteps(flushGroup: FlushGroup,
partitionIt: Iterator[TimeSeriesPartition]): Task[Response] = {
@@ -776,7 +799,7 @@ class TimeSeriesShard(val dataset: Dataset,
_.find(_.isInstanceOf[ErrorResponse]).getOrElse(Success)
}.flatMap {
case Success => blockHolder.markUsedBlocksReclaimable()
- commitCheckpoint(dataset.ref, shardNum, flushGroup)
+ commitCheckpoint(dataset.ref, shardNum, flushGroup)
case er: ErrorResponse => Future.successful(er)
}.recover { case e =>
logger.error(s"Internal Error when persisting chunks in dataset=${dataset.ref} shard=$shardNum - should " +
@@ -791,7 +814,9 @@ class TimeSeriesShard(val dataset: Dataset,
} catch { case e: Throwable =>
logger.error(s"Error when wrapping up doFlushSteps in dataset=${dataset.ref} shard=$shardNum", e)
}
- }
+ }(ingestSched)
+ // Note: The data structures accessed by flushDoneTasks can only be safely accessed by the
+ // ingestion thread, hence the onComplete steps are run from that thread.
Task.fromFuture(result)
}
@@ -843,7 +868,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 =>
- addPartKeyToTimebucketRb(timeBucketRb, p)
+ addPartKeyToTimebucketRb(cmd.timeBucket, timeBucketRb, p)
}
val numPartKeysInBucket = timeBucketBitmaps.get(cmd.timeBucket).cardinality()
logger.debug(s"Number of records in timebucket=${cmd.timeBucket} of " +
@@ -883,25 +908,21 @@ class TimeSeriesShard(val dataset: Dataset,
chunkSetIt: Iterator[ChunkSet],
partitionIt: Iterator[TimeSeriesPartition],
blockHolder: BlockMemFactory): Future[Response] = {
- if (chunkSetIt.isEmpty) {
- Future.successful(Success)
- } else {
- val chunkSetStream = Observable.fromIterator(chunkSetIt)
- logger.debug(s"Created flush ChunkSets stream for group ${flushGroup.groupNum} in " +
- s"dataset=${dataset.ref} shard=$shardNum")
-
- colStore.write(dataset, chunkSetStream, flushGroup.diskTimeToLiveSeconds).recover { case e =>
- logger.error(s"Critical! Chunk persistence failed after retries and skipped in dataset=${dataset.ref} " +
- s"shard=$shardNum", e)
- shardStats.flushesFailedChunkWrite.increment
-
- // Encode and free up the remainder of the WriteBuffers that have not been flushed yet. Otherwise they will
- // never be freed.
- partitionIt.foreach(_.encodeAndReleaseBuffers(blockHolder))
- // If the above futures fail with ErrorResponse because of DB failures, skip the chunk.
- // Sorry - need to drop the data to keep the ingestion moving
- DataDropped
- }
+ val chunkSetStream = Observable.fromIterator(chunkSetIt)
+ logger.debug(s"Created flush ChunkSets stream for group ${flushGroup.groupNum} in " +
+ s"dataset=${dataset.ref} shard=$shardNum")
+
+ colStore.write(dataset, chunkSetStream, flushGroup.diskTimeToLiveSeconds).recover { case e =>
+ logger.error(s"Critical! Chunk persistence failed after retries and skipped in dataset=${dataset.ref} " +
+ s"shard=$shardNum", e)
+ shardStats.flushesFailedChunkWrite.increment
+
+ // Encode and free up the remainder of the WriteBuffers that have not been flushed yet. Otherwise they will
+ // never be freed.
+ partitionIt.foreach(_.encodeAndReleaseBuffers(blockHolder))
+ // If the above futures fail with ErrorResponse because of DB failures, skip the chunk.
+ // Sorry - need to drop the data to keep the ingestion moving
+ DataDropped
}
}
@@ -920,19 +941,16 @@ class TimeSeriesShard(val dataset: Dataset,
private def updateIndexWithEndTime(p: TimeSeriesPartition,
partFlushChunks: Iterator[ChunkSet],
timeBucket: Int) = {
- // Synchronize for safe read-modify-write behavior.
+ // Below is coded to work concurrently with logic in getOrAddPartitionAndIngest
+ // where we try to activate an inactive time series
activelyIngesting.synchronized {
- if (partFlushChunks.isEmpty && activelyIngesting.get(p.partID)) {
+ if (partFlushChunks.isEmpty && p.ingesting) {
var endTime = p.timestampOfLatestSample
if (endTime == -1) endTime = System.currentTimeMillis() // this can happen if no sample after reboot
updatePartEndTimeInIndex(p, endTime)
timeBucketBitmaps.get(timeBucket).set(p.partID)
activelyIngesting.clear(p.partID)
- } else if (partFlushChunks.nonEmpty && !activelyIngesting.get(p.partID)) {
- // Partition started re-ingesting.
- updatePartEndTimeInIndex(p, Long.MaxValue)
- timeBucketBitmaps.get(timeBucket).set(p.partID)
- activelyIngesting.set(p.partID)
+ p.ingesting = false
}
}
}
@@ -965,10 +983,11 @@ class TimeSeriesShard(val dataset: Dataset,
private[memstore] val addPartitionsDisabled = AtomicBoolean(false)
// scalastyle:off null
- private[filodb] def getOrAddPartition(recordBase: Any, recordOff: Long, group: Int, ingestOffset: Long) = {
+ private[filodb] def getOrAddPartitionForIngestion(recordBase: Any, recordOff: Long,
+ group: Int, ingestOffset: Long) = {
var part = partSet.getWithIngestBR(recordBase, recordOff)
if (part == null) {
- part = addPartition(recordBase, recordOff, group)
+ part = addPartitionForIngestion(recordBase, recordOff, group)
}
part
}
@@ -980,7 +999,16 @@ class TimeSeriesShard(val dataset: Dataset,
*/
private def lookupPreviouslyAssignedPartId(partKeyBase: Array[Byte], partKeyOffset: Long): Int = {
shardStats.evictedPartKeyBloomFilterQueries.increment()
- if (evictedPartKeys.mightContain(PartKey(partKeyBase, partKeyOffset))) {
+
+ val mightContain = evictedPartKeys.synchronized {
+ if (!evictedPartKeysDisposed) {
+ evictedPartKeys.mightContain(PartKey(partKeyBase, partKeyOffset))
+ } else {
+ false
+ }
+ }
+
+ if (mightContain) {
val filters = dataset.partKeySchema.toStringPairs(partKeyBase, partKeyOffset)
.map { pair => ColumnFilter(pair._1, Filter.Equals(pair._2)) }
val matches = partKeyIndex.partIdsFromFilters2(filters, 0, Long.MaxValue)
@@ -999,8 +1027,9 @@ class TimeSeriesShard(val dataset: Dataset,
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)}")
+ logger.debug(s"There is already a partId=$partId assigned for " +
+ s"${dataset.partKeySchema.stringify(partKeyBase, partKeyOffset)} in" +
+ s" dataset=${dataset.ref} shard=$shardNum")
}
}
} while (iter.hasNext && partId != -1)
@@ -1018,7 +1047,7 @@ class TimeSeriesShard(val dataset: Dataset,
*
* This method also updates lucene index and time bucket bitmaps properly.
*/
- private def addPartition(recordBase: Any, recordOff: Long, group: Int) = {
+ private def addPartitionForIngestion(recordBase: Any, recordOff: Long, group: Int) = {
val partKeyOffset = recordComp.buildPartKeyFromIngest(recordBase, recordOff, partKeyBuilder)
val previousPartId = lookupPreviouslyAssignedPartId(partKeyArray, partKeyOffset)
val newPart = createNewPartition(partKeyArray, partKeyOffset, group, previousPartId)
@@ -1028,10 +1057,16 @@ class TimeSeriesShard(val dataset: Dataset,
val startTime = dataset.ingestionSchema.getLong(recordBase, recordOff, timestampColId)
if (previousPartId == CREATE_NEW_PARTID) {
// add new lucene entry if this partKey was never seen before
- partKeyIndex.addPartKey(newPart.partKeyBytes, partId, startTime)()
+ partKeyIndex.addPartKey(newPart.partKeyBytes, partId, startTime)() // causes endTime to be set to Long.MaxValue
+ } else {
+ // newly created partition is re-ingesting now, so update endTime
+ updatePartEndTimeInIndex(newPart, Long.MaxValue)
}
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
+ activelyIngesting.synchronized {
+ activelyIngesting.set(partId)
+ newPart.ingesting = true
+ }
val stamp = partSetLock.writeLock()
try {
partSet.add(newPart)
@@ -1053,16 +1088,38 @@ class TimeSeriesShard(val dataset: Dataset,
*/
def getOrAddPartitionAndIngest(recordBase: Any, recordOff: Long, group: Int, ingestOffset: Long): Unit =
try {
- val part: FiloPartition = getOrAddPartition(recordBase, recordOff, group, ingestOffset)
+ val part: FiloPartition = getOrAddPartitionForIngestion(recordBase, recordOff, group, ingestOffset)
if (part == OutOfMemPartition) { disableAddPartitions() }
- else { part.asInstanceOf[TimeSeriesPartition].ingest(binRecordReader, overflowBlockFactory) }
+ else {
+ val tsp = part.asInstanceOf[TimeSeriesPartition]
+ tsp.ingest(binRecordReader, overflowBlockFactory)
+ // Below is coded to work concurrently with logic in updateIndexWithEndTime
+ // where we try to de-activate an active time series
+ if (!tsp.ingesting) {
+ // DO NOT use activelyIngesting to check above condition since it is slow and is called for every sample
+ activelyIngesting.synchronized {
+ if (!tsp.ingesting) {
+ // time series was inactive and has just started re-ingesting
+ updatePartEndTimeInIndex(part.asInstanceOf[TimeSeriesPartition], Long.MaxValue)
+ timeBucketBitmaps.get(currentIndexTimeBucket).set(part.partID)
+ activelyIngesting.set(part.partID)
+ tsp.ingesting = true
+ }
+ }
+ }
+ }
} catch {
- case e: OutOfOffheapMemoryException => logger.error(s"Out of offheap memory in dataset=${dataset.ref} " +
- s"shard=$shardNum", e); disableAddPartitions()
+ case e: OutOfOffheapMemoryException => disableAddPartitions()
case e: Exception => logger.error(s"Unexpected ingestion err in dataset=${dataset.ref} " +
- s"shard=$shardNum", e); disableAddPartitions()
+ s"shard=$shardNum", e); disableAddPartitions()
}
+ private def shouldTrace(partKeyAddr: Long): Boolean = tracedPartFilters.nonEmpty && {
+ tracedPartFilters.forall { case (i, filtVal) =>
+ dataset.partKeySchema.asJavaString(UnsafeUtils.ZeroPointer, partKeyAddr, i) == filtVal
+ }
+ }
+
/**
* 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.
@@ -1079,18 +1136,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 = if (usePartId == CREATE_NEW_PARTID) {
- val id = nextPartitionID
- incrementPartitionID()
- id
- } else usePartId
- val newPart = new TimeSeriesPartition(
- partId, dataset, partKeyAddr, shardNum, bufferPool, shardStats, bufferMemoryManager, initMapSize)
+ val partId = if (usePartId == CREATE_NEW_PARTID) createPartitionID() else usePartId
+ val newPart = if (shouldTrace(partKeyAddr)) {
+ logger.debug(s"Adding tracing TSPartition dataset=${dataset.ref} shard=$shardNum group=$group partId=$partId")
+ new TracingTimeSeriesPartition(
+ partId, dataset, partKeyAddr, shardNum, bufferPool, shardStats, bufferMemoryManager, initMapSize)
+ } else {
+ 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
}
@@ -1108,23 +1165,32 @@ class TimeSeriesShard(val dataset: Dataset,
}
}
- // Ensures partition ID wraps around safely to 0, not negative numbers (which don't work with bitmaps)
- private def incrementPartitionID(): Unit = {
- nextPartitionID += 1
- if (nextPartitionID < 0) {
- nextPartitionID = 0
- logger.info(s"dataset=${dataset.ref} shard=$shardNum nextPartitionID has wrapped around to 0 again")
- }
- // Given that we have 2^31 range of partitionIDs, we should pretty much never run into this problem where
- // we wraparound and hit a previously used partitionID. Actually dealing with this is not easy; what if
- // the used one is still actively ingesting? We need to solve the issue of evicting actively ingesting
- // partitions first. For now, assert so at least we will catch this condition.
- require(!partitions.containsKey(nextPartitionID), s"Partition ID $nextPartitionID ran into existing partition" +
- s"in dataset=${dataset.ref} shard=$shardNum")
+ /**
+ * Returns a new non-negative partition ID which isn't used by any existing parition. A negative
+ * partition ID wouldn't work with bitmaps.
+ */
+ private def createPartitionID(): Int = {
+ val id = nextPartitionID
+
+ // It's unlikely that partition IDs will wrap around, and it's unlikely that collisions
+ // will be encountered. In case either of these conditions occur, keep incrementing the id
+ // until no collision is detected. A given shard is expected to support up to 1M actively
+ // ingesting partitions, and so in the worst case, the loop might run for up to ~100ms.
+ // Afterwards, a complete wraparound is required for collisions to be detected again.
+
+ do {
+ nextPartitionID += 1
+ if (nextPartitionID < 0) {
+ nextPartitionID = 0
+ logger.info(s"dataset=${dataset.ref} shard=$shardNum nextPartitionID has wrapped around to 0 again")
+ }
+ } while (partitions.containsKey(nextPartitionID))
+
+ id
}
/**
- * Check and evict partitions to free up memory and heap space. NOTE: This should be called in the ingestion
+ * Check and evict partitions to free up memory and heap space. NOTE: This must be called in the ingestion
* stream so that there won't be concurrent other modifications. Ideally this is called when trying to add partitions
* @return true if able to evict enough or there was already space, false if not able to evict and not enough mem
*/
@@ -1146,48 +1212,56 @@ class TimeSeriesShard(val dataset: Dataset,
partitionGroups(group) = partitionGroups(group).andNot(prunedPartitions)
}
- // Be sure the below executes in the ingestion thread for mutation safety
- ingestSched.executeTrampolined { () =>
- // Finally, prune partitions and keyMap data structures
- logger.info(s"Evicting partitions from dataset=${dataset.ref} shard=$shardNum, watermark=$evictionWatermark...")
- val intIt = prunedPartitions.intIterator
- var partsRemoved = 0
- var partsSkipped = 0
- var maxEndTime = evictionWatermark
- while (intIt.hasNext) {
- val partitionObj = partitions.get(intIt.next)
- if (partitionObj != UnsafeUtils.ZeroPointer) {
- // TODO we can optimize fetching of endTime by getting them along with top-k query
- val endTime = partKeyIndex.endTimeFromPartId(partitionObj.partID)
- 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())
- } 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)
- }
+ // Finally, prune partitions and keyMap data structures
+ logger.info(s"Evicting partitions from dataset=${dataset.ref} shard=$shardNum, watermark=$evictionWatermark...")
+ val intIt = prunedPartitions.intIterator
+ var partsRemoved = 0
+ var partsSkipped = 0
+ var maxEndTime = evictionWatermark
+ while (intIt.hasNext) {
+ val partitionObj = partitions.get(intIt.next)
+ if (partitionObj != UnsafeUtils.ZeroPointer) {
+ // TODO we can optimize fetching of endTime by getting them along with top-k query
+ val endTime = partKeyIndex.endTimeFromPartId(partitionObj.partID)
+ if (partitionObj.ingesting)
+ 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())
} else {
- partsSkipped += 1
+ 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.synchronized {
+ if (!evictedPartKeysDisposed) {
+ 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)
}
+ } else {
+ partsSkipped += 1
+ }
+ }
+ val elemCount = evictedPartKeys.synchronized {
+ if (!evictedPartKeysDisposed) {
+ evictedPartKeys.approximateElementCount()
+ } else {
+ 0
}
- 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,
- // we would be processing same partIds again and again without moving watermark forward.
- // We may skip evicting some partitions by doing this, but the imperfection is an acceptable
- // trade-off for performance and simplicity. The skipped partitions, will ve removed during purge.
- logger.info(s"dataset=${dataset.ref} shard=$shardNum: evicted $partsRemoved partitions," +
- s"skipped $partsSkipped, h20=$evictionWatermark")
- shardStats.partitionsEvicted.increment(partsRemoved)
}
+ shardStats.evictedPkBloomFilterSize.set(elemCount)
+ 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,
+ // we would be processing same partIds again and again without moving watermark forward.
+ // We may skip evicting some partitions by doing this, but the imperfection is an acceptable
+ // trade-off for performance and simplicity. The skipped partitions, will ve removed during purge.
+ logger.info(s"dataset=${dataset.ref} shard=$shardNum: evicted $partsRemoved partitions," +
+ s"skipped $partsSkipped, h20=$evictionWatermark")
+ shardStats.partitionsEvicted.increment(partsRemoved)
}
true
}
@@ -1305,12 +1379,16 @@ class TimeSeriesShard(val dataset: Dataset,
}
def shutdown(): Unit = {
- evictedPartKeys.dispose()
+ evictedPartKeys.synchronized {
+ if (!evictedPartKeysDisposed) {
+ evictedPartKeysDisposed = true
+ 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
method to ensure that no threads are accessing the memory before it's freed.
- bufferMemoryManager.shutdown()
blockStore.releaseBlocks()
*/
}
diff --git a/core/src/main/scala/filodb.core/memstore/WriteBufferPool.scala b/core/src/main/scala/filodb.core/memstore/WriteBufferPool.scala
index 67ce93b065..06ad2689b8 100644
--- a/core/src/main/scala/filodb.core/memstore/WriteBufferPool.scala
+++ b/core/src/main/scala/filodb.core/memstore/WriteBufferPool.scala
@@ -1,6 +1,7 @@
package filodb.core.memstore
import com.typesafe.scalalogging.StrictLogging
+import org.jctools.queues.MpscUnboundedArrayQueue
import scalaxy.loops._
import filodb.core.metadata.Dataset
@@ -8,6 +9,14 @@ import filodb.core.store.{ChunkSetInfo, StoreConfig}
import filodb.memory.BinaryRegion.NativePointer
import filodb.memory.MemFactory
+object WriteBufferPool {
+ /**
+ * Number of WriteBuffers to allocate at once. Usually no reason to change it.
+ * Higher number means higher latency during allocation, but more buffers can be individually allocated.
+ */
+ val AllocStepSize = 200
+}
+
/**
* 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
@@ -21,34 +30,33 @@ import filodb.memory.MemFactory
* 2. End of flush() - original buffers, now encoded, are released, reset, and can be made available to others
*
* @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,
storeConf: StoreConfig) extends StrictLogging {
import TimeSeriesPartition._
+ import WriteBufferPool._
- val queue = new collection.mutable.Queue[(NativePointer, AppenderArray)]
+ val queue = new MpscUnboundedArrayQueue[(NativePointer, AppenderArray)](storeConf.maxBufferPoolSize)
private def allocateBuffers(): Unit = {
- logger.debug(s"Allocating ${storeConf.allocStepSize} WriteBuffers....")
+ logger.debug(s"Allocating ${AllocStepSize} WriteBuffers....")
// Fill queue up
- (0 until storeConf.allocStepSize).foreach { n =>
+ (0 until 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 } {
ChunkSetInfo.setVectorPtr(info.infoAddr, colNo, builders(colNo).addr)
}
- queue.enqueue((info.infoAddr, builders))
+ queue.add((info.infoAddr, builders))
}
}
/**
* Returns the number of allocatable sets of buffers in the pool
*/
- def poolSize: Int = queue.length
+ def poolSize: Int = queue.size
/**
* Obtains a new set of AppendableVectors from the pool, creating additional buffers if there is memory available.
@@ -59,7 +67,7 @@ class WriteBufferPool(memFactory: MemFactory,
def obtain(): (NativePointer, AppenderArray) = {
// If queue is empty, try and allocate more buffers depending on if memFactory has more memory
if (queue.isEmpty) allocateBuffers()
- queue.dequeue
+ queue.remove()
}
/**
@@ -67,13 +75,18 @@ class WriteBufferPool(memFactory: MemFactory,
* The state of the appenders are reset.
*/
def release(metaAddr: NativePointer, appenders: AppenderArray): Unit = {
- // IMPORTANT: reset size in ChunkSetInfo metadata so there won't be an inconsistency between appenders and metadata
- // (in case some reader is still hanging on to this old info)
- ChunkSetInfo.resetNumRows(metaAddr)
- appenders.foreach(_.reset())
- queue.enqueue((metaAddr, appenders))
- // TODO: check number of buffers in queue, and release baack to free memory.
- // NOTE: no point to this until the pool shares a single MemFactory amongst multiple shards. In that case
- // we have to decide (w/ concurrency a concern): share a single MemFactory or a single WriteBufferPool?
+ if (poolSize >= storeConf.maxBufferPoolSize) {
+ // pool is at max size, release extra so memory can be shared. Be sure to release each vector's memory
+ for { colNo <- 0 until dataset.numDataColumns optimized } {
+ memFactory.freeMemory(ChunkSetInfo.getVectorPtr(metaAddr, colNo))
+ }
+ memFactory.freeMemory(metaAddr)
+ } else {
+ // IMPORTANT: reset size in ChunkSetInfo metadata so there won't be an inconsistency
+ // between appenders and metadata (in case some reader is still hanging on to this old info)
+ ChunkSetInfo.resetNumRows(metaAddr)
+ appenders.foreach(_.reset())
+ queue.add((metaAddr, appenders))
+ }
}
}
\ 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 decc5629d2..7424d1a895 100644
--- a/core/src/main/scala/filodb.core/metadata/Dataset.scala
+++ b/core/src/main/scala/filodb.core/metadata/Dataset.scala
@@ -232,25 +232,25 @@ object Dataset {
partitionColumns: Seq[String],
dataColumns: Seq[String],
keyColumns: Seq[String]): Dataset =
- apply(name, partitionColumns, dataColumns, keyColumns, Nil)
+ apply(name, partitionColumns, dataColumns, keyColumns, Nil, DatasetOptions.DefaultOptions)
def apply(name: String,
partitionColumns: Seq[String],
dataColumns: Seq[String],
keyColumns: Seq[String],
- downsamplers: Seq[String]): Dataset =
- make(name, partitionColumns, dataColumns, keyColumns, downsamplers).badMap(BadSchemaError).toTry.get
+ downsamplers: Seq[String], options : DatasetOptions): Dataset =
+ make(name, partitionColumns, dataColumns, keyColumns, downsamplers, options).badMap(BadSchemaError).toTry.get
def apply(name: String,
partitionColumns: Seq[String],
dataColumns: Seq[String],
- keyColumn: String): Dataset =
- apply(name, partitionColumns, dataColumns, Seq(keyColumn), Nil)
+ keyColumn: String, options: DatasetOptions): Dataset =
+ apply(name, partitionColumns, dataColumns, Seq(keyColumn), Nil, options)
def apply(name: String,
partitionColumns: Seq[String],
dataColumns: Seq[String]): Dataset =
- apply(name, partitionColumns, dataColumns, "timestamp")
+ apply(name, partitionColumns, dataColumns, "timestamp", DatasetOptions.DefaultOptions)
sealed trait BadSchema
case class BadDownsampler(msg: String) extends BadSchema
diff --git a/core/src/main/scala/filodb.core/query/RangeVector.scala b/core/src/main/scala/filodb.core/query/RangeVector.scala
index 01c5219e8a..d5255bdf6e 100644
--- a/core/src/main/scala/filodb.core/query/RangeVector.scala
+++ b/core/src/main/scala/filodb.core/query/RangeVector.scala
@@ -80,7 +80,8 @@ object CustomRangeVectorKey {
UTF8Str(str)
}
- val emptyAsZcUtf8 = toZcUtf8(CustomRangeVectorKey(Map.empty))
+ val empty = CustomRangeVectorKey(Map.empty)
+ val emptyAsZcUtf8 = toZcUtf8(empty)
}
/**
@@ -175,20 +176,22 @@ object SerializableRangeVector extends StrictLogging {
*/
def apply(rv: RangeVector,
builder: RecordBuilder,
- schema: RecordSchema): SerializableRangeVector = {
+ schema: RecordSchema,
+ execPlan: String): SerializableRangeVector = {
var numRows = 0
val oldContainerOpt = builder.currentContainer
val startRecordNo = oldContainerOpt.map(_.numRecords).getOrElse(0)
// Important TODO / TechDebt: We need to replace Iterators with cursors to better control
// the chunk iteration, lock acquisition and release. This is much needed for safe memory access.
try {
- ChunkMap.validateNoSharedLocks()
+ ChunkMap.validateNoSharedLocks(execPlan)
val rows = rv.rows
while (rows.hasNext) {
numRows += 1
builder.addFromReader(rows.next)
}
} finally {
+ // clear exec plan
// When the query is done, clean up lingering shared locks caused by iterator limit.
ChunkMap.releaseAllSharedLocks()
}
@@ -207,7 +210,7 @@ object SerializableRangeVector extends StrictLogging {
*/
def apply(rv: RangeVector, cols: Seq[ColumnInfo]): SerializableRangeVector = {
val schema = toSchema(cols)
- apply(rv, toBuilder(schema), schema)
+ apply(rv, toBuilder(schema), schema, "Test-Only-Plan")
}
// TODO: make this configurable....
diff --git a/core/src/main/scala/filodb.core/query/ResultTypes.scala b/core/src/main/scala/filodb.core/query/ResultTypes.scala
index 4663c8d8cd..41e0f6ee0c 100644
--- a/core/src/main/scala/filodb.core/query/ResultTypes.scala
+++ b/core/src/main/scala/filodb.core/query/ResultTypes.scala
@@ -27,10 +27,13 @@ final case class ColumnInfo(name: String, colType: Column.ColumnType)
/**
* 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
+ * @param brSchemas if any of the columns is a BinaryRecord: map of colNo -> inner BinaryRecord schema
+ * @param fixedVectorLen if defined, each vector is guaranteed to have exactly this many output elements.
+ * See PeriodicSampleMapper for an example of how this is used.
*/
final case class ResultSchema(columns: Seq[ColumnInfo], numRowKeyColumns: Int,
- brSchemas: Map[Int, Seq[ColumnInfo]] = Map.empty) {
+ brSchemas: Map[Int, Seq[ColumnInfo]] = Map.empty,
+ fixedVectorLen: Option[Int] = None) {
import Column.ColumnType._
def length: Int = columns.length
diff --git a/core/src/main/scala/filodb.core/store/ChunkSetInfo.scala b/core/src/main/scala/filodb.core/store/ChunkSetInfo.scala
index 3de4c53bfa..d0e14476bb 100644
--- a/core/src/main/scala/filodb.core/store/ChunkSetInfo.scala
+++ b/core/src/main/scala/filodb.core/store/ChunkSetInfo.scala
@@ -41,7 +41,7 @@ object ChunkSet {
def apply(dataset: Dataset, part: PartitionKey, rows: Seq[RowReader], factory: MemFactory): ChunkSet = {
require(rows.nonEmpty)
val startTime = dataset.timestamp(rows.head)
- val info = ChunkSetInfo(factory, dataset, newChunkID(startTime), rows.length,
+ val info = ChunkSetInfo(factory, dataset, chunkID(startTime), rows.length,
startTime,
dataset.timestamp(rows.last))
val filoSchema = Column.toFiloSchema(dataset.dataColumns)
@@ -102,6 +102,10 @@ final case class ChunkSetInfo(infoAddr: NativePointer) extends AnyVal {
None
}
}
+
+ def debugString: String =
+ if (infoAddr == 0) "ChunkSetInfo(NULL)"
+ else s"ChunkSetInfo(id=$id numRows=$numRows startTime=$startTime endTime=$endTime)"
}
case class ChunkRowSkipIndex(id: ChunkID, overrides: EWAHCompressedBitmap)
diff --git a/core/src/main/scala/filodb.core/store/ChunkSink.scala b/core/src/main/scala/filodb.core/store/ChunkSink.scala
index aba2b581ce..19db32546b 100644
--- a/core/src/main/scala/filodb.core/store/ChunkSink.scala
+++ b/core/src/main/scala/filodb.core/store/ChunkSink.scala
@@ -133,7 +133,7 @@ class NullColumnStore(implicit sched: Scheduler) extends ColumnStore with Strict
def readRawPartitions(dataset: Dataset,
columnIDs: Seq[Types.ColumnId],
partMethod: PartitionScanMethod,
- chunkMethod: ChunkScanMethod = AllChunkScan): Observable[RawPartData] = Observable.empty
+ chunkMethod: ChunkScanMethod = AllChunkScan): Observable[RawPartData] = Observable.empty
override def getScanSplits(dataset: DatasetRef, splitsPerNode: Int): Seq[ScanSplit] = Seq.empty
diff --git a/core/src/main/scala/filodb.core/store/IngestionConfig.scala b/core/src/main/scala/filodb.core/store/IngestionConfig.scala
index bb8353a92c..0e6971e437 100644
--- a/core/src/main/scala/filodb.core/store/IngestionConfig.scala
+++ b/core/src/main/scala/filodb.core/store/IngestionConfig.scala
@@ -19,8 +19,7 @@ final case class StoreConfig(flushInterval: FiniteDuration,
shardMemSize: Long,
// Number of bytes to allocate to ingestion write buffers per shard
ingestionBufferMemSize: Long,
- // Number of WriteBuffers to allocate at once
- allocStepSize: Int,
+ maxBufferPoolSize: Int,
numToEvict: Int,
groupsPerShard: Int,
numPagesPerBlock: Int,
@@ -32,7 +31,9 @@ final case class StoreConfig(flushInterval: FiniteDuration,
multiPartitionODP: Boolean,
demandPagingParallelism: Int,
demandPagingEnabled: Boolean,
- evictedPkBfCapacity: Int) {
+ evictedPkBfCapacity: Int,
+ // filters on ingested records to log in detail
+ traceFilters: Map[String, String]) {
import collection.JavaConverters._
def toConfig: Config =
ConfigFactory.parseMap(Map("flush-interval" -> (flushInterval.toSeconds + "s"),
@@ -42,7 +43,7 @@ final case class StoreConfig(flushInterval: FiniteDuration,
"max-blob-buffer-size" -> maxBlobBufferSize,
"shard-mem-size" -> shardMemSize,
"ingestion-buffer-mem-size" -> ingestionBufferMemSize,
- "buffer-alloc-step-size" -> allocStepSize,
+ "max-buffer-pool-size" -> maxBufferPoolSize,
"num-partitions-to-evict" -> numToEvict,
"groups-per-shard" -> groupsPerShard,
"num-block-pages" -> numPagesPerBlock,
@@ -68,7 +69,7 @@ object StoreConfig {
|max-chunks-size = 400
|max-blob-buffer-size = 15000
|ingestion-buffer-mem-size = 10M
- |buffer-alloc-step-size = 1000
+ |max-buffer-pool-size = 10000
|num-partitions-to-evict = 1000
|groups-per-shard = 60
|num-block-pages = 1000
@@ -80,6 +81,7 @@ object StoreConfig {
|demand-paging-parallelism = 4
|demand-paging-enabled = true
|evicted-pk-bloom-filter-capacity = 5000000
+ |trace-filters = {}
|""".stripMargin)
/** Pass in the config inside the store {} */
def apply(storeConfig: Config): StoreConfig = {
@@ -91,7 +93,7 @@ object StoreConfig {
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"),
+ config.getInt("max-buffer-pool-size"),
config.getInt("num-partitions-to-evict"),
config.getInt("groups-per-shard"),
config.getInt("num-block-pages"),
@@ -102,7 +104,8 @@ object StoreConfig {
config.getBoolean("multi-partition-odp"),
config.getInt("demand-paging-parallelism"),
config.getBoolean("demand-paging-enabled"),
- config.getInt("evicted-pk-bloom-filter-capacity"))
+ config.getInt("evicted-pk-bloom-filter-capacity"),
+ config.as[Map[String, String]]("trace-filters"))
}
}
diff --git a/core/src/main/scala/filodb.core/store/package.scala b/core/src/main/scala/filodb.core/store/package.scala
index 4b89bb782c..432cf3eef3 100644
--- a/core/src/main/scala/filodb.core/store/package.scala
+++ b/core/src/main/scala/filodb.core/store/package.scala
@@ -2,26 +2,18 @@ package filodb.core
import java.nio.ByteBuffer
-import com.github.rholder.fauxflake.IdGenerators
import net.jpountz.lz4.{LZ4Compressor, LZ4Factory, LZ4FastDecompressor}
import filodb.core.Types._
import filodb.core.metadata.Dataset
-import filodb.core.SingleKeyTypes.Long64HighBit
import filodb.memory.format.{RowReader, UnsafeUtils}
package object store {
val compressor = new ThreadLocal[LZ4Compressor]()
val decompressor = new ThreadLocal[LZ4FastDecompressor]()
- val machineIdLong = IdGenerators.newSnowflakeIdGenerator.generateId(1)
- val machineId1024 = (machineIdLong.asLong >> 12) & (0x03ff)
val msBitOffset = 21
- val machIdBitOffset = 11
- val baseNsBitOffset = 9 // 2 ** 9 = 512
- val nanoBitMask = Math.pow(2, machIdBitOffset).toInt - 1
- val lowerBitsMask = Math.pow(2, msBitOffset).toInt - 1
- val baseTimeMillis = org.joda.time.DateTime.parse("2016-01-01T00Z").getMillis
+ val lowerBitsMask = Math.pow(2, msBitOffset).toInt - 1
// Assume LZ4 compressor has state and is not thread safe. Use ThreadLocals.
private def getCompressor: LZ4Compressor = {
@@ -95,32 +87,13 @@ package object store {
}
/**
- * 64-bit TimeUUID function designed specifically for generating unique ChunkIDs. Chunks take a while
- * to encode so rarely would you be generating more than a few thousand chunks per second. Format:
- * bits 63-21 (43 bits): milliseconds since Jan 1, 2016 - enough for 278.7 years or through 2294
- * bits 20-11 (10 bits): SnowFlake-style machine ID from FauxFlake library
- * bits 10-0 (11 bits): nanosecond time in 512-ns increments.
- *
- * Bit 63 is inverted to allow for easy comparisons using standard signed Long math.
- *
- * The TimeUUID function generally increases in time but successive calls are not guaranteed to be strictly
- * increasing, but if called greater than 512ns apart should be unique.
- */
- def timeUUID64: Long = {
- ((System.currentTimeMillis - baseTimeMillis) << msBitOffset) |
- (machineId1024 << machIdBitOffset) |
- ((System.nanoTime >> baseNsBitOffset) & nanoBitMask) ^
- Long64HighBit
- }
-
- /**
- * New formulation for chunkID based on a combo of the start time for a chunk and the current time in the lower
+ * Formulation for chunkID based on a combo of the start time for a chunk and the current time in the lower
* bits to disambiguate two chunks which have the same start time.
*
* bits 63-21 (43 bits): milliseconds since Unix Epoch (1/1/1970) - enough for 278.7 years or through 2248
* bits 20-0 (21 bits): The lower 21 bits of nanotime for disambiguation
*/
- @inline final def newChunkID(startTime: Long): Long = chunkID(startTime, System.nanoTime)
+ @inline final def chunkID(startTime: Long): Long = chunkID(startTime, System.nanoTime)
@inline final def chunkID(startTime: Long, currentTime: Long): Long =
(startTime << msBitOffset) | (currentTime & lowerBitsMask)
diff --git a/core/src/test/resources/application_test.conf b/core/src/test/resources/application_test.conf
index 650256c6ca..e287e60892 100644
--- a/core/src/test/resources/application_test.conf
+++ b/core/src/test/resources/application_test.conf
@@ -56,6 +56,15 @@ filodb {
sample-limit = 1000000
min-step = 1 ms
}
+
+ spread-default = 1
+
+ spread-assignment = [
+ {
+ _ns = App-0,
+ _spread_ = 2
+ }
+ ]
}
query-actor-mailbox {
diff --git a/core/src/test/scala/filodb.core/TestData.scala b/core/src/test/scala/filodb.core/TestData.scala
index adae778c24..1f1d42cc40 100644
--- a/core/src/test/scala/filodb.core/TestData.scala
+++ b/core/src/test/scala/filodb.core/TestData.scala
@@ -39,11 +39,11 @@ 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
+ shard-mem-size = 100MB
groups-per-shard = 4
- ingestion-buffer-mem-size = 10MB
+ ingestion-buffer-mem-size = 80MB
+ max-buffer-pool-size = 250
flush-interval = 10 minutes
part-index-flush-max-delay = 10 seconds
part-index-flush-min-delay = 2 seconds
@@ -58,7 +58,7 @@ object NamesTestData {
def mapper(rows: Seq[Product]): Seq[RowReader] = rows.map(TupleRowReader)
val dataColSpecs = Seq("first:string", "last:string", "age:long:interval=10")
- val dataset = Dataset("dataset", Seq("seg:int"), dataColSpecs, "age")
+ val dataset = Dataset("dataset", Seq("seg:int"), dataColSpecs, "age", DatasetOptions.DefaultOptions)
// NOTE: first 3 columns are the data columns, thus names could be used for either complete record
// or the data column rowReader
@@ -91,7 +91,7 @@ object NamesTestData {
val sortedFirstNames = Seq("Khalil", "Rodney", "Ndamukong", "Terrance", "Peyton", "Jerry")
val sortedUtf8Firsts = sortedFirstNames.map(_.utf8)
- val largeDataset = Dataset("dataset", Seq("league:string"), dataColSpecs, "age")
+ val largeDataset = Dataset("dataset", Seq("league:string"), dataColSpecs, "age", DatasetOptions.DefaultOptions)
val lotLotNames = {
for { league <- Seq("nfc", "afc")
@@ -170,7 +170,7 @@ object GdeltTestData {
val seqReaders = records.map { record => SeqRowReader(record.productIterator.toList) }
// Dataset1: Partition keys (Actor2Code, Year) / Row key GLOBALEVENTID
- val dataset1 = Dataset("gdelt", Seq(schema(4), schema(3)), schema.patch(3, Nil, 2), "GLOBALEVENTID")
+ val dataset1 = Dataset("gdelt", Seq(schema(4), schema(3)), schema.patch(3, Nil, 2), "GLOBALEVENTID", DatasetOptions.DefaultOptions)
// Dataset2: Partition key (MonthYear) / Row keys (GLOBALEVENTID, Actor2Code)
val dataset2 = Dataset("gdelt", Seq(schema(2)), schema.patch(2, Nil, 1), Seq("GLOBALEVENTID", "Actor2Code"))
@@ -185,7 +185,9 @@ object GdeltTestData {
// val partBuilder4 = new RecordBuilder(TestData.nativeMem, dataset4.partKeySchema, 10240)
// Proj 6: partition Actor2Code,Actor2Name to test partition key bitmap indexing
- val dataset6 = Dataset("gdelt", schema.slice(4, 6), schema.patch(4, Nil, 2), "GLOBALEVENTID")
+ val datasetOptions = DatasetOptions.DefaultOptions.copy(
+ shardKeyColumns = Seq( "__name__","_ns"))
+ val dataset6 = Dataset("gdelt", schema.slice(4, 6), schema.patch(4, Nil, 2), "GLOBALEVENTID", datasetOptions)
}
// A simulation of machine metrics data
diff --git a/core/src/test/scala/filodb.core/memstore/PartKeyLuceneIndexSpec.scala b/core/src/test/scala/filodb.core/memstore/PartKeyLuceneIndexSpec.scala
index ba32d5e18c..611dc48f25 100644
--- a/core/src/test/scala/filodb.core/memstore/PartKeyLuceneIndexSpec.scala
+++ b/core/src/test/scala/filodb.core/memstore/PartKeyLuceneIndexSpec.scala
@@ -103,7 +103,7 @@ class PartKeyLuceneIndexSpec extends FunSpec with Matchers with BeforeAndAfter {
}
- it("should add part keys and fetch startTimes correctly") {
+ it("should add part keys and fetch startTimes correctly for more than 1024 keys") {
val numPartIds = 3000 // needs to be more than 1024 to test the lucene term limit
val start = System.currentTimeMillis()
// we dont care much about the partKey here, but the startTime against partId.
@@ -120,6 +120,31 @@ class PartKeyLuceneIndexSpec extends FunSpec with Matchers with BeforeAndAfter {
}
}
+ it("should add part keys and fetch partIdsEndedBefore and removePartKeys correctly for more than 1024 keys") {
+ val numPartIds = 3000 // needs to be more than 1024 to test the lucene term limit
+ val start = 1000
+ // we dont care much about the partKey here, but the startTime against partId.
+ val partKeys = Stream.continually(readers.head).take(numPartIds).toList
+ partKeyFromRecords(dataset6, records(dataset6, partKeys), Some(partBuilder))
+ .zipWithIndex.foreach { case (addr, i) =>
+ keyIndex.addPartKey(partKeyOnHeap(dataset6, ZeroPointer, addr), i, start + i, start + i + 100)()
+ }
+ keyIndex.commitBlocking()
+
+ val pIds = keyIndex.partIdsEndedBefore(start + 200)
+ for { i <- 0 until numPartIds} {
+ pIds.get(i) shouldEqual (if (i <= 100) true else false)
+ }
+
+ keyIndex.removePartKeys(pIds)
+ keyIndex.commitBlocking()
+
+ for { i <- 0 until numPartIds} {
+ keyIndex.partKeyFromPartId(i).isDefined shouldEqual (if (i <= 100) false else true)
+ }
+
+ }
+
it("should update part keys with endtime and parse filters correctly") {
val start = System.currentTimeMillis()
// Add the first ten keys and row numbers
diff --git a/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala b/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala
index a59f323bf5..1ae2680e2f 100644
--- a/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala
+++ b/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala
@@ -341,14 +341,15 @@ class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter w
memStore.setup(dataset1, 0, TestData.storeConf)
val initChunksWritten = chunksetsWritten
- val checkpoints = Map(0 -> 2L, 1 -> 4L, 2 -> 6L, 3 -> 8L)
+ val checkpoints = Map(0 -> 2L, 1 -> 21L, 2 -> 6L, 3 -> 8L)
// val stream = Observable.fromIterable(linearMultiSeries().take(100).grouped(5).toSeq.map(records(dataset1, _)))
- val stream = Observable.fromIterable(groupedRecords(dataset1, linearMultiSeries()))
- val offsets = memStore.recoverStream(dataset1.ref, 0, stream, checkpoints, 4L)
- .until(_ >= 50L).toListL.runAsync.futureValue
+ val stream = Observable.fromIterable(groupedRecords(dataset1, linearMultiSeries(), 200))
+ // recover from checkpoints.min to checkpoints.max
+ val offsets = memStore.recoverStream(dataset1.ref, 0, stream, 2, 21, checkpoints, 4L)
+ .until(_ >= 21L).toListL.runAsync.futureValue
- offsets shouldEqual Seq(7L, 11L, 15L, 19L)
+ offsets shouldEqual Seq(7L, 11L, 15L, 19L, 21L) // last offset is always reported
// no flushes
chunksetsWritten shouldEqual initChunksWritten
@@ -358,7 +359,7 @@ class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter w
val splits = memStore.getScanSplits(dataset1.ref, 1)
val data1 = memStore.scanRows(dataset1, Seq(1), FilteredPartitionScan(splits.head))
.map(_.getDouble(0)).toSeq
- data1.length shouldEqual 71
+ data1.length shouldEqual 47
}
it("should truncate shards properly") {
@@ -539,4 +540,39 @@ class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter w
.asInstanceOf[Seq[TimeSeriesPartition]]
parts.map(_.partID).toSet shouldEqual (0 to 20).toSet
}
+
+ it("should return extra WriteBuffers to memoryManager properly") {
+ val numSeries = 300
+ val policy2 = new FixedMaxPartitionsEvictionPolicy(numSeries * 2)
+ val store2 = new TimeSeriesMemStore(config, new NullColumnStore, new InMemoryMetaStore(), Some(policy2))
+
+ try {
+ // Ingest >250 partitions. Note how much memory is left after all the allocations
+ store2.setup(dataset1, 0, TestData.storeConf)
+ val shard = store2.getShardE(dataset1.ref, 0)
+
+ // Ingest normal multi series data with 10 partitions. Should have 10 partitions.
+ val data = records(dataset1, linearMultiSeries(numSeries = numSeries).take(numSeries))
+ store2.ingest(dataset1.ref, 0, data)
+ store2.commitIndexForTesting(dataset1.ref)
+
+ store2.numPartitions(dataset1.ref, 0) shouldEqual numSeries
+ shard.bufferPool.poolSize shouldEqual 100 // Two allocations of 200 each = 400; used up 300; 400-300=100
+ val afterIngestFree = shard.bufferMemoryManager.numFreeBytes
+
+ // Switch buffers, encode and release/return buffers for all partitions
+ val blockFactory = shard.overflowBlockFactory
+ for { n <- 0 until numSeries } {
+ val part = shard.partitions.get(n)
+ part.switchBuffers(blockFactory, encode = true)
+ }
+
+ // Ensure queue length does not get beyond 250, and some memory was freed (free bytes increases)
+ shard.bufferPool.poolSize shouldEqual 250
+ val nowFree = shard.bufferMemoryManager.numFreeBytes
+ nowFree should be > (afterIngestFree)
+ } finally {
+ store2.shutdown() // release snd free the memory
+ }
+ }
}
diff --git a/core/src/test/scala/filodb.core/memstore/TimeSeriesPartitionSpec.scala b/core/src/test/scala/filodb.core/memstore/TimeSeriesPartitionSpec.scala
index 951b5a4e0f..b38ae6e070 100644
--- a/core/src/test/scala/filodb.core/memstore/TimeSeriesPartitionSpec.scala
+++ b/core/src/test/scala/filodb.core/memstore/TimeSeriesPartitionSpec.scala
@@ -28,6 +28,13 @@ object TimeSeriesPartitionSpec {
new TimeSeriesPartition(partNo, dataset, partKey, 0, bufferPool,
new TimeSeriesShardStats(dataset.ref, 0), memFactory, 40)
}
+
+ def tracingPart(partNo: Int, dataset: Dataset,
+ partKey: NativePointer = defaultPartKey,
+ bufferPool: WriteBufferPool = myBufferPool): TimeSeriesPartition = {
+ new TracingTimeSeriesPartition(partNo, dataset, partKey, 0, bufferPool,
+ new TimeSeriesShardStats(dataset.ref, 0), memFactory, 40)
+ }
}
trait MemFactoryCleanupTest extends FunSpec with Matchers with BeforeAndAfter with BeforeAndAfterAll {
@@ -399,4 +406,28 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures {
readData2.toBuffer shouldEqual (timestamps take 5) ++ (timestamps drop 7)
}
+ it("TracingTSPartition should be able to ingest new rows") {
+ part = tracingPart(0, dataset1)
+ val data = singleSeriesReaders().take(11)
+ val minData = data.map(_.getDouble(1))
+ val initTS = data(0).getLong(0)
+ data.take(10).zipWithIndex.foreach { case (r, i) => part.ingest(r, ingestBlockHolder) }
+
+ val origPoolSize = myBufferPool.poolSize
+
+ // First 10 rows ingested. Now flush in a separate Future while ingesting the remaining row
+ part.switchBuffers(ingestBlockHolder)
+ // After switchBuffers, currentChunks should be null, pool size the same (nothing new allocated yet)
+ myBufferPool.poolSize shouldEqual origPoolSize
+ part.appendingChunkLen shouldEqual 0
+
+ // Before flush happens, should be able to read all chunks
+ part.unflushedChunksets shouldEqual 1
+ part.numChunks shouldEqual 1
+ val infos1 = part.infos(AllChunkScan).toBuffer
+ infos1 should have length 1
+ infos1.head.startTime shouldEqual initTS
+ val data1 = part.timeRangeRows(AllChunkScan, Array(1)).map(_.getDouble(0)).toBuffer
+ data1 shouldEqual (minData take 10)
+ }
}
diff --git a/core/src/test/scala/filodb.core/metadata/DatasetSpec.scala b/core/src/test/scala/filodb.core/metadata/DatasetSpec.scala
index 09853207a1..91c505cf5f 100644
--- a/core/src/test/scala/filodb.core/metadata/DatasetSpec.scala
+++ b/core/src/test/scala/filodb.core/metadata/DatasetSpec.scala
@@ -17,7 +17,7 @@ class DatasetSpec extends FunSpec with Matchers {
resp1.swap.get shouldEqual ColumnErrors(Seq(NotNameColonType("column2")))
intercept[BadSchemaError] {
- Dataset("dataset", Seq("part:string"), dataColSpecs :+ "column2:a:b", "age")
+ Dataset("dataset", Seq("part:string"), dataColSpecs :+ "column2:a:b", "age", DatasetOptions.DefaultOptions)
}
}
@@ -94,11 +94,11 @@ class DatasetSpec extends FunSpec with Matchers {
val mapCol = "tags:map"
// OK: only partition column is map
- val ds1 = Dataset("dataset", Seq(mapCol), dataColSpecs, "age")
+ val ds1 = Dataset("dataset", Seq(mapCol), dataColSpecs, "age", DatasetOptions.DefaultOptions)
ds1.partitionColumns.map(_.name) should equal (Seq("tags"))
// OK: last partition column is map
- val ds2 = Dataset("dataset", Seq("first:string", mapCol), dataColSpecs drop 1, "age")
+ val ds2 = Dataset("dataset", Seq("first:string", mapCol), dataColSpecs drop 1, "age", DatasetOptions.DefaultOptions)
ds2.partitionColumns.map(_.name) should equal (Seq("first", "tags"))
// Not OK: first partition column is map
@@ -118,7 +118,7 @@ class DatasetSpec extends FunSpec with Matchers {
}
it("should return a valid Dataset when a good specification passed") {
- val ds = Dataset("dataset", Seq("part:string"), dataColSpecs, "age")
+ val ds = Dataset("dataset", Seq("part:string"), dataColSpecs, "age", DatasetOptions.DefaultOptions)
ds.rowKeyIDs shouldEqual Seq(2)
ds.dataColumns should have length (3)
ds.dataColumns.map(_.id) shouldEqual Seq(0, 1, 2)
@@ -144,7 +144,7 @@ class DatasetSpec extends FunSpec with Matchers {
}
it("should return IDs for column names or seq of missing names") {
- val ds = Dataset("dataset", Seq("part:string"), dataColSpecs, "age")
+ val ds = Dataset("dataset", Seq("part:string"), dataColSpecs, "age", DatasetOptions.DefaultOptions)
ds.colIDs("first", "age").get shouldEqual Seq(0, 2)
ds.colIDs("part").get shouldEqual Seq(Dataset.PartColStartIndex)
@@ -155,7 +155,7 @@ class DatasetSpec extends FunSpec with Matchers {
}
it("should return ColumnInfos for colIDs") {
- val ds = Dataset("dataset", Seq("part:string"), dataColSpecs, "age")
+ val ds = Dataset("dataset", Seq("part:string"), dataColSpecs, "age", DatasetOptions.DefaultOptions)
val infos = ds.infosFromIDs(Seq(0, 2))
infos shouldEqual Seq(ColumnInfo("first", StringColumn), ColumnInfo("age", LongColumn))
@@ -172,7 +172,8 @@ class DatasetSpec extends FunSpec with Matchers {
describe("Dataset serialization") {
it("should serialize and deserialize") {
- val ds = Dataset("dataset", Seq("part:string"), dataColSpecs, Seq("age"), Seq("dMin(1)"))
+ val ds = Dataset("dataset", Seq("part:string"), dataColSpecs, Seq("age"), Seq("dMin(1)"),
+ DatasetOptions.DefaultOptions)
.copy(options = DatasetOptions.DefaultOptions.copy(
copyTags = Map("exporter" -> "_ns")))
Dataset.fromCompactString(ds.asCompactString) shouldEqual ds
diff --git a/core/src/test/scala/filodb.core/query/RangeVectorSpec.scala b/core/src/test/scala/filodb.core/query/RangeVectorSpec.scala
index 353dac1e9f..ef357366f7 100644
--- a/core/src/test/scala/filodb.core/query/RangeVectorSpec.scala
+++ b/core/src/test/scala/filodb.core/query/RangeVectorSpec.scala
@@ -47,7 +47,7 @@ class RangeVectorSpec extends FunSpec with Matchers {
val builder = SerializableRangeVector.toBuilder(schema)
// Sharing one builder across multiple input RangeVectors
- val srvs = rvs.map(rv => SerializableRangeVector(rv, builder, schema))
+ val srvs = rvs.map(rv => SerializableRangeVector(rv, builder, schema, "Unit-test"))
// Now verify each of them
val observedTs = srvs(0).rows.toSeq.map(_.getLong(0))
diff --git a/core/src/test/scala/filodb.core/store/MetaStoreSpec.scala b/core/src/test/scala/filodb.core/store/MetaStoreSpec.scala
index a6b59a993b..869331ad5c 100644
--- a/core/src/test/scala/filodb.core/store/MetaStoreSpec.scala
+++ b/core/src/test/scala/filodb.core/store/MetaStoreSpec.scala
@@ -19,7 +19,8 @@ with BeforeAndAfter with BeforeAndAfterAll with ScalaFutures {
metaStore.initialize().futureValue
}
- val dataset = Dataset("foo", Seq("part:string"), Seq("timestamp:long", "value:double"), "timestamp")
+ val dataset = Dataset("foo", Seq("part:string"), Seq("timestamp:long", "value:double"), "timestamp",
+ DatasetOptions.DefaultOptions)
before { metaStore.clearAllData().futureValue }
diff --git a/doc/http_api.md b/doc/http_api.md
index d7600d6779..9fb4de6718 100644
--- a/doc/http_api.md
+++ b/doc/http_api.md
@@ -176,6 +176,11 @@ Used to issue a promQL query for a time range with a `start` and `end` timestamp
For more details, see Prometheus HTTP API Documentation
[Range Queries](https://prometheus.io/docs/prometheus/latest/querying/api/#range-queries)
+```
+params:
+• `explainOnly` -- returns an ExecPlan instead of the query results
+• `spread` -- override default spread
+```
#### GET /promql/{dataset}/api/v1/query?query={promQLString}&time={timestamp}
@@ -183,6 +188,12 @@ Used to issue a promQL query for a single time instant `time`. Can also be used
range expression. For more details, see Prometheus HTTP API Documentation
[Instant Queries](https://prometheus.io/docs/prometheus/latest/querying/api/#instant-queries)
+```
+params:
+• `explainOnly` -- returns an ExecPlan instead of the query results
+• `spread` -- override default spread
+```
+
#### POST /promql/{dataset}/api/v1/read
Used to extract raw data for integration with other TSDB systems.
diff --git a/doc/ingestion.md b/doc/ingestion.md
index 6da92f5da8..089910edb3 100644
--- a/doc/ingestion.md
+++ b/doc/ingestion.md
@@ -72,9 +72,8 @@ sourceconfig {
# Assume 5 bytes per sample, should be roughly equal to (# samples per time series) * (# time series)
shard-mem-size = 256MB
- # 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 bytes of offheap mem to allocate to write buffers for all shards. Ex. 1000MB, 1G, 2GB
+ ingestion-buffer-mem-size = 200MB
# 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
@@ -108,7 +107,7 @@ See the TestConsumer for more info.
How much to allocate for `ingestion-buffer-mem-size` and `shard-mem-size` as well as heap? Here are some guidelines:
* **Heap memory** - heap usage grows by the number of time series stored by FiloDB in memory, but not by the number of chunks or amount of data within each series. As of 8/6/18 1.5 million time series will fit within 1GB of heap. At least 5-10 more GB is recommended though for extra memory for ingestion, recovery, and querying.
-* **Ingestion buffer** - The ingestion buffer is a per-shard offheap memory area for ingestion write buffers and some other time series-specific data structures. It needs to be scaled with the number of time series actively ingesting in the system, a few KB for each series. Once the ingestion buffer runs out, no more time series can be added and eviction of existing time series starting with the oldest non-actively ingesting time series will begin to free up room. If not enough room can be freed, new time series and in extreme cases even new data may not be ingested.
+* **Ingestion buffer** - The ingestion buffer is a per-dataset offheap memory area for ingestion write buffers and some other time series-specific data structures. It needs to be scaled with the number of time series actively ingesting in the system, a few KB for each series. Once the ingestion buffer runs out, no more time series can be added and eviction of existing time series starting with the oldest non-actively ingesting time series will begin to free up room. If not enough room can be freed, new time series and in extreme cases even new data may not be ingested.
* `shard-mem-size` - this is the offheap block storage used to store encoded chunks for the time series data samples and metadata for each chunk. This should be sized for the number of time series as well as the length of retention desired for all the time series. The configuration is currently **per-shard**. When this memory runs out, the oldest blocks will be reclaimed automatically and those chunks will be dropped from time series.
## Recovery and Persistence
diff --git a/doc/sharding.md b/doc/sharding.md
index 8751580d66..83d82ddf89 100644
--- a/doc/sharding.md
+++ b/doc/sharding.md
@@ -39,8 +39,19 @@ What the above means is that
## Spread, or How to Avoid Hotspotting
-The **spread** determines how many shards a given shard key is mapped to. The number of shards is equal to 2 to the power of the spread. It is used to manage how widely specific shard keys (such as applications, the job, or metrics) are distributed. For example, if one job or metric has a huge number of series, one can assign a higher spread to it to avoid hotspotting. (The management of spreads for individual shard keys is not currently included in the open source offering).
-
+The **spread** determines how many shards a given shard key is mapped to. The number of shards is equal to 2 to the power of the spread. It is used to manage how widely specific shard keys (such as applications, the job, or metrics) are distributed. For example, if one job or metric has a huge number of series, one can assign a higher spread to it to avoid hotspotting. Default spread can be overriden by appending spread as a parameter in the query. Spread can also be overriden for every application by specifying all non metric shard keys in the config
+```
+spread-assignment = [
+ {
+ _ns = App-0,
+ _spread_ = 2
+ },
+ {
+ _ns = App-5,
+ _spread_ = 0
+ }
+ ]
+```
## Shard Coordination
FiloDB Clients enable users to set up new datasets as needed. Internally clients send a `SetupDataset` command
to the [NodeClusterActor](../coordinator/src/main/scala/filodb.coordinator/NodeClusterActor.scala).
diff --git a/gateway/src/main/scala/filodb/gateway/GatewayServer.scala b/gateway/src/main/scala/filodb/gateway/GatewayServer.scala
index 49e5f6712e..76657905af 100644
--- a/gateway/src/main/scala/filodb/gateway/GatewayServer.scala
+++ b/gateway/src/main/scala/filodb/gateway/GatewayServer.scala
@@ -50,7 +50,7 @@ import filodb.timeseries.TestTimeseriesProducer
* 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 \
+ * --dataColumns timestamp:ts,sum:long,count:long,h:hist:counter=true --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
@@ -72,11 +72,11 @@ object GatewayServer extends StrictLogging {
// 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")
+ 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()
}
@@ -95,7 +95,7 @@ object GatewayServer extends StrictLogging {
// NOTE: the spread MUST match the default spread used in the HTTP module for consistency between querying
// and ingestion sharding
- val spread = config.getInt("filodb.default-spread")
+ val spread = config.getInt("filodb.spread-default")
val shardMapper = new ShardMapper(numShards)
val queueFullWait = config.as[FiniteDuration]("gateway.queue-full-wait").toMillis
diff --git a/gateway/src/main/scala/filodb/timeseries/TestTimeseriesProducer.scala b/gateway/src/main/scala/filodb/timeseries/TestTimeseriesProducer.scala
index 32ce1f7a1c..1b130985ef 100644
--- a/gateway/src/main/scala/filodb/timeseries/TestTimeseriesProducer.scala
+++ b/gateway/src/main/scala/filodb/timeseries/TestTimeseriesProducer.scala
@@ -71,7 +71,7 @@ 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 q = 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"
logger.info(s"Periodic Samples query URL: \n$periodicSamplesUrl")
@@ -147,6 +147,8 @@ object TestTimeseriesProducer extends StrictLogging {
* 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
+ * Note: the set of "instance" tags is unique for each invocation of genHistogramData. This helps increase
+ * the cardinality of time series for testing purposes.
*/
def genHistogramData(startTime: Long, dataset: Dataset, numTimeSeries: Int = 16): Stream[InputRecord] = {
require(dataset.dataColumns.map(_.columnType) == Seq(TimestampColumn, LongColumn, LongColumn, HistogramColumn))
@@ -160,8 +162,10 @@ object TestTimeseriesProducer extends StrictLogging {
}
}
+ val instanceBase = System.currentTimeMillis
+
Stream.from(0).map { n =>
- val instance = n % numTimeSeries
+ val instance = n % numTimeSeries + instanceBase
val dc = instance & oneBitMask
val partition = (instance >> 1) & twoBitMask
val app = (instance >> 3) & twoBitMask
diff --git a/http/src/main/scala/filodb/http/HttpSettings.scala b/http/src/main/scala/filodb/http/HttpSettings.scala
index 1339958279..3b63ac63d2 100644
--- a/http/src/main/scala/filodb/http/HttpSettings.scala
+++ b/http/src/main/scala/filodb/http/HttpSettings.scala
@@ -7,6 +7,6 @@ class HttpSettings(config: Config) {
lazy val httpServerBindPort = config.getInt("filodb.http.bind-port")
lazy val httpServerStartTimeout = config.getDuration("filodb.http.start-timeout")
- lazy val queryDefaultSpread = config.getInt("filodb.default-spread")
+ lazy val queryDefaultSpread = config.getInt("filodb.spread-default")
lazy val querySampleLimit = config.getInt("filodb.query.sample-limit")
}
diff --git a/http/src/main/scala/filodb/http/PrometheusApiRoute.scala b/http/src/main/scala/filodb/http/PrometheusApiRoute.scala
index 4ddab600cb..f7af8b2adc 100644
--- a/http/src/main/scala/filodb/http/PrometheusApiRoute.scala
+++ b/http/src/main/scala/filodb/http/PrometheusApiRoute.scala
@@ -14,27 +14,25 @@ import org.xerial.snappy.Snappy
import remote.RemoteStorage.ReadRequest
import filodb.coordinator.client.IngestionCommands.UnknownDataset
-import filodb.coordinator.client.QueryCommands.{LogicalPlan2Query, QueryOptions, SpreadChange, StaticSpreadProvider}
+import filodb.coordinator.client.QueryCommands._
import filodb.core.DatasetRef
import filodb.prometheus.ast.TimeStepParams
import filodb.prometheus.parse.Parser
import filodb.prometheus.query.PrometheusModel.Sampl
import filodb.query.{LogicalPlan, QueryError, QueryResult}
-
+import filodb.query.exec.ExecPlan
class PrometheusApiRoute(nodeCoord: ActorRef, settings: HttpSettings)(implicit am: ActorMaterializer)
extends FiloRoute with StrictLogging {
import FailFastCirceSupport._
import io.circe.generic.auto._
-
+ // DO NOT REMOVE PromCirceSupport import below assuming it is unused - Intellij removes it in auto-imports :( .
+ // Needed to override Sampl case class Encoder.
+ import PromCirceSupport._
import filodb.coordinator.client.Client._
import filodb.prometheus.query.PrometheusModel._
- val spreadProvider = new StaticSpreadProvider(SpreadChange(0, settings.queryDefaultSpread))
-
- val queryOptions = QueryOptions(spreadProvider, settings.querySampleLimit)
-
val route = pathPrefix( "promql" / Segment) { dataset =>
// Path: /promql//api/v1/query_range
// Used to issue a promQL query for a time range with a `start` and `end` timestamp and at regular `step` intervals.
@@ -43,9 +41,11 @@ class PrometheusApiRoute(nodeCoord: ActorRef, settings: HttpSettings)(implicit a
path( "api" / "v1" / "query_range") {
get {
parameter('query.as[String], 'start.as[Double], 'end.as[Double],
- 'step.as[Int], 'verbose.as[Boolean].?) { (query, start, end, step, verbose) =>
+ 'step.as[Int], 'explainOnly.as[Boolean].?, 'verbose.as[Boolean].?, 'spread.as[Int].?)
+ { (query, start, end, step, explainOnly, verbose, spread) =>
val logicalPlan = Parser.queryRangeToLogicalPlan(query, TimeStepParams(start.toLong, step, end.toLong))
- askQueryAndRespond(dataset, logicalPlan, verbose.getOrElse(false))
+ askQueryAndRespond(dataset, logicalPlan, explainOnly.getOrElse(false), verbose.getOrElse(false),
+ spread)
}
}
} ~
@@ -55,9 +55,12 @@ 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], 'verbose.as[Boolean].?) { (query, time, verbose) =>
+ parameter('query.as[String], 'time.as[Double], 'explainOnly.as[Boolean].?, 'verbose.as[Boolean].?,
+ 'spread.as[Int].?)
+ { (query, time, explainOnly, verbose, spread) =>
val logicalPlan = Parser.queryToLogicalPlan(query, time.toLong)
- askQueryAndRespond(dataset, logicalPlan, verbose.getOrElse(false))
+ askQueryAndRespond(dataset, logicalPlan, explainOnly.getOrElse(false),
+ verbose.getOrElse(false), spread)
}
}
} ~
@@ -79,7 +82,7 @@ class PrometheusApiRoute(nodeCoord: ActorRef, settings: HttpSettings)(implicit a
// but Akka doesnt support snappy out of the box. Elegant solution is a TODO for later.
val readReq = ReadRequest.parseFrom(Snappy.uncompress(bytes.toArray))
val asks = toFiloDBLogicalPlans(readReq).map { logicalPlan =>
- asyncAsk(nodeCoord, LogicalPlan2Query(DatasetRef.fromDotString(dataset), logicalPlan, queryOptions))
+ asyncAsk(nodeCoord, LogicalPlan2Query(DatasetRef.fromDotString(dataset), logicalPlan))
}
Future.sequence(asks)
}
@@ -103,11 +106,19 @@ class PrometheusApiRoute(nodeCoord: ActorRef, settings: HttpSettings)(implicit a
}
}
- private def askQueryAndRespond(dataset: String, logicalPlan: LogicalPlan, verbose: Boolean) = {
- val command = LogicalPlan2Query(DatasetRef.fromDotString(dataset), logicalPlan, queryOptions)
+ private def askQueryAndRespond(dataset: String, logicalPlan: LogicalPlan, explainOnly: Boolean, verbose: Boolean,
+ spread: Option[Int]) = {
+ val spreadProvider: Option[SpreadProvider] = spread.map(s => StaticSpreadProvider(SpreadChange(0, s)))
+ val command = if (explainOnly) {
+ ExplainPlan2Query(DatasetRef.fromDotString(dataset), logicalPlan, QueryOptions(spreadProvider))
+ }
+ else {
+ LogicalPlan2Query(DatasetRef.fromDotString(dataset), logicalPlan, QueryOptions(spreadProvider))
+ }
onSuccess(asyncAsk(nodeCoord, command)) {
case qr: QueryResult => complete(toPromSuccessResponse(qr, verbose))
case qr: QueryError => complete(toPromErrorResponse(qr))
+ case qr: ExecPlan => complete(toPromExplainPlanResponse(qr))
case UnknownDataset => complete(Codes.NotFound ->
ErrorResponse("badQuery", s"Dataset $dataset is not registered"))
}
diff --git a/http/src/test/scala/filodb/http/PrometheusApiRouteSpec.scala b/http/src/test/scala/filodb/http/PrometheusApiRouteSpec.scala
new file mode 100644
index 0000000000..c8d5eb2f2a
--- /dev/null
+++ b/http/src/test/scala/filodb/http/PrometheusApiRouteSpec.scala
@@ -0,0 +1,127 @@
+package filodb.http
+
+import akka.actor.ActorSystem
+import akka.http.scaladsl.model.{ContentTypes, StatusCodes}
+import akka.http.scaladsl.testkit.{RouteTestTimeout, ScalatestRouteTest}
+import akka.testkit.TestProbe
+import com.typesafe.config.ConfigFactory
+import de.heikoseeberger.akkahttpcirce.FailFastCirceSupport
+import org.scalatest.FunSpec
+import scala.concurrent.duration._
+
+import filodb.coordinator._
+import filodb.core.{AsyncTest, Success, TestData}
+import filodb.prometheus.FormatConversion
+import filodb.prometheus.query.PrometheusModel.ExplainPlanResponse
+
+object PrometheusApiRouteSpec extends ActorSpecConfig
+
+class PrometheusApiRouteSpec extends FunSpec with ScalatestRouteTest with AsyncTest {
+
+ import FailFastCirceSupport._
+ import NodeClusterActor._
+ import io.circe.generic.auto._
+
+ // Use our own ActorSystem with our test config so we can init cluster properly
+ override def createActorSystem(): ActorSystem = ClusterApiRouteSpec.getNewSystem
+
+ val cluster = FilodbCluster(system)
+ val probe = TestProbe()
+ implicit val timeout = RouteTestTimeout(20.minute)
+ cluster.coordinatorActor
+ cluster.join()
+ val clusterProxy = cluster.clusterSingleton(ClusterRole.Server, None)
+ val filoServerConfig = ConfigFactory.load("application_test.conf")
+ val config = GlobalConfig.systemConfig
+
+ val settings = new HttpSettings(config)
+ val prometheusAPIRoute = (new PrometheusApiRoute(cluster.coordinatorActor, settings)).route
+
+ private def setupDataset(): Unit = {
+ val command = SetupDataset(FormatConversion.dataset.ref, DatasetResourceSpec(8, 1), noOpSource, TestData.storeConf)
+ probe.send(clusterProxy, command)
+ probe.expectMsg(DatasetVerified)
+ // Give the coordinator nodes some time to get started
+ Thread sleep 5000
+ }
+
+ 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(FormatConversion.dataset).futureValue shouldEqual Success
+ probe.send(clusterProxy, NodeProtocol.ResetState)
+ probe.expectMsg(NodeProtocol.StateReset)
+ }
+
+ it("should get explainPlan for query") {
+ setupDataset()
+ val query = "heap_usage{_ns=\"App-0\"}"
+
+ Get(s"/promql/prometheus/api/v1/query_range?query=${query}&" +
+ s"start=1555427432&end=1555447432&step=15&explainOnly=true") ~> prometheusAPIRoute ~> check {
+
+ handled shouldBe true
+ status shouldEqual StatusCodes.OK
+ contentType shouldEqual ContentTypes.`application/json`
+ val resp = responseAs[ExplainPlanResponse]
+ resp.status shouldEqual "success"
+
+ resp.debugInfo(0).toString should startWith("E~DistConcatExec()")
+ resp.debugInfo(1) should startWith("-T~PeriodicSamplesMapper")
+ resp.debugInfo(2) should startWith("--E~SelectRawPartitionsExec")
+ resp.debugInfo(3) should startWith("-T~PeriodicSamplesMapper")
+ resp.debugInfo(4) should startWith("--E~SelectRawPartitionsExec")
+ }
+ }
+
+ it("should take spread override value from config for app") {
+ setupDataset()
+ val query = "heap_usage{_ns=\"App-0\"}"
+
+ Get(s"/promql/prometheus/api/v1/query_range?query=${query}&" +
+ s"start=1555427432&end=1555447432&step=15&explainOnly=true") ~> prometheusAPIRoute ~> check {
+
+ handled shouldBe true
+ status shouldEqual StatusCodes.OK
+ contentType shouldEqual ContentTypes.`application/json`
+ val resp = responseAs[ExplainPlanResponse]
+ resp.status shouldEqual "success"
+ resp.debugInfo.filter(_.startsWith("--E~SelectRawPartitionsExec")).length shouldEqual 4
+ }
+ }
+
+ it("should get explainPlan for query based on spread as query parameter") {
+ setupDataset()
+ val query = "heap_usage{_ns=\"App-1\"}"
+
+ Get(s"/promql/prometheus/api/v1/query_range?query=${query}&" +
+ s"start=1555427432&end=1555447432&step=15&explainOnly=true&spread=3") ~> prometheusAPIRoute ~> check {
+
+ handled shouldBe true
+ status shouldEqual StatusCodes.OK
+ contentType shouldEqual ContentTypes.`application/json`
+ val resp = responseAs[ExplainPlanResponse]
+ resp.status shouldEqual "success"
+ resp.debugInfo.filter(_.startsWith("--E~SelectRawPartitionsExec")).length shouldEqual 8
+ }
+ }
+
+ it("should take default spread value if there is no override") {
+ setupDataset()
+ val query = "heap_usage{_ns=\"App-1\"}"
+
+ Get(s"/promql/prometheus/api/v1/query_range?query=${query}&" +
+ s"start=1555427432&end=1555447432&step=15&explainOnly=true") ~> prometheusAPIRoute ~> check {
+
+ handled shouldBe true
+ status shouldEqual StatusCodes.OK
+ contentType shouldEqual ContentTypes.`application/json`
+ val resp = responseAs[ExplainPlanResponse]
+ resp.status shouldEqual "success"
+ resp.debugInfo.filter(_.startsWith("--E~SelectRawPartitionsExec")).length shouldEqual 2
+ }
+ }
+}
+
diff --git a/jmh/src/main/scala/filodb.jmh/EncodingBenchmark.scala b/jmh/src/main/scala/filodb.jmh/EncodingBenchmark.scala
index eb5a06b6ca..9ea310d729 100644
--- a/jmh/src/main/scala/filodb.jmh/EncodingBenchmark.scala
+++ b/jmh/src/main/scala/filodb.jmh/EncodingBenchmark.scala
@@ -119,7 +119,7 @@ class EncodingBenchmark {
@BenchmarkMode(Array(Mode.Throughput))
@OutputTimeUnit(TimeUnit.SECONDS)
def newDictUtf8VectorEncoding(): Unit = {
- val hint = Encodings.AutoDictString(samplingRate=0.5)
+ val hint = Encodings.AutoDictString(samplingRate = 0.5)
UTF8Vector(memFactory, utf8strings).optimize(memFactory, hint)
}
}
\ No newline at end of file
diff --git a/jmh/src/main/scala/filodb.jmh/GatewayBenchmark.scala b/jmh/src/main/scala/filodb.jmh/GatewayBenchmark.scala
index 14232331b7..d4a805b8be 100644
--- a/jmh/src/main/scala/filodb.jmh/GatewayBenchmark.scala
+++ b/jmh/src/main/scala/filodb.jmh/GatewayBenchmark.scala
@@ -44,7 +44,7 @@ class GatewayBenchmark extends StrictLogging {
val singlePromTSBytes = timeseries(tagMap).toByteArray
- val singleInfluxRec = s"${tagMap("__name__")},${influxTags.map{case (k,v) => s"$k=$v"}.mkString(",")} " +
+ val singleInfluxRec = s"${tagMap("__name__")}, ${influxTags.map{case (k, v) => s"$k=$v"}.mkString(",")} " +
s"counter=$value ${initTimestamp}000000"
val singleInfluxBuf = ChannelBuffers.buffer(1024)
singleInfluxBuf.writeBytes(singleInfluxRec.getBytes)
@@ -52,7 +52,7 @@ class GatewayBenchmark extends StrictLogging {
// Histogram containing 8 buckets + sum and count
val histBuckets = Map("0.025" -> 0, "0.05" -> 0, "0.1" -> 2, "0.25" -> 2,
- "0.5" -> 5, "1.0" -> 9, "2.5" -> 11, "+Inf" -> 11)
+ "0.5" -> 5, "1.0" -> 9, "2.5" -> 11, "+Inf" -> 11)
val histSum = histBuckets.values.sum
val histPromSeries =
@@ -62,8 +62,8 @@ class GatewayBenchmark extends StrictLogging {
timeseries(tagMap ++ Map("__name__" -> "heap_usage_count"), histBuckets.size))
val histPromBytes = histPromSeries.map(_.toByteArray)
- val histInfluxRec = s"${tagMap("__name__")},${influxTags.map{case (k,v) => s"$k=$v"}.mkString(",")} " +
- s"${histBuckets.map { case (k,v) => s"$k=$v"}.mkString(",") },sum=$histSum,count=8 " +
+ val histInfluxRec = s"${tagMap("__name__")}, ${influxTags.map{case (k, v) => s"$k=$v"}.mkString(",")} " +
+ s"${histBuckets.map { case (k, v) => s"$k=$v"}.mkString(",") }, sum=$histSum,count=8 " +
s"${initTimestamp}000000"
val histInfluxBuf = ChannelBuffers.buffer(1024)
histInfluxBuf.writeBytes(histInfluxRec.getBytes)
diff --git a/jmh/src/main/scala/filodb.jmh/HistogramIngestBenchmark.scala b/jmh/src/main/scala/filodb.jmh/HistogramIngestBenchmark.scala
index f8df8dc4e4..ca73a59144 100644
--- a/jmh/src/main/scala/filodb.jmh/HistogramIngestBenchmark.scala
+++ b/jmh/src/main/scala/filodb.jmh/HistogramIngestBenchmark.scala
@@ -95,8 +95,8 @@ class HistogramIngestBenchmark {
@Benchmark
@BenchmarkMode(Array(Mode.SingleShotTime))
@OutputTimeUnit(TimeUnit.SECONDS)
- @Warmup(batchSize=50)
- @Measurement(batchSize=100)
+ @Warmup(batchSize = 50)
+ @Measurement(batchSize = 100)
def ingestHistColumn1(): Unit = {
hShard.ingest(histContainers(containerNo), 0)
containerNo += 1
@@ -113,8 +113,8 @@ class HistogramIngestBenchmark {
@Benchmark
@BenchmarkMode(Array(Mode.SingleShotTime))
@OutputTimeUnit(TimeUnit.SECONDS)
- @Warmup(batchSize=50)
- @Measurement(batchSize=100)
+ @Warmup(batchSize = 50)
+ @Measurement(batchSize = 100)
def ingestPromHistograms(): Unit = {
pShard.ingest(promContainers(containerNo), 0)
containerNo += 1
diff --git a/jmh/src/main/scala/filodb.jmh/HistogramQueryBenchmark.scala b/jmh/src/main/scala/filodb.jmh/HistogramQueryBenchmark.scala
index 72a252b20e..0bbc5b4bbd 100644
--- a/jmh/src/main/scala/filodb.jmh/HistogramQueryBenchmark.scala
+++ b/jmh/src/main/scala/filodb.jmh/HistogramQueryBenchmark.scala
@@ -85,7 +85,8 @@ class HistogramQueryBenchmark {
// Single-threaded query test
val numQueries = 500
- val qOptions = QueryOptions(1, 100).copy(shardOverrides = Some(Seq(0)))
+ val qOptions = QueryOptions(Some(new StaticSpreadProvider(SpreadChange(0, 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")
diff --git a/jmh/src/main/scala/filodb.jmh/IngestionBenchmark.scala b/jmh/src/main/scala/filodb.jmh/IngestionBenchmark.scala
index a78dd553ec..baabb4ddc5 100644
--- a/jmh/src/main/scala/filodb.jmh/IngestionBenchmark.scala
+++ b/jmh/src/main/scala/filodb.jmh/IngestionBenchmark.scala
@@ -30,7 +30,7 @@ class IngestionBenchmark {
org.slf4j.LoggerFactory.getLogger("filodb").asInstanceOf[Logger].setLevel(Level.ERROR)
// # of records in a container to test ingestion speed
- val dataStream = withMap(linearMultiSeries(), extraTags=extraTags)
+ val dataStream = withMap(linearMultiSeries(), extraTags = extraTags)
val schemaWithPredefKeys = RecordSchema.ingestion(dataset2,
Seq("job", "instance"))
diff --git a/jmh/src/main/scala/filodb.jmh/IntSumReadBenchmark.scala b/jmh/src/main/scala/filodb.jmh/IntSumReadBenchmark.scala
index 382ea8c12e..377ea0699d 100644
--- a/jmh/src/main/scala/filodb.jmh/IntSumReadBenchmark.scala
+++ b/jmh/src/main/scala/filodb.jmh/IntSumReadBenchmark.scala
@@ -9,12 +9,13 @@ import org.openjdk.jmh.annotations._
import scalaxy.loops._
import filodb.core.{NamesTestData, TestData}
-import filodb.core.metadata.Dataset
+import filodb.core.metadata.{Dataset, DatasetOptions}
import filodb.core.store.ChunkSet
import filodb.memory.format.{vectors => bv, TupleRowReader, UnsafeUtils}
object IntSumReadBenchmark {
- val dataset = Dataset("dataset", Seq("part:int"), Seq("int:int", "rownum:long"), "rownum")
+ val dataset = Dataset("dataset", Seq("part:int"), Seq("int:int", "rownum:long"), "rownum",
+ DatasetOptions.DefaultOptions)
val rowIt = Iterator.from(0).map { row => (Some(scala.util.Random.nextInt), Some(row.toLong), Some(0)) }
val partKey = NamesTestData.defaultPartKey
val rowColumns = Seq("int", "rownum", "part")
diff --git a/jmh/src/main/scala/filodb.jmh/QueryAndIngestBenchmark.scala b/jmh/src/main/scala/filodb.jmh/QueryAndIngestBenchmark.scala
index 8a0df92511..9124fb08d9 100644
--- a/jmh/src/main/scala/filodb.jmh/QueryAndIngestBenchmark.scala
+++ b/jmh/src/main/scala/filodb.jmh/QueryAndIngestBenchmark.scala
@@ -128,7 +128,7 @@ class QueryAndIngestBenchmark extends StrictLogging {
val qParams = TimeStepParams(queryTime/1000, queryStep, (queryTime/1000) + queryIntervalMin*60)
val logicalPlans = queries.map { q => Parser.queryRangeToLogicalPlan(q, qParams) }
val queryCommands = logicalPlans.map { plan =>
- LogicalPlan2Query(dataset.ref, plan, QueryOptions(1, 1000000))
+ LogicalPlan2Query(dataset.ref, plan, QueryOptions(Some(new StaticSpreadProvider(SpreadChange(0, 1))), 1000000))
}
private var testProducingFut: Option[Future[Unit]] = None
diff --git a/jmh/src/main/scala/filodb.jmh/QueryLeafScanInMemoryBenchmark.scala b/jmh/src/main/scala/filodb.jmh/QueryHiCardInMemoryBenchmark.scala
similarity index 88%
rename from jmh/src/main/scala/filodb.jmh/QueryLeafScanInMemoryBenchmark.scala
rename to jmh/src/main/scala/filodb.jmh/QueryHiCardInMemoryBenchmark.scala
index 026b08897c..7d6e7288da 100644
--- a/jmh/src/main/scala/filodb.jmh/QueryLeafScanInMemoryBenchmark.scala
+++ b/jmh/src/main/scala/filodb.jmh/QueryHiCardInMemoryBenchmark.scala
@@ -24,10 +24,11 @@ import filodb.timeseries.TestTimeseriesProducer
//scalastyle:off regex
/**
- * Benchmark for scan performed at the lowest leaf of an ExecPlan tree for various queries
+ * Benchmark for high-cardinality sum() aggregation, each query aggregates 2000 time series.
+ * Scan is performed at the lowest leaf of an ExecPlan tree for various queries
*/
@State(Scope.Thread)
-class QueryLeafScanInMemoryBenchmark extends StrictLogging {
+class QueryHiCardInMemoryBenchmark extends StrictLogging {
org.slf4j.LoggerFactory.getLogger("filodb").asInstanceOf[Logger].setLevel(Level.WARN)
import filodb.coordinator._
@@ -36,9 +37,9 @@ class QueryLeafScanInMemoryBenchmark extends StrictLogging {
import NodeClusterActor._
val numShards = 1
- val numSeries = 200
- val samplesDuration = 2.hours
- val numSamples = numSeries * (samplesDuration / 10.seconds).toInt
+ val numSeries = 8000 // NOTE: num of time series queried is this / 4
+ val samplesDuration = 15.minutes
+ val numSamples = (samplesDuration / 10.seconds).toInt // # samples PER TIME SERIES
val ingestionStartTime = System.currentTimeMillis - samplesDuration.toMillis
val spread = 0
val config = ConfigFactory.load("filodb-defaults.conf")
@@ -65,8 +66,8 @@ class QueryLeafScanInMemoryBenchmark extends StrictLogging {
val storeConf = StoreConfig(ConfigFactory.parseString("""
| flush-interval = 1h
- | shard-mem-size = 512MB
- | ingestion-buffer-mem-size = 50MB
+ | shard-mem-size = 2GB
+ | ingestion-buffer-mem-size = 1GB
| groups-per-shard = 4
| demand-paging-enabled = false
""".stripMargin))
@@ -79,7 +80,6 @@ class QueryLeafScanInMemoryBenchmark extends StrictLogging {
Thread sleep 2000 // Give setup command some time to set up dataset shards etc.
val (producingFut, containerStream) = TestTimeseriesProducer.metricsToContainerStream(ingestionStartTime,
numShards, numSeries, numSamples * numSeries, dataset, shardMapper, spread)
- println(s"Ingesting $numSamples samples")
val ingestTask = containerStream.groupBy(_._1)
// Asynchronously subcribe and ingest each shard
.mapAsync(numShards) { groupedStream =>
@@ -102,7 +102,7 @@ class QueryLeafScanInMemoryBenchmark extends StrictLogging {
import filodb.coordinator.queryengine2.QueryEngine
val engine = new QueryEngine(dataset, shardMapper)
- val numQueries = 500 // Please make sure this number matches the OperationsPerInvocation below
+ val numQueries = 100 // Please make sure this number matches the OperationsPerInvocation below
val queryIntervalSec = samplesDuration.toSeconds // # minutes between start and stop
val queryStep = 150 // # of seconds between each query sample "step"
@@ -110,7 +110,8 @@ class QueryLeafScanInMemoryBenchmark extends StrictLogging {
private def toExecPlan(query: String): ExecPlan = {
val queryStartTime = ingestionStartTime + 5.minutes.toMillis // 5 minutes from start until 60 minutes from start
val qParams = TimeStepParams(queryStartTime/1000, queryStep, queryStartTime/1000 + queryIntervalSec)
- val execPlan = engine.materialize(Parser.queryRangeToLogicalPlan(query, qParams), QueryOptions(0, 20000))
+ val execPlan = engine.materialize(Parser.queryRangeToLogicalPlan(query, qParams),
+ QueryOptions(Some(new StaticSpreadProvider(SpreadChange(0, 0))), 20000))
var child = execPlan
while (child.children.size > 0) child = child.children(0)
child
@@ -125,7 +126,7 @@ class QueryLeafScanInMemoryBenchmark extends StrictLogging {
@Benchmark
@BenchmarkMode(Array(Mode.AverageTime))
@OutputTimeUnit(TimeUnit.MICROSECONDS)
- @OperationsPerInvocation(500)
+ @OperationsPerInvocation(100)
def scanSumOfRateBenchmark(): Unit = {
(0 until numQueries).foreach { _ =>
Await.result(scanSumOfRate.execute(store, dataset, queryConfig).runAsync, 60.seconds)
diff --git a/jmh/src/main/scala/filodb.jmh/QueryInMemoryBenchmark.scala b/jmh/src/main/scala/filodb.jmh/QueryInMemoryBenchmark.scala
index 66b46d3b49..c40f76de83 100644
--- a/jmh/src/main/scala/filodb.jmh/QueryInMemoryBenchmark.scala
+++ b/jmh/src/main/scala/filodb.jmh/QueryInMemoryBenchmark.scala
@@ -119,7 +119,7 @@ class QueryInMemoryBenchmark extends StrictLogging {
val qParams = TimeStepParams(queryTime/1000, queryStep, (queryTime/1000) + queryIntervalMin*60)
val logicalPlans = queries.map { q => Parser.queryRangeToLogicalPlan(q, qParams) }
val queryCommands = logicalPlans.map { plan =>
- LogicalPlan2Query(dataset.ref, plan, QueryOptions(1, 20000))
+ LogicalPlan2Query(dataset.ref, plan, QueryOptions(Some(new StaticSpreadProvider(SpreadChange(0, 1))), 20000))
}
@TearDown
@@ -148,7 +148,7 @@ class QueryInMemoryBenchmark extends StrictLogging {
val qParams2 = TimeStepParams(queryTime/1000, noOverlapStep, (queryTime/1000) + queryIntervalMin*60)
val logicalPlans2 = queries.map { q => Parser.queryRangeToLogicalPlan(q, qParams2) }
val queryCommands2 = logicalPlans2.map { plan =>
- LogicalPlan2Query(dataset.ref, plan, QueryOptions(1, 10000))
+ LogicalPlan2Query(dataset.ref, plan, QueryOptions(Some(new StaticSpreadProvider(SpreadChange(0, 1))), 10000))
}
@Benchmark
@@ -168,7 +168,7 @@ class QueryInMemoryBenchmark extends StrictLogging {
}
// Single-threaded query test
- val qOptions = QueryOptions(1, 10000)
+ val qOptions = QueryOptions(Some(new StaticSpreadProvider(SpreadChange(0, 1))), 10000)
val logicalPlan = Parser.queryRangeToLogicalPlan(rawQuery, qParams)
// Pick the children nodes, not the DistConcatExec. Thus we can run in a single thread this way
val execPlan = engine.materialize(logicalPlan, qOptions).children.head
diff --git a/jmh/src/main/scala/filodb.jmh/QueryOnDemandBenchmark.scala b/jmh/src/main/scala/filodb.jmh/QueryOnDemandBenchmark.scala
index 7f919bfb3c..dbc3945dbd 100644
--- a/jmh/src/main/scala/filodb.jmh/QueryOnDemandBenchmark.scala
+++ b/jmh/src/main/scala/filodb.jmh/QueryOnDemandBenchmark.scala
@@ -135,7 +135,7 @@ class QueryOnDemandBenchmark extends StrictLogging {
val qParams = TimeStepParams(queryTime/1000, queryStep, (queryTime/1000) + queryIntervalMin*60)
val logicalPlans = queries.map { q => Parser.queryRangeToLogicalPlan(q, qParams) }
val queryCommands = logicalPlans.map { plan =>
- LogicalPlan2Query(dataset.ref, plan, QueryOptions(1, 20000))
+ LogicalPlan2Query(dataset.ref, plan, QueryOptions(Some(new StaticSpreadProvider(SpreadChange(0, 1))), 20000))
}
@TearDown
diff --git a/kafka/src/main/scala/filodb/kafka/KafkaDownsamplePublisher.scala b/kafka/src/main/scala/filodb/kafka/KafkaDownsamplePublisher.scala
index 3b3981fbb0..730e026ff1 100644
--- a/kafka/src/main/scala/filodb/kafka/KafkaDownsamplePublisher.scala
+++ b/kafka/src/main/scala/filodb/kafka/KafkaDownsamplePublisher.scala
@@ -51,7 +51,7 @@ class KafkaDownsamplePublisher(downsampleConfig: Config) extends DownsamplePubli
override def start(): Unit = {
logger.info(s"Starting Kafka Downsampling Publisher. Will be publishing to $topics with config: $kafkaConfig")
- producer = new KafkaProducer(kafkaConfig.asJava)
+ producer = new KafkaProducer(kafkaConfig.asJava)
}
override def stop(): Unit = {
diff --git a/kafka/src/main/scala/filodb/kafka/PartitionedConsumerObservable.scala b/kafka/src/main/scala/filodb/kafka/PartitionedConsumerObservable.scala
index b425dea058..5dddd56bc5 100644
--- a/kafka/src/main/scala/filodb/kafka/PartitionedConsumerObservable.scala
+++ b/kafka/src/main/scala/filodb/kafka/PartitionedConsumerObservable.scala
@@ -51,7 +51,7 @@ private[filodb] class PartitionedConsumerObservable[K, V] private
consumer: Task[KafkaConsumer[K, V]])
extends Observable[ConsumerRecord[K, V]] {
- override def unsafeSubscribeFn(out: Subscriber[ConsumerRecord[K,V]]): Cancelable = {
+ override def unsafeSubscribeFn(out: Subscriber[ConsumerRecord[K, V]]): Cancelable = {
import out.scheduler
feedTask(out).runAsync(new Callback[Unit] {
diff --git a/kafka/src/main/scala/filodb/kafka/TestConsumer.scala b/kafka/src/main/scala/filodb/kafka/TestConsumer.scala
index 4e814abf1d..52422a0d6f 100644
--- a/kafka/src/main/scala/filodb/kafka/TestConsumer.scala
+++ b/kafka/src/main/scala/filodb/kafka/TestConsumer.scala
@@ -18,8 +18,9 @@ import filodb.core.store.IngestionConfig
* 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.
+ * It will keep reading records, printing out the offsets of each record.
* Optional: pass in a second arg which is the offset to seek to.
+ * Optional: third arg which is key=value, allows filtering output by contents of any stringColumn
*/
object TestConsumer extends App {
val settings = new FilodbSettings()
@@ -28,6 +29,7 @@ object TestConsumer extends App {
val sourceConfPath = args(0)
val offsetOpt = args.drop(1).headOption.map(_.toLong)
val shard = if (args.length > 1) args(1).toInt else 0
+ val filterArg = if (args.length > 2) Some(args(2)) else None
val sourceConf = ConfigFactory.parseFile(new java.io.File(sourceConfPath))
//scalastyle:off
@@ -41,13 +43,33 @@ object TestConsumer extends App {
val ctor = Class.forName(ingestConf.streamFactoryClass).getConstructors.head
val streamFactory = ctor.newInstance().asInstanceOf[IngestionStreamFactory]
+ // Figure out filter. What field # in BinaryRecord to filter by?
+ val (filterField, filterVal) =
+ filterArg.map { filt =>
+ val parts = filt.split('=')
+ if (parts.size == 2) {
+ val partColIndex = dataset.partitionColumns.indexWhere(_.name == parts(0))
+ if (partColIndex >= 0) { (dataset.ingestionSchema.partitionFieldStart.get + partColIndex, parts(1)) }
+ else { (-1, "") }
+ } else {
+ (-1, "")
+ }
+ }.getOrElse((-1, ""))
+
val stream = streamFactory.create(sourceConf, dataset, shard, offsetOpt)
- val fut = stream.get.take(10)
+ val fut = stream.get//.take(10)
.foreach { case SomeData(container, offset) =>
println(s"\n----- Offset $offset -----")
- container.foreach { case (base, offset) =>
- println(s" ${dataset.ingestionSchema.stringify(base, offset)}")
- }
+ // Use record reader to filter? Or maybe just use ingestionSchema getString etc.
+ if (filterField >= 0)
+ container.foreach { case (base, offset) =>
+ if (dataset.ingestionSchema.asJavaString(base, offset, filterField) == filterVal)
+ println(s" ${dataset.ingestionSchema.stringify(base, offset)}")
+ }
+ else
+ container.foreach { case (base, offset) =>
+ println(s" ${dataset.ingestionSchema.stringify(base, offset)}")
+ }
}
Await.result(fut, 10.minutes)
}
\ No newline at end of file
diff --git a/memory/src/main/scala/filodb.memory/BlockManager.scala b/memory/src/main/scala/filodb.memory/BlockManager.scala
index 7096858a8b..8cc17026b2 100644
--- a/memory/src/main/scala/filodb.memory/BlockManager.scala
+++ b/memory/src/main/scala/filodb.memory/BlockManager.scala
@@ -192,7 +192,7 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long,
// 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)}}")
+ s"usedBlocksTimeOrdered=${usedBlocksTimeOrdered.asScala.toList.map{case(n, l) => (n, l.size)}}")
}
def reclaimFrom(list: util.LinkedList[Block], reclaimedCounter: Counter): Unit = {
diff --git a/memory/src/main/scala/filodb.memory/MemFactory.scala b/memory/src/main/scala/filodb.memory/MemFactory.scala
index b02fc2701c..232dfcefef 100644
--- a/memory/src/main/scala/filodb.memory/MemFactory.scala
+++ b/memory/src/main/scala/filodb.memory/MemFactory.scala
@@ -1,12 +1,13 @@
package filodb.memory
import java.nio.ByteBuffer
-import java.util.concurrent.atomic.{AtomicLong, AtomicReference}
+import java.util.concurrent.atomic.AtomicReference
import scala.collection.mutable.ListBuffer
import com.kenai.jffi.MemoryIO
import com.typesafe.scalalogging.StrictLogging
+import kamon.Kamon
import filodb.memory.BinaryRegion.Memory
import filodb.memory.format.UnsafeUtils
@@ -28,9 +29,10 @@ trait MemFactory {
(UnsafeUtils.ZeroPointer, allocateOffheap(size), size)
/**
- * Allocates offheap memory and returns a native 64-bit pointer
- * @param size Request memory allocation size in bytes
- * @param zero if true, zeroes out the contents of the memory first
+ * Allocates offheap memory and returns a native 64-bit pointer, throwing
+ * OutOfOffheapMemoryException if no memory is available.
+ * @param size Request memory allocation size in bytes
+ * @param zero if true, zeroes out the contents of the memory first
*/
def allocateOffheap(size: Int, zero: Boolean = false): BinaryRegion.NativePointer
@@ -46,6 +48,11 @@ trait MemFactory {
*/
def numFreeBytes: Long
+ /**
+ * Call to update (and publish) stats associated with this factory. Implementation might do nothing.
+ */
+ def updateStats(): Unit = {}
+
def fromBuffer(buf: ByteBuffer): Memory = {
if (buf.hasArray) {
(buf.array, UnsafeUtils.arayOffset.toLong + buf.arrayOffset + buf.position(), buf.limit() - buf.position())
@@ -74,43 +81,60 @@ object MemFactory {
* first four bytes. That in fact matches what is needed for BinaryVector and BinaryRecord allocations.
* Have an allocateOffheapWithSizeHeader which just returns the address to the size bytes :)
* For now we still get millions of allocations/sec with synchronized
+ *
+ * @param tags Kamon tags used by updateStats method
*/
-class NativeMemoryManager(val upperBoundSizeInBytes: Long) extends MemFactory {
- protected val usedSoFar = new AtomicLong(0)
- protected val sizeMapping = debox.Map.empty[Long, Int]
+class NativeMemoryManager(val upperBoundSizeInBytes: Long, val tags: Map[String, String] = Map.empty)
+ extends MemFactory {
+
+ val statFree = Kamon.gauge("memstore-writebuffer-bytes-free").refine(tags)
+ val statUsed = Kamon.gauge("memstore-writebuffer-bytes-used").refine(tags)
+ val statEntries = Kamon.gauge("memstore-writebuffer-entries").refine(tags)
- def usedMemory: Long = usedSoFar.get()
+ private val sizeMapping = debox.Map.empty[Long, Int]
+ @volatile private var usedSoFar = 0L
- def availableDynMemory: Long = upperBoundSizeInBytes - usedSoFar.get()
+ def usedMemory: Long = usedSoFar
- def numFreeBytes: Long = availableDynMemory
+ def numFreeBytes: Long = upperBoundSizeInBytes - usedSoFar
// Allocates a native 64-bit pointer, or throws an exception if not enough space
- def allocateOffheap(size: Int, zero: Boolean = true): BinaryRegion.NativePointer = synchronized {
- val currentSize = usedSoFar.get()
- val resultantSize = currentSize + size
- if (!(resultantSize > upperBoundSizeInBytes)) {
+ def allocateOffheap(size: Int, zero: Boolean = true): BinaryRegion.NativePointer = {
+ var currentSize = usedSoFar
+
+ if (currentSize + size <= upperBoundSizeInBytes) {
+ // Optimistically allocate without being synchronized.
val address: Long = MemoryIO.getCheckedInstance().allocateMemory(size, zero)
- usedSoFar.compareAndSet(currentSize, currentSize + size)
- sizeMapping(address) = size
- address
- } else {
- throw OutOfOffheapMemoryException(size, availableDynMemory)
+
+ synchronized {
+ currentSize = usedSoFar
+ if (currentSize + size <= upperBoundSizeInBytes) {
+ // Still within the upper bound, so all is good.
+ usedSoFar = currentSize + size;
+ sizeMapping(address) = size
+ return address
+ }
+ }
+
+ // Allocated too much due to optimistic failure, so free it.
+ MemoryIO.getCheckedInstance().freeMemory(address)
}
+
+ throw OutOfOffheapMemoryException(size, upperBoundSizeInBytes - currentSize)
}
- override def freeMemory(startAddress: Long): Unit = synchronized {
- val address = startAddress
- val size = sizeMapping.getOrElse(address, -1)
- if (size >= 0) {
- val currentSize = usedSoFar.get()
- MemoryIO.getCheckedInstance().freeMemory(address)
- usedSoFar.compareAndSet(currentSize, currentSize - size)
- val removed = sizeMapping.remove(address)
- } else {
- val msg = s"Address $address was not allocated by this memory manager"
- throw new IllegalArgumentException(msg)
+ override def freeMemory(address: Long): Unit = {
+ synchronized {
+ val size = sizeMapping.getOrElse(address, -1)
+ if (size < 0) {
+ val msg = s"Address $address was not allocated by this memory manager"
+ throw new IllegalArgumentException(msg)
+ }
+ sizeMapping.remove(address)
+ usedSoFar -= size
}
+
+ MemoryIO.getCheckedInstance().freeMemory(address)
}
protected[memory] def freeAll(): Unit = synchronized {
@@ -118,7 +142,18 @@ class NativeMemoryManager(val upperBoundSizeInBytes: Long) extends MemFactory {
MemoryIO.getCheckedInstance().freeMemory(addr)
}
sizeMapping.clear()
- usedSoFar.set(0)
+ usedSoFar = 0
+ }
+
+ override def updateStats(): Unit = {
+ val used = usedSoFar
+ statUsed.set(used)
+ statFree.set(upperBoundSizeInBytes - used)
+ statEntries.set(entries)
+ }
+
+ private def entries = synchronized {
+ sizeMapping.size
}
def shutdown(): Unit = {
diff --git a/memory/src/main/scala/filodb.memory/data/ChunkMap.scala b/memory/src/main/scala/filodb.memory/data/ChunkMap.scala
index c852b89d16..b7ba4ef8f7 100644
--- a/memory/src/main/scala/filodb.memory/data/ChunkMap.scala
+++ b/memory/src/main/scala/filodb.memory/data/ChunkMap.scala
@@ -1,7 +1,8 @@
package filodb.memory.data
-import scala.collection.mutable.HashMap
-import scala.collection.mutable.Map
+import java.util.concurrent.ConcurrentHashMap
+
+import scala.collection.mutable.{HashMap, Map}
import scala.concurrent.duration._
import com.typesafe.scalalogging.StrictLogging
@@ -9,6 +10,7 @@ import kamon.Kamon
import filodb.memory.BinaryRegion.NativePointer
import filodb.memory.MemFactory
+import filodb.memory.OutOfOffheapMemoryException
import filodb.memory.format.UnsafeUtils
/**
@@ -48,16 +50,24 @@ object ChunkMap extends StrictLogging {
classOf[ChunkMap].getDeclaredField("lockState"))
private val InitialExclusiveRetryTimeoutNanos = 1.millisecond.toNanos
- private val MaxExclusiveRetryTimeoutNanos = 1.second.toNanos
+ private val MaxExclusiveRetryTimeoutNanos = 1.minute.toNanos
private val exclusiveLockWait = Kamon.counter("memory-exclusive-lock-waits")
private val sharedLockLingering = Kamon.counter("memory-shared-lock-lingering")
+ private val chunkEvictions = Kamon.counter("memory-chunk-evictions")
// Tracks all the shared locks held, by each thread.
private val sharedLockCounts = new ThreadLocal[Map[ChunkMap, Int]] {
override def initialValue() = new HashMap[ChunkMap, Int]
}
+ /**
+ * FIXME: Remove this after debugging is done.
+ * This keeps track of which thread is running which execPlan.
+ * Entry is added on lock acquisition, removed when lock is released.
+ */
+ private val execPlanTracker = new ConcurrentHashMap[Thread, String]
+
// Returns true if the current thread has acquired the shared lock at least once.
private def hasSharedLock(inst: ChunkMap): Boolean = sharedLockCounts.get.contains(inst)
@@ -83,6 +93,7 @@ object ChunkMap extends StrictLogging {
*/
//scalastyle:off null
def releaseAllSharedLocks(): Int = {
+ execPlanTracker.remove(Thread.currentThread())
var total = 0
val countMap = sharedLockCounts.get
if (countMap != null) {
@@ -109,12 +120,18 @@ object ChunkMap extends StrictLogging {
* consumption from a query iterator. If there are lingering locks,
* it is quite possible a lock acquire or release bug exists
*/
- def validateNoSharedLocks(): Unit = {
+ def validateNoSharedLocks(execPlan: String): Unit = {
+ val t = Thread.currentThread()
+ if (execPlanTracker.containsKey(t)) {
+ logger.debug(s"Current thread ${t.getName} did not release lock for execPlan: ${execPlanTracker.get(t)}")
+ }
+
val numLocksReleased = ChunkMap.releaseAllSharedLocks()
if (numLocksReleased > 0) {
logger.warn(s"Number of locks was non-zero: $numLocksReleased. " +
s"This is indicative of a possible lock acquisition/release bug.")
}
+ execPlanTracker.put(t, execPlan)
}
}
@@ -122,13 +139,13 @@ object ChunkMap extends StrictLogging {
* @param memFactory a THREAD-SAFE factory for allocating offheap space
* @param capacity initial capacity of the map; must be more than 0
*/
-class ChunkMap(val memFactory: MemFactory, var capacity: Int) {
+class ChunkMap(val memFactory: MemFactory, var capacity: Int) extends StrictLogging {
require(capacity > 0)
private var lockState: Int = 0
private var size: Int = 0
private var first: Int = 0
- private var arrayPtr = memFactory.allocateOffheap(capacity << 3, zero=true)
+ private var arrayPtr = memFactory.allocateOffheap(capacity << 3, zero = true)
import ChunkMap._
@@ -238,6 +255,8 @@ class ChunkMap(val memFactory: MemFactory, var capacity: Int) {
var timeoutNanos = InitialExclusiveRetryTimeoutNanos
var warned = false
+ // scalastyle:off null
+ var locks1: ConcurrentHashMap[Thread, String] = null
while (true) {
if (tryAcquireExclusive(timeoutNanos)) {
return
@@ -254,7 +273,16 @@ class ChunkMap(val memFactory: MemFactory, var capacity: Int) {
}
exclusiveLockWait.increment()
_logger.warn(s"Waiting for exclusive lock: $this")
+ locks1 = new ConcurrentHashMap[Thread, String](execPlanTracker)
warned = true
+ } else if (warned && timeoutNanos >= MaxExclusiveRetryTimeoutNanos) {
+ val locks2 = new ConcurrentHashMap[Thread, String](execPlanTracker)
+ locks2.entrySet().retainAll(locks1.entrySet())
+ val lockState = UnsafeUtils.getIntVolatile(this, lockStateOffset)
+ logger.error(s"Following execPlan locks have not been released for a while: " +
+ s"$locks2 $locks1 $execPlanTracker $lockState")
+ logger.error(s"Shutting down process since it may be in an unstable/corrupt state.")
+ Runtime.getRuntime.halt(1)
}
}
}
@@ -376,10 +404,15 @@ class ChunkMap(val memFactory: MemFactory, var capacity: Int) {
* exclusive lock.
* @param element the native pointer to the offheap element; must be able to apply
* chunkmapKeyRetrieve to it to get the key
+ * @param evictKey The highest key which can be evicted (removed) if necessary to make room
+ * if no additional native memory can be allocated. The memory for the evicted chunks isn't
+ * freed here, under the assumption that the chunk is a part of a Block, which gets evicted
+ * later. This in turn calls chunkmapDoRemove, which does nothing because the chunk reference
+ * was already removed.
*/
- final def chunkmapDoPut(element: NativePointer): Unit = {
+ final def chunkmapDoPut(element: NativePointer, evictKey: Long = Long.MinValue): Unit = {
require(element != 0)
- chunkmapDoPut(chunkmapKeyRetrieve(element), element)
+ chunkmapDoPut(chunkmapKeyRetrieve(element), element, evictKey)
}
/**
@@ -387,18 +420,18 @@ class ChunkMap(val memFactory: MemFactory, var capacity: Int) {
* already in the map. Caller must hold exclusive lock.
* @return true if the element was inserted, false otherwise
*/
- final def chunkmapDoPutIfAbsent(element: NativePointer): Boolean = {
+ final def chunkmapDoPutIfAbsent(element: NativePointer, evictKey: Long = Long.MinValue): Boolean = {
require(element != 0)
val key = chunkmapKeyRetrieve(element)
if (doBinarySearch(key) >= 0) {
return false
}
- chunkmapDoPut(key, element)
+ chunkmapDoPut(key, element, evictKey)
true
}
//scalastyle:off
- private def chunkmapDoPut(key: Long, element: NativePointer): Unit = {
+ private def chunkmapDoPut(key: Long, element: NativePointer, evictKey: Long): Unit = {
if (size == 0) {
arraySet(0, element)
first = 0
@@ -409,20 +442,36 @@ class ChunkMap(val memFactory: MemFactory, var capacity: Int) {
// Ensure enough capacity, under the assumption that in most cases the element is
// inserted and not simply replaced.
if (size >= capacity) {
- val newArrayPtr = memFactory.allocateOffheap(capacity << 4, zero=true)
- if (first == 0) {
- // No wraparound.
- UnsafeUtils.unsafe.copyMemory(arrayPtr, newArrayPtr, size << 3)
- } else {
- // Wraparound correction.
- val len = (capacity - first) << 3
- UnsafeUtils.unsafe.copyMemory(arrayPtr + (first << 3), newArrayPtr, len)
- UnsafeUtils.unsafe.copyMemory(arrayPtr, newArrayPtr + len, first << 3)
- first = 0
+ try {
+ val newArrayPtr = memFactory.allocateOffheap(capacity << 4, zero=true)
+ if (first == 0) {
+ // No wraparound.
+ UnsafeUtils.unsafe.copyMemory(arrayPtr, newArrayPtr, size << 3)
+ } else {
+ // Wraparound correction.
+ val len = (capacity - first) << 3
+ UnsafeUtils.unsafe.copyMemory(arrayPtr + (first << 3), newArrayPtr, len)
+ UnsafeUtils.unsafe.copyMemory(arrayPtr, newArrayPtr + len, first << 3)
+ first = 0
+ }
+ memFactory.freeMemory(arrayPtr)
+ arrayPtr = newArrayPtr
+ capacity <<= 1
+ } catch {
+ case e: OutOfOffheapMemoryException => {
+ // Try to evict the first entry instead of expanding the array.
+ if (evictKey == Long.MinValue || chunkmapKeyRetrieve(arrayGet(first)) > evictKey) {
+ throw e
+ }
+ chunkEvictions.increment()
+ first += 1
+ if (first >= capacity) {
+ // Wraparound.
+ first = 0
+ }
+ size -= 1
+ }
}
- memFactory.freeMemory(arrayPtr)
- arrayPtr = newArrayPtr
- capacity <<= 1
}
{
diff --git a/memory/src/main/scala/filodb.memory/format/BinaryVector.scala b/memory/src/main/scala/filodb.memory/format/BinaryVector.scala
index 69f5ef4cb8..eb3b66439d 100644
--- a/memory/src/main/scala/filodb.memory/format/BinaryVector.scala
+++ b/memory/src/main/scala/filodb.memory/format/BinaryVector.scala
@@ -542,7 +542,7 @@ extends BinaryAppendableVector[A] {
UnsafeUtils.unsafe.setMemory(UnsafeUtils.ZeroPointer, bitmapOffset, bitmapMaskBufferSize, 0)
- UnsafeUtils.setInt(addr, 8 + bitmapMaskBufferSize)
+ UnsafeUtils.setInt(addr, 8 + bitmapMaskBufferSize)
BinaryVector.writeMajorAndSubType(addr, vectMajorType, vectSubType)
val subVectOffset = 12 + bitmapMaskBufferSize
UnsafeUtils.setInt(addr + 8, subVectOffset)
diff --git a/memory/src/main/scala/filodb.memory/format/NibblePack.scala b/memory/src/main/scala/filodb.memory/format/NibblePack.scala
index 0ab9d06287..ca77f63a0a 100644
--- a/memory/src/main/scala/filodb.memory/format/NibblePack.scala
+++ b/memory/src/main/scala/filodb.memory/format/NibblePack.scala
@@ -37,6 +37,7 @@ object NibblePack {
/**
* 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.
+ * @return the final position within the buffer after packing
*/
final def packDelta(input: Array[Long], buf: MutableDirectBuffer, bufindex: Int): Int = {
val inputArray = tempArray
@@ -62,6 +63,38 @@ object NibblePack {
pos
}
+ /**
+ * Packs Double values using XOR encoding to find minimal # of bits difference between successive values.
+ * Initial Double value is written first.
+ * @return the final position within the buffer after packing
+ */
+ final def packDoubles(inputs: Array[Double], buf: MutableDirectBuffer, bufindex: Int): Int = {
+ require(inputs.size > 0)
+ buf.putDouble(bufindex, inputs(0), LITTLE_ENDIAN)
+ var pos = bufindex + 8
+
+ val inputArray = tempArray
+ var last = java.lang.Double.doubleToLongBits(inputs(0))
+ var i = 0
+ while (i < (inputs.size - 1)) {
+ val bits = java.lang.Double.doubleToLongBits(inputs(i + 1))
+ inputArray(i % 8) = bits ^ last
+ last = bits
+ 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.
@@ -179,6 +212,19 @@ object NibblePack {
}
}
+ final case class DoubleXORSink(outArray: Array[Double], initial: Long) extends Sink {
+ private var lastBits = initial
+ private var pos: Int = 1
+ def process(data: Long): Unit = {
+ if (pos < outArray.size) {
+ val nextBits = lastBits ^ data
+ outArray(pos) = java.lang.Double.longBitsToDouble(nextBits)
+ lastBits = nextBits
+ 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
@@ -201,6 +247,18 @@ object NibblePack {
res
}
+ final def unpackDoubleXOR(compressed: DirectBuffer, outArray: Array[Double]): UnpackResult = {
+ if (compressed.capacity < 8) {
+ InputTooShort
+ } else {
+ val initVal = readLong(compressed, 0)
+ val sink = DoubleXORSink(outArray, initVal)
+ outArray(0) = java.lang.Double.longBitsToDouble(initVal)
+ subslice(compressed, 8)
+ unpackToSink(compressed, sink, outArray.size - 1)
+ }
+ }
+
/**
* 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
diff --git a/memory/src/main/scala/filodb.memory/format/RowToVectorBuilder.scala b/memory/src/main/scala/filodb.memory/format/RowToVectorBuilder.scala
index 6637eeb9be..ca123ee1b5 100644
--- a/memory/src/main/scala/filodb.memory/format/RowToVectorBuilder.scala
+++ b/memory/src/main/scala/filodb.memory/format/RowToVectorBuilder.scala
@@ -58,7 +58,7 @@ object RowToVectorBuilder {
class RowToVectorBuilder(schema: Seq[VectorInfo], memFactory: MemFactory) {
import RowToVectorBuilder._
val builders = schema.zipWithIndex.map {
- case (VectorInfo(_, dataType),index)=> dataType match {
+ case (VectorInfo(_, dataType), index)=> dataType match {
case Classes.Int => IntBinaryVector.appendingVector(memFactory, MaxElements)
case Classes.Long => LongBinaryVector.appendingVector(memFactory, MaxElements)
case Classes.Double => DoubleVector.appendingVector(memFactory, MaxElements)
diff --git a/memory/src/main/scala/filodb.memory/format/vectors/ConstVector.scala b/memory/src/main/scala/filodb.memory/format/vectors/ConstVector.scala
index 73fc8987a0..703bb90c8c 100644
--- a/memory/src/main/scala/filodb.memory/format/vectors/ConstVector.scala
+++ b/memory/src/main/scala/filodb.memory/format/vectors/ConstVector.scala
@@ -26,7 +26,7 @@ object ConstVector {
*/
def make(memFactory: MemFactory, len: Int, neededBytes: Int)(fillBytes: NativePointer => Unit): BinaryVectorPtr = {
val (_, addr, _) = memFactory.allocate(12 + neededBytes)
- UnsafeUtils.setInt(ZeroPointer, addr, 8 + neededBytes)
+ UnsafeUtils.setInt(ZeroPointer, addr, 8 + neededBytes)
UnsafeUtils.setInt(ZeroPointer, addr + 4, WireFormat(WireFormat.VECTORTYPE_BINSIMPLE, WireFormat.SUBTYPE_REPEATED))
UnsafeUtils.setInt(ZeroPointer, addr + 8, len)
fillBytes(addr + DataOffset)
diff --git a/memory/src/main/scala/filodb.memory/format/vectors/DeltaDeltaVector.scala b/memory/src/main/scala/filodb.memory/format/vectors/DeltaDeltaVector.scala
index b95b1f4fa0..5bbfe734d6 100644
--- a/memory/src/main/scala/filodb.memory/format/vectors/DeltaDeltaVector.scala
+++ b/memory/src/main/scala/filodb.memory/format/vectors/DeltaDeltaVector.scala
@@ -68,7 +68,10 @@ object DeltaDeltaVector {
minMax <- getDeltasMinMax(inputVect, slope)
nbitsSigned <- getNbitsSignedFromMinMax(minMax, maxNBits)
} yield {
- if (minMax.min == minMax.max) {
+ // Min and max == 0 for constant slope otherwise we can have some funny edge cases such as the first value
+ // being diff from all others which are the same (eg 55, 60, 60, ....). That results in erroneous Const
+ // encoding.
+ if (minMax.min == 0 && minMax.max == 0) {
const(memFactory, inputVect.length, inputVect(0), slope)
} else if (approxConst && minMax.min >= MinApproxDelta && minMax.max <= MaxApproxDelta) {
const(memFactory, inputVect.length, inputVect(0), slope)
@@ -93,7 +96,7 @@ object DeltaDeltaVector {
*/
def const(memFactory: MemFactory, numElements: Int, initValue: Long, slope: Int): BinaryVectorPtr = {
val addr = memFactory.allocateOffheap(24)
- UnsafeUtils.setInt(addr, 20)
+ UnsafeUtils.setInt(addr, 20)
UnsafeUtils.setInt(addr + 4, WireFormat(VECTORTYPE_DELTA2, SUBTYPE_REPEATED))
UnsafeUtils.setInt(addr + 8, numElements)
UnsafeUtils.setLong(addr + 12, initValue)
@@ -141,8 +144,6 @@ object DeltaDeltaVector {
* Thus overall header for DDV = 28 bytes
*/
object DeltaDeltaDataReader extends LongVectorDataReader {
- import BinaryRegion._
-
val InnerVectorOffset = 20
override def length(vector: BinaryVectorPtr): Int =
IntBinaryVector.simple(vector + InnerVectorOffset).length(vector + InnerVectorOffset)
@@ -150,12 +151,12 @@ object DeltaDeltaDataReader extends LongVectorDataReader {
final def slope(vector: BinaryVectorPtr): Int = UnsafeUtils.getInt(vector + 16)
final def apply(vector: BinaryVectorPtr, n: Int): Long = {
val inner = vector + InnerVectorOffset
- initValue(vector) + slope(vector) * n + IntBinaryVector.simple(inner)(inner, n)
+ initValue(vector) + slope(vector).toLong * n + IntBinaryVector.simple(inner)(inner, n)
}
// Should be close to O(1), initial guess should be almost spot on
def binarySearch(vector: BinaryVectorPtr, item: Long): Int = {
- val _slope = slope(vector)
+ val _slope = slope(vector).toLong
val _len = length(vector)
var elemNo = if (_slope == 0) { if (item <= initValue(vector)) 0 else length(vector) }
else { ((item - initValue(vector) + (_slope - 1)) / _slope).toInt }
@@ -192,7 +193,7 @@ object DeltaDeltaDataReader extends LongVectorDataReader {
}
// Efficient iterator as we keep track of current value and inner iterator
- class DeltaDeltaIterator(innerIt: IntIterator, slope: Int, var curBase: NativePointer) extends LongIterator {
+ class DeltaDeltaIterator(innerIt: IntIterator, slope: Int, var curBase: Long) extends LongIterator {
final def next: Long = {
val out: Long = curBase + innerIt.next
curBase += slope
@@ -203,7 +204,7 @@ object DeltaDeltaDataReader extends LongVectorDataReader {
final def iterate(vector: BinaryVectorPtr, startElement: Int = 0): LongIterator = {
val inner = vector + InnerVectorOffset
val innerIt = IntBinaryVector.simple(inner).iterate(inner, startElement)
- new DeltaDeltaIterator(innerIt, slope(vector), initValue(vector) + startElement * slope(vector))
+ new DeltaDeltaIterator(innerIt, slope(vector), initValue(vector) + startElement * slope(vector).toLong)
}
}
@@ -221,7 +222,7 @@ object DeltaDeltaConstDataReader extends LongVectorDataReader {
// This is O(1) since we can find exactly where on line it is
final def binarySearch(vector: BinaryVectorPtr, item: Long): Int = {
- val _slope = slope(vector)
+ val _slope = slope(vector).toLong
val guess = if (_slope == 0) { if (item <= initValue(vector)) 0 else length(vector) }
else { ((item - initValue(vector) + (_slope - 1)) / _slope).toInt }
if (guess < 0) { 0x80000000 }
@@ -241,11 +242,11 @@ object DeltaDeltaConstDataReader extends LongVectorDataReader {
private[memory] def slopeSum(initVal: Long, slope: Int, start: Int, end: Int): Double = {
val len = end - start + 1
- len.toDouble * (initVal + start * slope) + ((end-start)*len/2) * slope
+ len.toDouble * (initVal + start * slope.toLong) + ((end-start)*len/2) * slope.toLong
}
final def iterate(vector: BinaryVectorPtr, startElement: Int = 0): LongIterator = new LongIterator {
- private final var curBase = initValue(vector) + startElement * slope(vector)
+ private final var curBase = initValue(vector) + startElement * slope(vector).toLong
final def next: Long = {
val out = curBase
curBase += slope(vector)
@@ -275,7 +276,7 @@ class DeltaDeltaAppendingVector(val addr: BinaryRegion.NativePointer,
override def length: Int = deltas.length
final def isAvailable(index: Int): Boolean = true
- final def apply(index: Int): Long = initValue + slope * index + deltas(index)
+ final def apply(index: Int): Long = initValue + slope.toLong * index + deltas(index)
final def numBytes: Int = 20 + deltas.numBytes
final def reader: VectorDataReader = DeltaDeltaDataReader
final def copyToBuffer: Buffer[Long] = DeltaDeltaDataReader.toBuffer(addr)
diff --git a/memory/src/main/scala/filodb.memory/format/vectors/DictUTF8Vector.scala b/memory/src/main/scala/filodb.memory/format/vectors/DictUTF8Vector.scala
index 3e05c66c78..2869ae0ce6 100644
--- a/memory/src/main/scala/filodb.memory/format/vectors/DictUTF8Vector.scala
+++ b/memory/src/main/scala/filodb.memory/format/vectors/DictUTF8Vector.scala
@@ -87,7 +87,7 @@ object DictUTF8Vector {
codeVect.addVector(info.codes)
// Write 12 bytes of metadata at beginning
- UnsafeUtils.setInt(addr, bytesRequired - 4)
+ UnsafeUtils.setInt(addr, bytesRequired - 4)
UnsafeUtils.setInt(addr + 4, WireFormat(VECTORTYPE_BINDICT, SUBTYPE_UTF8))
UnsafeUtils.setInt(addr + 8, 12 + dictVectSize)
addr
diff --git a/memory/src/main/scala/filodb.memory/format/vectors/DoubleVector.scala b/memory/src/main/scala/filodb.memory/format/vectors/DoubleVector.scala
index aea1ec93d5..bc741b30df 100644
--- a/memory/src/main/scala/filodb.memory/format/vectors/DoubleVector.scala
+++ b/memory/src/main/scala/filodb.memory/format/vectors/DoubleVector.scala
@@ -20,8 +20,8 @@ object DoubleVector {
def appendingVector(memFactory: MemFactory, maxElements: Int): BinaryAppendableVector[Double] = {
val bytesRequired = 12 + BitmapMask.numBytesRequired(maxElements) + 8 + 8 * maxElements
val addr = memFactory.allocateOffheap(bytesRequired)
- val dispose = () => memFactory.freeMemory(addr)
- GrowableVector(memFactory,new MaskedDoubleAppendingVector(addr, bytesRequired, maxElements, dispose))
+ val dispose = () => memFactory.freeMemory(addr)
+ GrowableVector(memFactory, new MaskedDoubleAppendingVector(addr, bytesRequired, maxElements, dispose))
}
/**
@@ -32,7 +32,7 @@ object DoubleVector {
def appendingVectorNoNA(memFactory: MemFactory, maxElements: Int): BinaryAppendableVector[Double] = {
val bytesRequired = 8 + 8 * maxElements
val addr = memFactory.allocateOffheap(bytesRequired)
- val dispose = () => memFactory.freeMemory(addr)
+ val dispose = () => memFactory.freeMemory(addr)
new DoubleAppendingVector(addr, bytesRequired, dispose)
}
@@ -54,8 +54,8 @@ object DoubleVector {
* DoubleVectorDataReader object for parsing it
*/
def apply(vector: BinaryVectorPtr): DoubleVectorDataReader = BinaryVector.vectorType(vector) match {
- case x if x == WireFormat(VECTORTYPE_DELTA2, SUBTYPE_INT_NOMASK) => DoubleLongWrapDataReader
- case x if x == WireFormat(VECTORTYPE_DELTA2, SUBTYPE_REPEATED) => DoubleLongWrapDataReader
+ case x if x == WireFormat(VECTORTYPE_DELTA2, SUBTYPE_INT_NOMASK) => DoubleLongWrapDataReader
+ case x if x == WireFormat(VECTORTYPE_DELTA2, SUBTYPE_REPEATED) => DoubleLongWrapDataReader
case x if x == WireFormat(VECTORTYPE_BINSIMPLE, SUBTYPE_PRIMITIVE) => MaskedDoubleDataReader
case x if x == WireFormat(VECTORTYPE_BINSIMPLE, SUBTYPE_PRIMITIVE_NOMASK) => DoubleVectorDataReader64
}
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 ee8ac39214..d69f3b9255 100644
--- a/memory/src/main/scala/filodb.memory/format/vectors/Histogram.scala
+++ b/memory/src/main/scala/filodb.memory/format/vectors/Histogram.scala
@@ -136,9 +136,7 @@ final case class LongHistogram(buckets: HistogramBuckets, values: Array[Long]) e
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)
- }
+ BinaryHistogram.writeDelta(buckets, values, buf)
buf
}
}
@@ -158,9 +156,7 @@ final case class MutableHistogram(buckets: HistogramBuckets, values: Array[Doubl
final def bucketValue(no: Int): Double = values(no)
final def serialize(intoBuf: Option[MutableDirectBuffer] = None): MutableDirectBuffer = {
val buf = intoBuf.getOrElse(BinaryHistogram.histBuf)
- buckets match {
- case g: GeometricBuckets => BinaryHistogram.writeDelta(g, values.map(_.toLong), buf)
- }
+ BinaryHistogram.writeDelta(buckets, values.map(_.toLong), buf)
buf
}
@@ -258,12 +254,22 @@ sealed trait HistogramBuckets {
}
override def toString: String = allBucketTops.mkString("buckets[", ", ", "]")
+
+ /**
+ * 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
+ */
+ def serialize(buf: MutableDirectBuffer, pos: Int): Int
}
object HistogramBuckets {
val OffsetNumBuckets = 0
val OffsetBucketDetails = 2
+ val MAX_BUCKETS = 8192
+
import BinaryHistogram._
/**
@@ -274,6 +280,7 @@ object HistogramBuckets {
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 HistFormat_Custom_Delta => custom(buffer.byteArray, buffer.addressOffset)
case _ => emptyBuckets
}
@@ -281,6 +288,7 @@ object HistogramBuckets {
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 HistFormat_Custom_Delta => custom(UnsafeUtils.ZeroArray, bucketsDef.addr)
case _ => emptyBuckets
}
@@ -291,6 +299,21 @@ object HistogramBuckets {
UnsafeUtils.getShort(bucketsDefBase, bucketsDefOffset + OffsetNumBuckets).toInt,
minusOne)
+ /**
+ * Creates a CustomBuckets definition.
+ * @param bucketsDefOffset must point to the 2-byte length prefix of the bucket definition
+ */
+ def custom(bucketsDefBase: Array[Byte], bucketsDefOffset: Long): CustomBuckets = {
+ val numBuckets = UnsafeUtils.getShort(bucketsDefBase, bucketsDefOffset + 2) & 0x0ffff
+ val les = new Array[Double](numBuckets)
+ UnsafeUtils.wrapDirectBuf(bucketsDefBase,
+ bucketsDefOffset + 4,
+ (UnsafeUtils.getShort(bucketsDefBase, bucketsDefOffset) & 0xffff) - 2,
+ valuesBuf)
+ require(NibblePack.unpackDoubleXOR(valuesBuf, les) == NibblePack.Ok)
+ CustomBuckets(les)
+ }
+
// 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(2.0d, 2.0d, 64, minusOne = true)
@@ -312,12 +335,7 @@ final case class GeometricBuckets(firstBucket: Double,
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
@@ -328,3 +346,28 @@ final case class GeometricBuckets(firstBucket: Double,
pos + 2 + 2 + 8 + 8
}
}
+
+/**
+ * A bucketing scheme with custom bucket/LE values.
+ *
+ * Binary/serialized: short:numBytes, short:numBuckets, then NibblePacked XOR-compressed bucket/LE defs
+ */
+final case class CustomBuckets(les: Array[Double]) extends HistogramBuckets {
+ require(les.size < HistogramBuckets.MAX_BUCKETS)
+ def numBuckets: Int = les.size
+ def bucketTop(no: Int): Double = les(no)
+ final def serialize(buf: MutableDirectBuffer, pos: Int): Int = {
+ buf.putShort(pos + 2, les.size.toShort)
+ val finalPos = NibblePack.packDoubles(les, buf, pos + 4)
+ require((finalPos - pos) <= 65535, s"Packing of ${les.size} buckets takes too much space!")
+ buf.putShort(pos, (finalPos - pos - 2).toShort)
+ finalPos
+ }
+
+ override def equals(other: Any): Boolean = other match {
+ case CustomBuckets(otherLes) => les.toSeq == otherLes.toSeq
+ case other: Any => false
+ }
+
+ override def hashCode: Int = les.hashCode
+}
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 933d4e8bbb..7ee0e862e4 100644
--- a/memory/src/main/scala/filodb.memory/format/vectors/HistogramVector.scala
+++ b/memory/src/main/scala/filodb.memory/format/vectors/HistogramVector.scala
@@ -21,6 +21,7 @@ import filodb.memory.format.BinaryVector.BinaryVectorPtr
* 0x00 Empty/null histogram
* 0x03 geometric + NibblePacked delta Long values
* 0x04 geometric_1 + NibblePacked delta Long values (see [[HistogramBuckets]])
+ * 0x05 custom LE/bucket values + NibblePacked delta Long values
*
* +0003 u16 2-byte length of Histogram bucket definition
* +0005 [u8] Histogram bucket definition, see [[HistogramBuckets]]
@@ -45,6 +46,9 @@ object BinaryHistogram extends StrictLogging {
}
override def toString: String = s""
+ def debugStr: String = s"totalLen=$totalLength numBuckets=$numBuckets formatCode=$formatCode " +
+ s"bucketDef=$bucketDefNumBytes bytes valuesIndex=$valuesIndex values=$valuesNumBytes bytes"
+
/**
* 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
@@ -58,6 +62,9 @@ object BinaryHistogram extends StrictLogging {
case HistFormat_Geometric1_Delta =>
val bucketDef = HistogramBuckets.geometric(buf.byteArray, bucketDefOffset, true)
LongHistogram.fromPacked(bucketDef, valuesByteSlice).getOrElse(Histogram.empty)
+ case HistFormat_Custom_Delta =>
+ val bucketDef = HistogramBuckets.custom(buf.byteArray, bucketDefOffset - 2)
+ LongHistogram.fromPacked(bucketDef, valuesByteSlice).getOrElse(Histogram.empty)
case x =>
logger.debug(s"Unrecognizable histogram format code $x, returning empty histogram")
Histogram.empty
@@ -85,9 +92,11 @@ object BinaryHistogram extends StrictLogging {
val HistFormat_Null = 0x00.toByte
val HistFormat_Geometric_Delta = 0x03.toByte
val HistFormat_Geometric1_Delta = 0x04.toByte
+ val HistFormat_Custom_Delta = 0x05.toByte
def isValidFormatCode(code: Byte): Boolean =
- (code == HistFormat_Null) || (code == HistFormat_Geometric1_Delta) || (code == HistFormat_Geometric_Delta)
+ (code == HistFormat_Null) || (code == HistFormat_Geometric1_Delta) || (code == HistFormat_Geometric_Delta) ||
+ (code == HistFormat_Custom_Delta)
/**
* Writes binary histogram with geometric bucket definition and data which is non-increasing, but will be
@@ -110,8 +119,8 @@ object BinaryHistogram extends StrictLogging {
finalPos
}
- def writeDelta(buckets: GeometricBuckets, values: Array[Long]): Int =
- writeNonIncreasing(buckets, values, histBuf)
+ def writeDelta(buckets: HistogramBuckets, values: Array[Long]): Int =
+ writeDelta(buckets, values, histBuf)
/**
* Encodes binary histogram with geometric bucket definition and data which is strictly increasing and positive.
@@ -121,9 +130,13 @@ object BinaryHistogram extends StrictLogging {
* so it can grow.
* @return the number of bytes written, including the length prefix
*/
- def writeDelta(buckets: GeometricBuckets, values: Array[Long], buf: MutableDirectBuffer): Int = {
+ def writeDelta(buckets: HistogramBuckets, values: Array[Long], buf: MutableDirectBuffer): Int = {
require(buckets.numBuckets == values.size, s"Values array size of ${values.size} != ${buckets.numBuckets}")
- val formatCode = if (buckets.minusOne) HistFormat_Geometric1_Delta else HistFormat_Geometric_Delta
+ val formatCode = buckets match {
+ case g: GeometricBuckets if g.minusOne => HistFormat_Geometric1_Delta
+ case g: GeometricBuckets => HistFormat_Geometric_Delta
+ case c: CustomBuckets => HistFormat_Custom_Delta
+ }
buf.putByte(2, formatCode)
val valuesIndex = buckets.serialize(buf, 3)
@@ -140,7 +153,7 @@ object HistogramVector {
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 OffsetBucketDefSize = 9 // # of bytes of bucket definition
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
diff --git a/memory/src/main/scala/filodb.memory/format/vectors/LongBinaryVector.scala b/memory/src/main/scala/filodb.memory/format/vectors/LongBinaryVector.scala
index 3240ef3ab7..a150b7b276 100644
--- a/memory/src/main/scala/filodb.memory/format/vectors/LongBinaryVector.scala
+++ b/memory/src/main/scala/filodb.memory/format/vectors/LongBinaryVector.scala
@@ -19,7 +19,7 @@ object LongBinaryVector {
def appendingVector(memFactory: MemFactory, maxElements: Int): BinaryAppendableVector[Long] = {
val bytesRequired = 12 + BitmapMask.numBytesRequired(maxElements) + 8 + 8 * maxElements
val addr = memFactory.allocateOffheap(bytesRequired)
- val dispose = () => memFactory.freeMemory(addr)
+ val dispose = () => memFactory.freeMemory(addr)
GrowableVector(memFactory, new MaskedLongAppendingVector(addr, bytesRequired, maxElements, dispose))
}
@@ -30,7 +30,7 @@ object LongBinaryVector {
def appendingVectorNoNA(memFactory: MemFactory, maxElements: Int): BinaryAppendableVector[Long] = {
val bytesRequired = 8 + 8 * maxElements
val addr = memFactory.allocateOffheap(bytesRequired)
- val dispose = () => memFactory.freeMemory(addr)
+ val dispose = () => memFactory.freeMemory(addr)
new LongAppendingVector(addr, bytesRequired, dispose)
}
@@ -41,7 +41,7 @@ object LongBinaryVector {
def timestampVector(memFactory: MemFactory, maxElements: Int): BinaryAppendableVector[Long] = {
val bytesRequired = 8 + 8 * maxElements
val addr = memFactory.allocateOffheap(bytesRequired)
- val dispose = () => memFactory.freeMemory(addr)
+ val dispose = () => memFactory.freeMemory(addr)
new TimestampAppendingVector(addr, bytesRequired, dispose)
}
@@ -54,8 +54,8 @@ object LongBinaryVector {
* LongVectorDataReader object for parsing it
*/
def apply(vector: BinaryVectorPtr): LongVectorDataReader = BinaryVector.vectorType(vector) match {
- case x if x == WireFormat(VECTORTYPE_DELTA2, SUBTYPE_INT_NOMASK) => DeltaDeltaDataReader
- case x if x == WireFormat(VECTORTYPE_DELTA2, SUBTYPE_REPEATED) => DeltaDeltaConstDataReader
+ case x if x == WireFormat(VECTORTYPE_DELTA2, SUBTYPE_INT_NOMASK) => DeltaDeltaDataReader
+ case x if x == WireFormat(VECTORTYPE_DELTA2, SUBTYPE_REPEATED) => DeltaDeltaConstDataReader
case x if x == WireFormat(VECTORTYPE_BINSIMPLE, SUBTYPE_PRIMITIVE) => MaskedLongDataReader
case x if x == WireFormat(VECTORTYPE_BINSIMPLE, SUBTYPE_PRIMITIVE_NOMASK) => LongVectorDataReader64
}
diff --git a/memory/src/test/resources/timeseries_bug1.txt b/memory/src/test/resources/timeseries_bug1.txt
new file mode 100644
index 0000000000..00f21aaba3
--- /dev/null
+++ b/memory/src/test/resources/timeseries_bug1.txt
@@ -0,0 +1,117 @@
+1556726220000 3.058186633E9
+1556726280000 2.705809862E9
+1556726340000 3.049237033E9
+1556726400000 3.046266821E9
+1556726460000 2.693516135E9
+1556726520000 2.67155914E9
+1556726580000 3.010067451E9
+1556726640000 3.01355728E9
+1556726700000 2.682072219E9
+1556726760000 3.04152299E9
+1556726820000 2.703304098E9
+1556726880000 3.048756488E9
+1556726940000 2.712551286E9
+1556727000000 3.037394868E9
+1556727060000 3.051997896E9
+1556727120000 2.698401382E9
+1556727180000 3.036601543E9
+1556727240000 2.694740719E9
+1556727300000 3.032804455E9
+1556727360000 2.699330496E9
+1556727420000 3.010692296E9
+1556727480000 3.041792131E9
+1556727540000 2.6920077E9
+1556727600000 3.056571932E9
+1556727660000 2.700888138E9
+1556727720000 3.037392581E9
+1556727780000 2.698572505E9
+1556727840000 3.044995396E9
+1556727900000 3.022555165E9
+1556727960000 2.704270521E9
+1556728020000 3.041891853E9
+1556728080000 2.704237403E9
+1556728140000 3.030879522E9
+1556728200000 2.694017362E9
+1556728260000 3.024370585E9
+1556728320000 2.997038934E9
+1556728380000 2.686496473E9
+1556728440000 3.019983096E9
+1556728500000 2.69921766E9
+1556728560000 2.39745681E8
+1556728620000 2.1406499E7
+1556728680000 1.8867463E7
+1556728740000 2.1324913E7
+1556728800000 2.1325887E7
+1556728860000 1.9000008E7
+1556728920000 1.8995469E7
+1556728980000 2.147689E7
+1556729040000 2.1504414E7
+1556729100000 1.8062283E7
+1556729160000 2.2119509E7
+1556729220000 1.9271485E7
+1556729280000 2.1797443E7
+1556729340000 1.9364096E7
+1556729400000 2.1904196E7
+1556729460000 1.917167E7
+1556729520000 2.1467983E7
+1556729580000 2.1776499E7
+1556729640000 1.9550949E7
+1556729760000 2.0261423E7
+1556729820000 2.3110437E7
+1556729880000 2.058786E7
+1556729940000 2.355348E7
+1556730000000 2.3105697E7
+1556730060000 2.0550393E7
+1556730120000 2.3513976E7
+1556730180000 2.0633176E7
+1556730240000 2.2726853E7
+1556730300000 2.0795383E7
+1556730360000 2.3375281E7
+1556730420000 2.3326026E7
+1556730540000 2.2999678E7
+1556730600000 2.037102E7
+1556730660000 2.3324408E7
+1556730720000 2.0861379E7
+1556730780000 2.3690467E7
+1556730840000 2.3296293E7
+1556730900000 2.1246669E7
+1556730960000 2.5014739E7
+1556731020000 2.2217168E7
+1556731080000 2.401642E7
+1556731140000 2.0744796E7
+1556731200000 2.3078938E7
+1556731260000 2.3284528E7
+1556731320000 2.0219743E7
+1556731380000 2.3364032E7
+1556731440000 2.0871388E7
+1556731500000 2.3454416E7
+1556731560000 1.9740681E7
+1556731620000 2.2231099E7
+1556731680000 2.172768E7
+1556731740000 1.9389319E7
+1556731800000 2.2054833E7
+1556731860000 1.9258172E7
+1556731920000 2.1991292E7
+1556731980000 1.9653949E7
+1556732040000 2.2523137E7
+1556732100000 2.2239381E7
+1556732160000 1.9316076E7
+1556732220000 3.7530475E7
+1556732280000 4.6782123E7
+1556732340000 5.4185413E7
+1556732400000 5.4084217E7
+1556732460000 4.8514667E7
+1556732520000 5.6207724E7
+1556732580000 4.8256646E7
+1556732700000 4.7774409E7
+1556732760000 5.2815775E7
+1556732820000 3.0075494E7
+1556732880000 2.0046619E7
+1556732940000 2.2873048E7
+1556733000000 2.0332452E7
+1556733060000 2.2669098E7
+1556733120000 4.82090408E8
+1556733180000 2.588263E7
+1556733240000 2.5987032E7
+1556733300000 2.7023622E7
+1556733360000 5.0440297E7
diff --git a/memory/src/test/scala/filodb.memory/data/ChunkMapTest.scala b/memory/src/test/scala/filodb.memory/data/ChunkMapTest.scala
index 8fa43f08ca..59226447fa 100644
--- a/memory/src/test/scala/filodb.memory/data/ChunkMapTest.scala
+++ b/memory/src/test/scala/filodb.memory/data/ChunkMapTest.scala
@@ -8,6 +8,7 @@ import debox.Buffer
import org.scalatest.concurrent.ScalaFutures
import filodb.memory.BinaryRegion.NativePointer
+import filodb.memory.NativeMemoryManager
import filodb.memory.format.UnsafeUtils
import filodb.memory.format.vectors.NativeVectorTest
@@ -754,4 +755,26 @@ class ChunkMapTest extends NativeVectorTest with ScalaFutures {
map.chunkmapFree()
}
+
+ it("should evict when out of memory") {
+ // Very little memory is available.
+ val mm = new NativeMemoryManager(200)
+
+ // Tiny initial capacity.
+ val map = new ChunkMap(mm, 1)
+
+ val elems = makeElems((0 to 19).map(_.toLong))
+ elems.foreach { elem =>
+ map.chunkmapDoPut(elem, 9) // Evict anything less than or equal to 9.
+ }
+
+ // Not 20 due to evictions.
+ map.chunkmapSize shouldEqual 16
+
+ for (i <- 4 to 19) {
+ map.chunkmapContains(i) shouldEqual true
+ }
+
+ mm.freeAll()
+ }
}
diff --git a/memory/src/test/scala/filodb.memory/format/NibblePackTest.scala b/memory/src/test/scala/filodb.memory/format/NibblePackTest.scala
index 7065b6f9be..09224cb672 100644
--- a/memory/src/test/scala/filodb.memory/format/NibblePackTest.scala
+++ b/memory/src/test/scala/filodb.memory/format/NibblePackTest.scala
@@ -93,6 +93,19 @@ class NibblePackTest extends FunSpec with Matchers with PropertyChecks {
sink2.outArray shouldEqual inputs2
}
+ it("should pack and unpack double values") {
+ val inputs = Array(0.0, 2.5, 5.0, 7.5, 8, 13.2, 18.9, 89, 101.1, 102.3)
+ val buf = new ExpandableArrayBuffer()
+ val bytesWritten = NibblePack.packDoubles(inputs, buf, 0)
+
+ val bufSlice = new UnsafeBuffer(buf, 0, bytesWritten)
+ val out = new Array[Double](inputs.size)
+ val res = NibblePack.unpackDoubleXOR(bufSlice, out)
+
+ res shouldEqual NibblePack.Ok
+ out shouldEqual inputs
+ }
+
import org.scalacheck._
// Generate a list of increasing integers, every time bound it slightly differently
@@ -118,4 +131,21 @@ class NibblePackTest extends FunSpec with Matchers with PropertyChecks {
sink.outArray shouldEqual inputs
}
}
+
+ def increasingDoubleList: Gen[Seq[Double]] = increasingLongList.map(_.map(_.toDouble)).filter(_.length > 0)
+
+ it("should pack and unpack random list of increasing Doubles via XOR") {
+ val buf = new ExpandableArrayBuffer()
+ forAll(increasingDoubleList) { doubles =>
+ val inputs = doubles.toArray
+ val bytesWritten = NibblePack.packDoubles(inputs, buf, 0)
+
+ val bufSlice = new UnsafeBuffer(buf, 0, bytesWritten)
+ val out = new Array[Double](inputs.size)
+ val res = NibblePack.unpackDoubleXOR(bufSlice, out)
+
+ res shouldEqual NibblePack.Ok
+ out shouldEqual inputs
+ }
+ }
}
\ No newline at end of file
diff --git a/memory/src/test/scala/filodb.memory/format/vectors/DoubleVectorTest.scala b/memory/src/test/scala/filodb.memory/format/vectors/DoubleVectorTest.scala
index 092d165b54..be64415a76 100644
--- a/memory/src/test/scala/filodb.memory/format/vectors/DoubleVectorTest.scala
+++ b/memory/src/test/scala/filodb.memory/format/vectors/DoubleVectorTest.scala
@@ -2,7 +2,7 @@ package filodb.memory.format.vectors
import debox.Buffer
-import filodb.memory.format.{BinaryVector, GrowableVector}
+import filodb.memory.format.{ArrayStringRowReader, BinaryVector, GrowableVector}
class DoubleVectorTest extends NativeVectorTest {
describe("DoubleMaskedAppendableVector") {
@@ -77,6 +77,18 @@ class DoubleVectorTest extends NativeVectorTest {
BinaryVector.totalBytes(optimized) shouldEqual 24 // Const DeltaDeltaVector (since this is linearly increasing)
}
+ it("should encode some edge cases correctly to DDV") {
+ val orig = Seq(55.0, 60.0) ++ Seq.fill(10)(60.0)
+ val appender = DoubleVector.appendingVectorNoNA(memFactory, 100)
+ orig.foreach(appender.addData)
+ appender.length shouldEqual orig.length
+
+ val optimized = appender.optimize(memFactory)
+ DoubleVector(optimized).length(optimized) shouldEqual orig.length
+ DoubleVector(optimized).toBuffer(optimized).toList shouldEqual orig
+ BinaryVector.totalBytes(optimized) should be > (24) // Not const DDV!
+ }
+
it("should be able to optimize off-heap No NA integral vector to DeltaDeltaVector") {
val builder = DoubleVector.appendingVectorNoNA(memFactory, 100)
// Use higher numbers to verify they can be encoded efficiently too
@@ -152,4 +164,32 @@ class DoubleVectorTest extends NativeVectorTest {
cb.copyToBuffer.toList shouldEqual orig2
}
}
+
+ describe("bugs") {
+ it("should enumerate same samples regardless of where start enumeration from") {
+ val data = scala.io.Source.fromURL(getClass.getResource("/timeseries_bug1.txt"))
+ .getLines.map(_.split(' '))
+ .map(ArrayStringRowReader).toSeq
+ val origValues = data.map(_.getDouble(1))
+ val timestampAppender = LongBinaryVector.appendingVectorNoNA(memFactory, data.length)
+ val valuesAppender = DoubleVector.appendingVectorNoNA(memFactory, data.length)
+ data.foreach { reader =>
+ timestampAppender.addData(reader.getLong(0))
+ valuesAppender.addData(reader.getDouble(1))
+ }
+
+ val tsEncoded = timestampAppender.optimize(memFactory)
+ val valuesEncoded = valuesAppender.optimize(memFactory)
+ val tsReader = LongBinaryVector(tsEncoded)
+ val dReader = DoubleVector(valuesEncoded)
+
+ val samples = new collection.mutable.ArrayBuffer[Double]
+ for { i <- 0 until timestampAppender.length by 10 } {
+ samples.clear()
+ val iter = dReader.iterate(valuesEncoded, i)
+ (i until timestampAppender.length).foreach(_ => samples.append(iter.next))
+ samples shouldEqual origValues.drop(i)
+ }
+ }
+ }
}
\ No newline at end of file
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 140ce72e36..bf4ba949bd 100644
--- a/memory/src/test/scala/filodb.memory/format/vectors/HistogramTest.scala
+++ b/memory/src/test/scala/filodb.memory/format/vectors/HistogramTest.scala
@@ -4,6 +4,7 @@ import org.agrona.ExpandableArrayBuffer
object HistogramTest {
val bucketScheme = GeometricBuckets(1.0, 2.0, 8)
+ val customScheme = CustomBuckets(Array(0.25, 0.5, 1.0, 2.5, 5.0, 10, Double.PositiveInfinity))
val rawHistBuckets = Seq(
Array[Double](10, 15, 17, 20, 25, 34, 76, 82),
Array[Double](6, 16, 26, 26, 36, 38, 56, 59),
@@ -15,6 +16,10 @@ object HistogramTest {
MutableHistogram(bucketScheme, buckets)
}
+ val customHistograms = rawHistBuckets.map { buckets =>
+ LongHistogram(customScheme, buckets.take(customScheme.numBuckets).map(_.toLong))
+ }
+
val quantile50Result = Seq(37.333333333333336, 10.8, 8.666666666666666, 28.75)
}
@@ -23,6 +28,8 @@ import BinaryHistogram._
class HistogramTest extends NativeVectorTest {
val writeBuf = new ExpandableArrayBuffer()
+ import HistogramTest._
+
describe("HistogramBuckets") {
it("can list out bucket definition LE values properly for Geometric and Geometric_1") {
val buckets1 = GeometricBuckets(5.0, 3.0, 4)
@@ -40,11 +47,12 @@ class HistogramTest extends NativeVectorTest {
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
+
+ customScheme.serialize(writeBuf, 0) shouldEqual 26
+ HistogramBuckets(writeBuf, HistFormat_Custom_Delta) shouldEqual customScheme
}
}
- import HistogramTest._
-
describe("Histogram") {
it("should calculate quantile correctly") {
mutableHistograms.zip(quantile50Result).foreach { case (h, res) =>
@@ -68,6 +76,16 @@ class HistogramTest extends NativeVectorTest {
}
}
+ it("should serialize to and from BinaryHistograms with custom buckets") {
+ val longHist = LongHistogram(customScheme, Array[Long](10, 15, 17, 20, 25, 34, 76))
+ val buf = new ExpandableArrayBuffer()
+ longHist.serialize(Some(buf))
+
+ val hist2 = BinaryHistogram.BinHistogram(buf).toHistogram
+ hist2 shouldEqual longHist
+ hist2.hashCode shouldEqual longHist.hashCode
+ }
+
it("should serialize to and from an empty Histogram") {
val binEmptyHist = BinaryHistogram.BinHistogram(Histogram.empty.serialize())
binEmptyHist.numBuckets shouldEqual 0
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 9133079745..9e6d104aef 100644
--- a/memory/src/test/scala/filodb.memory/format/vectors/HistogramVectorTest.scala
+++ b/memory/src/test/scala/filodb.memory/format/vectors/HistogramVectorTest.scala
@@ -50,6 +50,23 @@ class HistogramVectorTest extends NativeVectorTest {
.zipWithIndex.foreach { case (h, i) => verifyHistogram(h, i) }
}
+ it("should accept LongHistograms with custom scheme and query them back") {
+ val appender = HistogramVector.appending(memFactory, 1024)
+ customHistograms.foreach { custHist =>
+ custHist.serialize(Some(buffer))
+ appender.addData(buffer) shouldEqual Ack
+ }
+
+ appender.length shouldEqual customHistograms.length
+
+ val reader = appender.reader.asInstanceOf[RowHistogramReader]
+ reader.length shouldEqual customHistograms.length
+
+ (0 until customHistograms.length).foreach { i =>
+ reader(i) shouldEqual customHistograms(i)
+ }
+ }
+
it("should optimize histograms and be able to query optimized vectors") {
val appender = HistogramVector.appending(memFactory, 1024)
rawLongBuckets.foreach { rawBuckets =>
diff --git a/prometheus/src/main/scala/filodb/prometheus/ast/Expressions.scala b/prometheus/src/main/scala/filodb/prometheus/ast/Expressions.scala
index 6199fc6891..965e810d13 100644
--- a/prometheus/src/main/scala/filodb/prometheus/ast/Expressions.scala
+++ b/prometheus/src/main/scala/filodb/prometheus/ast/Expressions.scala
@@ -47,7 +47,14 @@ trait Expressions extends Aggregates with Functions {
val seriesPlanLhs = series.toPeriodicSeriesPlan(timeParams)
val seriesPlanRhs = rhs.asInstanceOf[PeriodicSeries].toPeriodicSeriesPlan(timeParams)
val cardinality = vectorMatch.map(_.cardinality.cardinality).getOrElse(Cardinality.OneToOne)
- BinaryJoin(seriesPlanLhs, operator.getPlanOperator, cardinality, seriesPlanRhs)
+
+ val matcher = vectorMatch.flatMap(_.matching)
+ val onLabels = matcher.filter(_.isInstanceOf[On]).map(_.labels)
+ val ignoringLabels = matcher.filter(_.isInstanceOf[Ignoring]).map(_.labels)
+
+ BinaryJoin(seriesPlanLhs, operator.getPlanOperator, cardinality, seriesPlanRhs,
+ onLabels.getOrElse(Nil), ignoringLabels.getOrElse(Nil))
+
}
}
}
diff --git a/prometheus/src/main/scala/filodb/prometheus/ast/Functions.scala b/prometheus/src/main/scala/filodb/prometheus/ast/Functions.scala
index 5270bd1228..0db0ae778e 100644
--- a/prometheus/src/main/scala/filodb/prometheus/ast/Functions.scala
+++ b/prometheus/src/main/scala/filodb/prometheus/ast/Functions.scala
@@ -17,13 +17,11 @@ trait Functions extends Base with Operators with Vectors {
def toPeriodicSeriesPlan(timeParams: TimeRangeParams): PeriodicSeriesPlan = {
val seriesParam = allParams.filter(_.isInstanceOf[Series]).head.asInstanceOf[Series]
- val otherParams = allParams.filter(!_.equals(seriesParam)).map(_ match {
+ val otherParams = allParams.filter(!_.equals(seriesParam)).map {
case num: ScalarExpression => num.toScalar
- case s: InstantExpression => s.metricName.replaceAll("^\"|\"$", "")
- case _ =>
- throw new IllegalArgumentException("Parameters can be a string or number")
+ case s: InstantExpression => s.realMetricName.replaceAll("^\"|\"$", "")
+ case _ => throw new IllegalArgumentException("Parameters can be a string or number")
}
- )
val instantFunctionIdOpt = InstantFunctionId.withNameInsensitiveOption(name)
val filoFunctionIdOpt = FiloFunctionId.withNameInsensitiveOption(name)
diff --git a/prometheus/src/main/scala/filodb/prometheus/ast/Vectors.scala b/prometheus/src/main/scala/filodb/prometheus/ast/Vectors.scala
index 374c43aa44..ff77af19f2 100644
--- a/prometheus/src/main/scala/filodb/prometheus/ast/Vectors.scala
+++ b/prometheus/src/main/scala/filodb/prometheus/ast/Vectors.scala
@@ -84,13 +84,30 @@ trait Vectors extends Scalars with TimeUnits with Base {
}
sealed trait Vector extends Expression {
+
+ def metricName: Option[String]
+ def labelSelection: Seq[LabelMatch]
+
+ def realMetricName: String = {
+ val nameLabelValue = labelSelection.find(_.label == PromMetricLabel).map(_.value)
+ if (nameLabelValue.nonEmpty && metricName.nonEmpty) {
+ throw new IllegalArgumentException("Metric name should not be set twice")
+ }
+ metricName.orElse(nameLabelValue)
+ .getOrElse(throw new IllegalArgumentException("Metric name is not present"))
+ }
+
protected def labelMatchesToFilters(labels: Seq[LabelMatch]) =
labels.map { labelMatch =>
- labelMatch.labelMatchOp match {
- case EqualMatch => ColumnFilter(labelMatch.label, query.Filter.Equals(labelMatch.value))
- case NotRegexMatch => ColumnFilter(labelMatch.label, query.Filter.NotEqualsRegex(labelMatch.value))
- case RegexMatch => ColumnFilter(labelMatch.label, query.Filter.EqualsRegex(labelMatch.value))
- case NotEqual(false) => ColumnFilter(labelMatch.label, query.Filter.NotEquals(labelMatch.value))
+ val labelValue = labelMatch.value.replace("\\\\", "\\")
+ .replace("\\\"", "\"")
+ .replace("\\n", "\n")
+ .replace("\\t", "\t")
+ labelMatch.labelMatchOp match {
+ case EqualMatch => ColumnFilter(labelMatch.label, query.Filter.Equals(labelValue))
+ case NotRegexMatch => ColumnFilter(labelMatch.label, query.Filter.NotEqualsRegex(labelValue))
+ case RegexMatch => ColumnFilter(labelMatch.label, query.Filter.EqualsRegex(labelValue))
+ case NotEqual(false) => ColumnFilter(labelMatch.label, query.Filter.NotEquals(labelValue))
case other: Any => throw new IllegalArgumentException(s"Unknown match operator $other")
}
// Remove the column selector as that is not a real time series filter
@@ -114,21 +131,16 @@ trait Vectors extends Scalars with TimeUnits with Base {
* appending a set of labels to match in curly braces ({}).
*/
- case class InstantExpression(metricName: String,
- labelSelection: Seq[LabelMatch],
+ case class InstantExpression(override val metricName: Option[String],
+ override val labelSelection: Seq[LabelMatch],
offset: Option[Duration]) extends Vector with PeriodicSeries {
- val staleDataLookbackSeconds = 5 * 60 // 5 minutes
-
- private val nameLabels = labelSelection.filter(_.label == PromMetricLabel)
- if (nameLabels.nonEmpty && !nameLabels.head.label.equals(metricName)) {
- throw new IllegalArgumentException("Metric name should not be set twice")
- }
+ val staleDataLookbackSeconds = 5 * 60 // 5 minutes
private[prometheus] val columnFilters = labelMatchesToFilters(labelSelection)
private[prometheus] val columns = labelMatchesToColumnName(labelSelection)
- private[prometheus] val nameFilter = ColumnFilter(PromMetricLabel, query.Filter.Equals(metricName))
- def getColFilters: Seq[ColumnFilter] = columnFilters :+ nameFilter
+ private[prometheus] val nameFilter = ColumnFilter(PromMetricLabel, query.Filter.Equals(realMetricName))
+ def getColFilters: Seq[ColumnFilter] = if (metricName.isDefined) columnFilters :+ nameFilter else columnFilters
def toPeriodicSeriesPlan(timeParams: TimeRangeParams): PeriodicSeriesPlan = {
@@ -140,12 +152,9 @@ trait Vectors extends Scalars with TimeUnits with Base {
timeParams.start * 1000, timeParams.step * 1000, timeParams.end * 1000
)
}
- }
- case class MetadataExpression(instantExpression: InstantExpression) extends Vector with Metadata {
-
- override def toMetadataQueryPlan(timeParams: TimeRangeParams): MetadataQueryPlan = {
- SeriesKeysByFilters(instantExpression.getColFilters, timeParams.start * 1000, timeParams.end * 1000)
+ def toMetadataPlan(timeParams: TimeRangeParams): SeriesKeysByFilters = {
+ SeriesKeysByFilters(getColFilters, timeParams.start * 1000, timeParams.end * 1000)
}
}
@@ -156,21 +165,16 @@ trait Vectors extends Scalars with TimeUnits with Base {
* at the end of a vector selector to specify how far back in time values
* should be fetched for each resulting range vector element.
*/
- case class RangeExpression(metricName: String,
- labelSelection: Seq[LabelMatch],
+ case class RangeExpression(override val metricName: Option[String],
+ override val labelSelection: Seq[LabelMatch],
window: Duration,
offset: Option[Duration]) extends Vector with SimpleSeries {
- private val nameLabels = labelSelection.filter(_.label == PromMetricLabel)
-
- if (nameLabels.nonEmpty && !nameLabels.head.label.equals(metricName)) {
- throw new IllegalArgumentException("Metric name should not be set twice")
- }
private[prometheus] val columnFilters = labelMatchesToFilters(labelSelection)
private[prometheus] val columns = labelMatchesToColumnName(labelSelection)
- private[prometheus] val nameFilter = ColumnFilter(PromMetricLabel, query.Filter.Equals(metricName))
+ private[prometheus] val nameFilter = ColumnFilter(PromMetricLabel, query.Filter.Equals(realMetricName))
- val allFilters: Seq[ColumnFilter] = columnFilters :+ nameFilter
+ val allFilters: Seq[ColumnFilter] = if (metricName.isDefined) columnFilters :+ nameFilter else columnFilters
def toRawSeriesPlan(timeParams: TimeRangeParams, isRoot: Boolean): RawSeriesPlan = {
if (isRoot && timeParams.start != timeParams.end) {
diff --git a/prometheus/src/main/scala/filodb/prometheus/parse/Parser.scala b/prometheus/src/main/scala/filodb/prometheus/parse/Parser.scala
index 1bfaff3537..375c1b70f0 100644
--- a/prometheus/src/main/scala/filodb/prometheus/parse/Parser.scala
+++ b/prometheus/src/main/scala/filodb/prometheus/parse/Parser.scala
@@ -194,22 +194,36 @@ trait Join extends Numeric {
////////////////////// SELECTORS ///////////////////////////////////////////
trait Selector extends Operator with Unit with BaseParser {
protected lazy val simpleSeries: PackratParser[InstantExpression] =
- "([\"'])(?:\\\\\\1|.)*?\\1".r ^^ { str => InstantExpression(str, Seq.empty, None) }
+ "([\"'])(?:\\\\\\1|.)*?\\1".r ^^ { str => InstantExpression(Some(str), Seq.empty, None) }
lazy val instantVectorSelector: PackratParser[InstantExpression]
= labelIdentifier ~ labelSelection.? ~ offset.? ^^ {
case metricName ~ ls ~ opt =>
- InstantExpression(metricName.str, ls.getOrElse(Seq.empty), opt.map(_.duration))
+ InstantExpression(Some(metricName.str), ls.getOrElse(Seq.empty), opt.map(_.duration))
+ }
+
+ lazy val instantVectorSelector2: PackratParser[InstantExpression]
+ = labelSelection ~ offset.? ^^ {
+ case ls ~ opt =>
+ InstantExpression(None, ls, opt.map(_.duration))
}
lazy val rangeVectorSelector: PackratParser[RangeExpression] =
labelIdentifier ~ labelSelection.? ~ "[" ~ duration ~ "]" ~ offset.? ^^ {
case metricName ~ ls ~ leftBracket ~ td ~ rightBracket ~ opt =>
- RangeExpression(metricName.str, ls.getOrElse(Seq.empty), td, opt.map(_.duration))
+ RangeExpression(Some(metricName.str), ls.getOrElse(Seq.empty), td, opt.map(_.duration))
}
- lazy val vector: PackratParser[Vector] = rangeVectorSelector | instantVectorSelector
+ lazy val rangeVectorSelector2: PackratParser[RangeExpression] =
+ labelSelection ~ "[" ~ duration ~ "]" ~ offset.? ^^ {
+ case ls ~ leftBracket ~ td ~ rightBracket ~ opt =>
+ RangeExpression(None, ls, td, opt.map(_.duration))
+ }
+
+ lazy val vector: PackratParser[Vector] =
+ rangeVectorSelector2 | rangeVectorSelector | instantVectorSelector2 | instantVectorSelector
+
}
////////////////////// END SELECTORS ///////////////////////////////////////////
@@ -304,7 +318,7 @@ object Parser extends Expression {
*/
override lazy val skipWhitespace: Boolean = true
- override val whiteSpace = "[ \t\r\f]+".r
+ override val whiteSpace = "[ \t\r\f\n]+".r
val FiveMinutes = Duration(5, Minute).millis
@@ -319,7 +333,7 @@ object Parser extends Expression {
def metadataQueryToLogicalPlan(query: String, timeParams: TimeRangeParams): LogicalPlan = {
val expression = parseQuery(query)
expression match {
- case p: InstantExpression => MetadataExpression(p).toMetadataQueryPlan(timeParams)
+ case p: InstantExpression => p.toMetadataPlan(timeParams)
case _ => throw new UnsupportedOperationException()
}
}
diff --git a/prometheus/src/main/scala/filodb/prometheus/query/PrometheusModel.scala b/prometheus/src/main/scala/filodb/prometheus/query/PrometheusModel.scala
index 3953317a71..123e5777e7 100644
--- a/prometheus/src/main/scala/filodb/prometheus/query/PrometheusModel.scala
+++ b/prometheus/src/main/scala/filodb/prometheus/query/PrometheusModel.scala
@@ -4,6 +4,7 @@ import remote.RemoteStorage._
import filodb.core.query.{ColumnFilter, Filter, SerializableRangeVector}
import filodb.query.{IntervalSelector, LogicalPlan, QueryResultType, RawSeries}
+import filodb.query.exec.ExecPlan
object PrometheusModel {
@@ -15,6 +16,8 @@ object PrometheusModel {
final case class SuccessResponse(data: Data, status: String = "success") extends PromQueryResponse
+ final case class ExplainPlanResponse(debugInfo: Seq[String], status: String = "success") extends PromQueryResponse
+
final case class Data(resultType: String, result: Seq[Result])
final case class Result(metric: Map[String, String], values: Seq[Sampl])
@@ -77,7 +80,12 @@ object PrometheusModel {
}
def toPromSuccessResponse(qr: filodb.query.QueryResult, verbose: Boolean): SuccessResponse = {
- SuccessResponse(Data(toPromResultType(qr.resultType), qr.result.map(toPromResult(_, verbose))))
+ SuccessResponse(Data(toPromResultType(qr.resultType),
+ qr.result.map(toPromResult(_, verbose)).filter(_.values.nonEmpty)))
+ }
+
+ def toPromExplainPlanResponse(ex: ExecPlan): ExplainPlanResponse = {
+ ExplainPlanResponse(ex.getPlan())
}
def toPromResultType(r: QueryResultType): String = {
diff --git a/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala b/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala
index ad3ecfb7f6..1aeafe88a1 100644
--- a/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala
+++ b/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala
@@ -4,7 +4,6 @@ import org.scalatest.{FunSpec, Matchers}
import filodb.prometheus.ast.TimeStepParams
-
//noinspection ScalaStyle
// scalastyle:off
class ParserSpec extends FunSpec with Matchers {
@@ -265,7 +264,7 @@ class ParserSpec extends FunSpec with Matchers {
"sum(http_requests_total) without (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(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(build_version))),List()),1524855988000,1000000,1524855988000),List(\"version\"),List(),List())",
+ "Aggregate(CountValues,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(build_version))),List()),1524855988000,1000000,1524855988000),List(Some(\"version\")),List(),List())",
"label_replace(up{job=\"api-server\",service=\"a:c\"}, \"foo\", \"$1\", \"service\", \"(.*):.*\")" ->
"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)" ->
@@ -276,8 +275,12 @@ class ParserSpec extends FunSpec with Matchers {
"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(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" ->
+ "method_code:http_errors:rate5m{code=\"500\"} / method:http_requests:rate5m" ->
"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())",
+ "method_code:http_errors:rate5m{code=\"500\"} / ignoring(code) method:http_requests:rate5m" ->
+ "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(code))",
+ "method_code:http_errors:rate5m{code=\"500\"} / on(method) method:http_requests:rate5m" ->
+ "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(method),List())",
"histogram_quantile(0.9, rate(http_request_duration_seconds_bucket[10m]))" ->
"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))" ->
@@ -312,8 +315,10 @@ class ParserSpec extends FunSpec with Matchers {
"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(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(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())",
+ "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(code))",
+
"increase(http_requests_total{job=\"api-server\"}[5m])" ->
"PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(job,Equals(api-server)), ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000,300000,Increase,List())",
"sum(http_requests_total{method=\"GET\"} offset 5m)" ->
@@ -321,7 +326,17 @@ class ParserSpec extends FunSpec with Matchers {
"absent(nonexistent{job=\"myjob\",instance=~\".*\"})" ->
"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(1524855688000,1524855988000),List(ColumnFilter(job,Equals(myjob)), ColumnFilter(__name__,Equals(nonexistent))),List()),1524855988000,1000000,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())",
+ """{__name__="foo\\\"\n\t",job="myjob"}[5m]""" ->
+ "RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(foo\\\"\n\t)), ColumnFilter(job,Equals(myjob))),List())",
+ "{__name__=\"foo\",job=\"myjob\"}" ->
+ "PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(foo)), ColumnFilter(job,Equals(myjob))),List()),1524855988000,1000000,1524855988000)",
+ "{__name__=\"foo\",job=\"myjob\"}[5m]" ->
+ "RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(foo)), ColumnFilter(job,Equals(myjob))),List())",
+ "sum({__name__=\"foo\",job=\"myjob\"})" ->
+ "Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(foo)), ColumnFilter(job,Equals(myjob))),List()),1524855988000,1000000,1524855988000),List(),List(),List())",
+ "sum(http_requests_total) \n \n / \n\n sum(http_requests_total)" ->
+ "BinaryJoin(Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000),List(),List(),List()),DIV,OneToOne,Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000),List(),List(),List()),List(),List())"
)
val qts: Long = 1524855988L
@@ -331,7 +346,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/QueryConfig.scala b/query/src/main/scala/filodb/query/QueryConfig.scala
index dafc80e685..807a46f141 100644
--- a/query/src/main/scala/filodb/query/QueryConfig.scala
+++ b/query/src/main/scala/filodb/query/QueryConfig.scala
@@ -13,4 +13,5 @@ class QueryConfig(queryConfig: Config) {
lazy val askTimeout = queryConfig.as[FiniteDuration]("ask-timeout")
lazy val staleSampleAfterMs = queryConfig.getDuration("stale-sample-after").toMillis
lazy val minStepMs = queryConfig.getDuration("min-step").toMillis
+ lazy val fastReduceMaxWindows = queryConfig.getInt("fastreduce-max-windows")
}
diff --git a/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala b/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala
index e074600949..2ad3c5eb12 100644
--- a/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala
+++ b/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala
@@ -7,6 +7,7 @@ import scala.collection.mutable
import com.tdunning.math.stats.{ArrayDigest, TDigest}
import com.typesafe.scalalogging.StrictLogging
import monix.reactive.Observable
+import scalaxy.loops._
import filodb.core.binaryrecord2.RecordBuilder
import filodb.core.metadata.Column.ColumnType
@@ -65,6 +66,7 @@ final case class AggregateMapReduce(aggrOp: AggregationOperator,
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))
@@ -72,7 +74,17 @@ final case class AggregateMapReduce(aggrOp: AggregationOperator,
else Map.empty
CustomRangeVectorKey(groupBy)
}
- RangeVectorAggregator.mapReduce(aggregator, skipMapPhase = false, source, grouping)
+
+ // IF no grouping is done AND prev transformer is Periodic (has fixed length), use optimal path
+ if (without.isEmpty && by.isEmpty && sourceSchema.fixedVectorLen.isDefined) {
+ sourceSchema.fixedVectorLen.filter(_ <= queryConfig.fastReduceMaxWindows).map { numWindows =>
+ RangeVectorAggregator.fastReduce(aggregator, false, source, numWindows)
+ }.getOrElse {
+ RangeVectorAggregator.mapReduce(aggregator, skipMapPhase = false, source, grouping)
+ }
+ } else {
+ RangeVectorAggregator.mapReduce(aggregator, skipMapPhase = false, source, grouping)
+ }
}
override def schema(dataset: Dataset, source: ResultSchema): ResultSchema = {
@@ -152,12 +164,11 @@ object RangeVectorAggregator extends StrictLogging {
val mapInto = rowAgg.newRowToMapInto
rvs.groupBy(grouping).mapValues { rvs =>
new Iterator[rowAgg.AggHolderType] {
- val rowIterators = rvs.map(_.rows)
- val rvKeys = rvs.map(_.key)
- def hasNext: Boolean = rowIterators.forall(_.hasNext)
+ val itsAndKeys = rvs.map { rv => (rv.rows, rv.key) }
+ def hasNext: Boolean = itsAndKeys.forall(_._1.hasNext)
def next(): rowAgg.AggHolderType = {
acc.resetToZero()
- rowIterators.zip(rvKeys).foreach { case (rowIter, rvk) =>
+ itsAndKeys.foreach { case (rowIter, rvk) =>
val mapped = if (skipMapPhase) rowIter.next() else rowAgg.map(rvk, rowIter.next(), mapInto)
acc = if (skipMapPhase) rowAgg.reduceAggregate(acc, mapped) else rowAgg.reduceMappedRow(acc, mapped)
}
@@ -166,6 +177,54 @@ object RangeVectorAggregator extends StrictLogging {
}
}
}
+
+ /**
+ * A fast reduce method intended specifically for the case when no grouping needs to be done AND
+ * the previous transformer is a PeriodicSampleMapper with fixed output lengths.
+ * It's much faster than mapReduce() since it iterates through each vector first and then from vector to vector.
+ * Time wise first iteration also uses less memory for high-cardinality use cases and reduces the
+ * time window of holding chunk map locks to each time series, instead of the entire query.
+ */
+ def fastReduce(rowAgg: RowAggregator,
+ skipMapPhase: Boolean,
+ source: Observable[RangeVector],
+ outputLen: Int): Observable[RangeVector] = {
+ // Can't use an Array here because rowAgg.AggHolderType does not have a ClassTag
+ val accs = collection.mutable.ArrayBuffer.fill(outputLen)(rowAgg.zero)
+ var count = 0
+
+ // FoldLeft means we create the source PeriodicMapper etc and process immediately. We can release locks right away
+ // NOTE: ChunkedWindowIterator automatically releases locks after last window. So it should all just work. :)
+ val aggObs = if (skipMapPhase) {
+ source.foldLeftF(accs) { case (_, rv) =>
+ count += 1
+ val rowIter = rv.rows
+ for { i <- 0 until outputLen optimized } {
+ accs(i) = rowAgg.reduceAggregate(accs(i), rowIter.next)
+ }
+ accs
+ }
+ } else {
+ val mapIntos = Array.fill(outputLen)(rowAgg.newRowToMapInto)
+ source.foldLeftF(accs) { case (_, rv) =>
+ count += 1
+ val rowIter = rv.rows
+ for { i <- 0 until outputLen optimized } {
+ val mapped = rowAgg.map(rv.key, rowIter.next, mapIntos(i))
+ accs(i) = rowAgg.reduceMappedRow(accs(i), mapped)
+ }
+ accs
+ }
+ }
+
+ aggObs.flatMap { _ =>
+ if (count > 0) {
+ Observable.now(new IteratorBackedRangeVector(CustomRangeVectorKey.empty, accs.toIterator.map(_.toRowReader)))
+ } else {
+ Observable.empty
+ }
+ }
+ }
}
trait AggregateHolder {
@@ -199,6 +258,7 @@ trait RowAggregator {
* Note that one object is used per aggregation. The returned object
* is reused to aggregate each row-key of each RangeVector by resetting
* before aggregation of next row-key.
+ * Should return a new AggHolder.
*/
def zero: AggHolderType
@@ -520,6 +580,12 @@ class TopBottomKRowAggregator(k: Int, bottomK: Boolean) extends RowAggregator {
row.setDouble(i + 1, el.value)
i += 2
}
+ // Reset remaining values of row to overwrite previous row value
+ while (i < numRowReaderColumns) {
+ row.setString(i, CustomRangeVectorKey.emptyAsZcUtf8)
+ row.setDouble(i + 1, if (bottomK) Double.MaxValue else Double.MinValue)
+ i += 2
+ }
row
}
def resetToZero(): Unit = { heap.clear() }
@@ -563,7 +629,7 @@ class TopBottomKRowAggregator(k: Int, bottomK: Boolean) extends RowAggregator {
// Important TODO / TechDebt: We need to replace Iterators with cursors to better control
// the chunk iteration, lock acquisition and release. This is much needed for safe memory access.
try {
- ChunkMap.validateNoSharedLocks()
+ ChunkMap.validateNoSharedLocks(s"TopkQuery-$k-$bottomK")
// We limit the results wherever it is materialized first. So it is done here.
aggRangeVector.rows.take(limit).foreach { row =>
var i = 1
diff --git a/query/src/main/scala/filodb/query/exec/ExecPlan.scala b/query/src/main/scala/filodb/query/exec/ExecPlan.scala
index 1d0eb59ac3..d6e2f16599 100644
--- a/query/src/main/scala/filodb/query/exec/ExecPlan.scala
+++ b/query/src/main/scala/filodb/query/exec/ExecPlan.scala
@@ -124,7 +124,7 @@ trait ExecPlan extends QueryCommand {
srv
case rv: RangeVector =>
// materialize, and limit rows per RV
- val srv = SerializableRangeVector(rv, builder, recSchema)
+ val srv = SerializableRangeVector(rv, builder, recSchema, printTree(false))
numResultSamples += srv.numRows
// fail the query instead of limiting range vectors and returning incomplete/inaccurate results
if (numResultSamples > limit)
@@ -189,15 +189,27 @@ trait ExecPlan extends QueryCommand {
* @param useNewline pass false if the result string needs to be in one line
*/
final def printTree(useNewline: Boolean = true, level: Int = 0): String = {
- val transf = rangeVectorTransformers.reverse.zipWithIndex.map { case (t, i) =>
- s"${"-"*(level + i)}T~${t.getClass.getSimpleName}(${t.args})"
- }
+ val transf = printRangeVectorTransformersForLevel(level)
val nextLevel = rangeVectorTransformers.size + level
val curNode = s"${"-"*nextLevel}E~${getClass.getSimpleName}($args) on ${dispatcher}"
val childr = children.map(_.printTree(useNewline, nextLevel + 1))
((transf :+ curNode) ++ childr).mkString(if (useNewline) "\n" else " @@@ ")
}
+ final def getPlan(level: Int = 0): Seq[String] = {
+ val transf = printRangeVectorTransformersForLevel(level)
+ val nextLevel = rangeVectorTransformers.size + level
+ val curNode = s"${"-"*nextLevel}E~${getClass.getSimpleName}($args) on ${dispatcher}"
+ val childr : Seq[String]= children.flatMap(_.getPlan(nextLevel + 1))
+ ((transf :+ curNode) ++ childr)
+ }
+
+ protected def printRangeVectorTransformersForLevel(level: Int = 0) = {
+ rangeVectorTransformers.reverse.zipWithIndex.map { case (t, i) =>
+ s"${"-" * (level + i)}T~${t.getClass.getSimpleName}(${t.args})"
+ }
+ }
+
protected def rowIterAccumulator(srvsList: List[Seq[SerializableRangeVector]]): Iterator[RowReader] = {
new Iterator[RowReader] {
diff --git a/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala b/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala
index de0c9cae83..7153a14e12 100644
--- a/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala
+++ b/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala
@@ -88,13 +88,13 @@ final case class PeriodicSamplesMapper(start: Long,
// Transform source double or long to double schema
override def schema(dataset: Dataset, source: ResultSchema): ResultSchema =
source.copy(columns = source.columns.zipWithIndex.map {
- // Transform if its not a row key column
- case (ColumnInfo(name, ColumnType.LongColumn), i) if i >= source.numRowKeyColumns =>
- ColumnInfo(name, ColumnType.DoubleColumn)
- case (ColumnInfo(name, ColumnType.IntColumn), i) if i >= source.numRowKeyColumns =>
- ColumnInfo(name, ColumnType.DoubleColumn)
- case (c: ColumnInfo, _) => c
- })
+ // Transform if its not a row key column
+ case (ColumnInfo(name, ColumnType.LongColumn), i) if i >= source.numRowKeyColumns =>
+ ColumnInfo(name, ColumnType.DoubleColumn)
+ case (ColumnInfo(name, ColumnType.IntColumn), i) if i >= source.numRowKeyColumns =>
+ ColumnInfo(name, ColumnType.DoubleColumn)
+ case (c: ColumnInfo, _) => c
+ }, fixedVectorLen = Some(((end - start)/step).toInt + 1))
}
/**
diff --git a/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala b/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala
index 7e6d8cf684..133687cf7e 100644
--- a/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala
+++ b/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala
@@ -8,7 +8,7 @@ import filodb.core.query._
import filodb.memory.format.RowReader
import filodb.query.{BinaryOperator, InstantFunctionId, MiscellaneousFunctionId, QueryConfig}
import filodb.query.InstantFunctionId.HistogramQuantile
-import filodb.query.MiscellaneousFunctionId.LabelReplace
+import filodb.query.MiscellaneousFunctionId.{LabelJoin, LabelReplace}
import filodb.query.exec.binaryOp.BinaryOperatorFunction
import filodb.query.exec.rangefn._
@@ -180,6 +180,7 @@ final case class MiscellaneousFunctionMapper(function: MiscellaneousFunctionId,
val miscFunction: MiscellaneousFunction = {
function match {
case LabelReplace => LabelReplaceFunction(funcParams)
+ case LabelJoin => LabelJoinFunction(funcParams)
case _ => throw new UnsupportedOperationException(s"$function not supported.")
}
}
diff --git a/query/src/main/scala/filodb/query/exec/TransientRow.scala b/query/src/main/scala/filodb/query/exec/TransientRow.scala
index 0a8aeb9b7b..f5af0b4536 100644
--- a/query/src/main/scala/filodb/query/exec/TransientRow.scala
+++ b/query/src/main/scala/filodb/query/exec/TransientRow.scala
@@ -148,10 +148,10 @@ final class QuantileAggTransientRow() extends MutableRowReader {
def getFloat(columnNo: Int): Float = ???
def getString(columnNo: Int): String = ???
def getAny(columnNo: Int): Any = ???
- def getBlobBase(columnNo: Int): Any = if (columnNo == 1) blobBase
- else throw new IllegalArgumentException()
- def getBlobOffset(columnNo: Int): Long = if (columnNo == 1) blobOffset
- else throw new IllegalArgumentException()
+ def getBlobBase(columnNo: Int): Any = if (columnNo == 1) blobBase
+ else throw new IllegalArgumentException()
+ def getBlobOffset(columnNo: Int): Long = if (columnNo == 1) blobOffset
+ else throw new IllegalArgumentException()
def getBlobNumBytes(columnNo: Int): Int = if (columnNo == 1) blobLength
else throw new IllegalArgumentException()
diff --git a/query/src/main/scala/filodb/query/exec/rangefn/MiscellaneousFunction.scala b/query/src/main/scala/filodb/query/exec/rangefn/MiscellaneousFunction.scala
index 3856ca180b..ee0366a5d9 100644
--- a/query/src/main/scala/filodb/query/exec/rangefn/MiscellaneousFunction.scala
+++ b/query/src/main/scala/filodb/query/exec/rangefn/MiscellaneousFunction.scala
@@ -43,7 +43,7 @@ case class LabelReplaceFunction(funcParams: Seq[Any])
}
}
- def labelReplaceImpl(rangeVectorKey: RangeVectorKey, funcParams: Seq[Any]): RangeVectorKey = {
+ private def labelReplaceImpl(rangeVectorKey: RangeVectorKey, funcParams: Seq[Any]): RangeVectorKey = {
val value: ZeroCopyUTF8String = if (rangeVectorKey.labelValues.contains(ZeroCopyUTF8String(srcLabel))) {
rangeVectorKey.labelValues.get(ZeroCopyUTF8String(srcLabel)).get
@@ -78,3 +78,51 @@ case class LabelReplaceFunction(funcParams: Seq[Any])
return rangeVectorKey;
}
}
+
+case class LabelJoinFunction(funcParams: Seq[Any])
+ extends MiscellaneousFunction {
+
+ val labelIdentifier: String = "[a-zA-Z_][a-zA-Z0-9_:\\-\\.]*"
+
+ require(funcParams.size >= 2,
+ "expected at least 3 argument(s) in call to label_join")
+
+ val dstLabel: String = funcParams(0).asInstanceOf[String]
+ val separator: String = funcParams(1).asInstanceOf[String]
+
+ require(dstLabel.asInstanceOf[String].matches(labelIdentifier), "Invalid destination label name in label_join()")
+ var srcLabel =
+ funcParams.drop(2).map { x =>
+ require(x.asInstanceOf[String].matches(labelIdentifier),
+ "Invalid source label name in label_join()")
+ x.asInstanceOf[String]
+ }
+
+ override def execute(source: Observable[RangeVector]): Observable[RangeVector] = {
+ source.map { rv =>
+ val newLabel = labelJoinImpl(rv.key)
+ IteratorBackedRangeVector(newLabel, rv.rows)
+ }
+ }
+
+ private def labelJoinImpl(rangeVectorKey: RangeVectorKey): RangeVectorKey = {
+
+ val srcLabelValues = srcLabel.map(x=> rangeVectorKey.labelValues.get(ZeroCopyUTF8String(x)).
+ map(_.toString).getOrElse(""))
+
+ val labelJoinValue = srcLabelValues.mkString(separator)
+
+ if (labelJoinValue.length > 0) {
+ return CustomRangeVectorKey(rangeVectorKey.labelValues.
+ updated(ZeroCopyUTF8String(dstLabel), ZeroCopyUTF8String(labelJoinValue)), rangeVectorKey.sourceShards)
+ }
+ else {
+ // Drop label if new value is empty
+ return CustomRangeVectorKey(rangeVectorKey.labelValues -
+ ZeroCopyUTF8String(dstLabel), rangeVectorKey.sourceShards)
+ }
+
+ }
+}
+
+
diff --git a/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala b/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala
index f926d00118..e42e08d701 100644
--- a/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala
+++ b/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala
@@ -225,7 +225,7 @@ class AggrOverRangeVectorsSpec extends RawDataWindowingSpec with ScalaFutures {
val recSchema = SerializableRangeVector.toSchema(Seq(ColumnInfo("timestamp", ColumnType.LongColumn),
ColumnInfo("tdig", ColumnType.StringColumn)))
val builder = SerializableRangeVector.toBuilder(recSchema)
- val srv = SerializableRangeVector(result7(0), builder, recSchema)
+ val srv = SerializableRangeVector(result7(0), builder, recSchema, "Unit-Test")
val resultObs7b = RangeVectorAggregator.present(agg7, Observable.now(srv), 1000)
val finalResult = resultObs7b.toListL.runAsync.futureValue
@@ -336,6 +336,26 @@ class AggrOverRangeVectorsSpec extends RawDataWindowingSpec with ScalaFutures {
compareIter(result6b(0).rows.map(_.getDouble(1)), Seq(5.4d,5.6d).iterator)
}
+ it("topK should not have any trailing value ") {
+
+ // The value before NaN should not get carried over. Topk result for timestamp 1556744173L should have Double.NaN
+ val samples: Array[RangeVector] = Array(
+ toRv(Seq((1556744143L, 42d), (1556744158L, 42d),(1556744173L, Double.NaN)))
+ )
+
+ val agg6 = RowAggregator(AggregationOperator.TopK, Seq(5.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(0).key shouldEqual noKey
+ val result6b = resultObs6b.toListL.runAsync.futureValue
+ result6b.size shouldEqual 1
+ result6b(0).key shouldEqual ignoreKey
+ compareIter(result6b(0).rows.map(_.getDouble(1)), Seq(42d,42d).iterator)
+ }
+
import filodb.memory.format.{vectors => bv}
it("should sum histogram RVs") {
diff --git a/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala b/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala
index 3f458f50b3..e791fc90b8 100644
--- a/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala
+++ b/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala
@@ -53,6 +53,27 @@ class BinaryJoinExecSpec extends FunSpec with Matchers with ScalaFutures {
}
}
+ val samplesLhsGrouping: Array[RangeVector] = Array.tabulate(2) { i =>
+ new RangeVector {
+ val key: RangeVectorKey = CustomRangeVectorKey(
+ Map("__name__".utf8 -> s"someMetricLhs".utf8,
+ "tag1".utf8 -> s"tag1-$i".utf8,
+ "tag2".utf8 -> s"tag2-1".utf8,
+ "job".utf8 -> s"somejob".utf8))
+ val rows: Iterator[RowReader] = data(i).iterator
+ }
+ }
+
+ val samplesRhsGrouping: Array[RangeVector] = Array.tabulate(2) { i =>
+ new RangeVector {
+ val key: RangeVectorKey = CustomRangeVectorKey(
+ Map("__name__".utf8 -> s"someMetricRhs".utf8,
+ "tag1".utf8 -> s"tag1-$i".utf8,
+ "job".utf8 -> s"somejob".utf8))
+ val rows: Iterator[RowReader] = data(i).iterator
+ }
+ }
+
it("should join one-to-one without on or ignoring") {
val samplesRhs2 = scala.util.Random.shuffle(samplesRhs.toList) // they may come out of order
@@ -185,4 +206,66 @@ class BinaryJoinExecSpec extends FunSpec with Matchers with ScalaFutures {
e shouldBe a[BadQueryException]
}
}
+ it("should join one-to-one with ignoring") {
+
+ val execPlan = BinaryJoinExec("someID", dummyDispatcher,
+ Array(dummyPlan), // cannot be empty as some compose's rely on the schema
+ new Array[ExecPlan](1), // empty since we test compose, not execute or doExecute
+ BinaryOperator.ADD,
+ Cardinality.OneToOne,
+ Nil, Seq("tag2"))
+
+ val schema = Seq(ColumnInfo("timestamp", ColumnType.LongColumn),
+ ColumnInfo("value", ColumnType.DoubleColumn))
+
+ // scalastyle:off
+ val lhs = QueryResult("someId", null, samplesLhsGrouping.map(rv => SerializableRangeVector(rv, schema)))
+ // val lhs = QueryResult("someId", null, samplesLhs.filter(rv => rv.key.labelValues.get(ZeroCopyUTF8String("tag2")).get.equals("tag1-1")).map(rv => SerializableRangeVector(rv, schema)))
+ val rhs = QueryResult("someId", null, samplesRhsGrouping.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(dataset, Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), queryConfig)
+ .toListL.runAsync.futureValue
+
+ result.foreach { rv =>
+ rv.key.labelValues.contains("__name__".utf8) shouldEqual false
+ rv.key.labelValues.contains("tag1".utf8) shouldEqual true
+ rv.key.labelValues.contains("tag2".utf8) shouldEqual false
+ val i = rv.key.labelValues("tag1".utf8).asNewString.split("-")(1)
+ rv.rows.map(_.getDouble(1)).foreach(_ shouldEqual i.toDouble * 2)
+ }
+
+ result.map(_.key).toSet.size shouldEqual 2
+ }
+
+ it("should join one-to-one with on") {
+
+ val execPlan = BinaryJoinExec("someID", dummyDispatcher,
+ Array(dummyPlan), // cannot be empty as some compose's rely on the schema
+ new Array[ExecPlan](1), // empty since we test compose, not execute or doExecute
+ BinaryOperator.ADD,
+ Cardinality.OneToOne,
+ Seq("tag1", "job"), Nil)
+
+ val schema = Seq(ColumnInfo("timestamp", ColumnType.LongColumn),
+ ColumnInfo("value", ColumnType.DoubleColumn))
+
+ // scalastyle:off
+ val lhs = QueryResult("someId", null, samplesLhsGrouping.map(rv => SerializableRangeVector(rv, schema)))
+ val rhs = QueryResult("someId", null, samplesRhsGrouping.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(dataset, Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), queryConfig)
+ .toListL.runAsync.futureValue
+
+ result.foreach { rv =>
+ rv.key.labelValues.contains("__name__".utf8) shouldEqual false
+ rv.key.labelValues.contains("tag1".utf8) shouldEqual true
+ rv.key.labelValues.contains("tag2".utf8) shouldEqual false
+ val i = rv.key.labelValues("tag1".utf8).asNewString.split("-")(1)
+ rv.rows.map(_.getDouble(1)).foreach(_ shouldEqual i.toDouble * 2)
+ }
+
+ result.map(_.key).toSet.size shouldEqual 2
+ }
}
diff --git a/query/src/test/scala/filodb/query/exec/PeriodicSamplesMapperSpec.scala b/query/src/test/scala/filodb/query/exec/PeriodicSamplesMapperSpec.scala
index d412c544d0..03ec1a07f8 100644
--- a/query/src/test/scala/filodb/query/exec/PeriodicSamplesMapperSpec.scala
+++ b/query/src/test/scala/filodb/query/exec/PeriodicSamplesMapperSpec.scala
@@ -37,5 +37,8 @@ class PeriodicSamplesMapperSpec extends FunSpec with Matchers with ScalaFutures
resultRows.foreach(_.toList shouldEqual expectedResults)
+ val outSchema = periodicSamplesVectorFnMapper.schema(MetricsTestData.timeseriesDataset, resultSchema)
+ outSchema.columns shouldEqual resultSchema.columns
+ outSchema.fixedVectorLen shouldEqual Some(6)
}
}
diff --git a/query/src/test/scala/filodb/query/exec/rangefn/LableJoinSpec.scala b/query/src/test/scala/filodb/query/exec/rangefn/LableJoinSpec.scala
new file mode 100644
index 0000000000..74e728ffdc
--- /dev/null
+++ b/query/src/test/scala/filodb/query/exec/rangefn/LableJoinSpec.scala
@@ -0,0 +1,266 @@
+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 LableJoinSpec 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"),
+ ZeroCopyUTF8String("src1") -> ZeroCopyUTF8String("source-value-1"),
+ ZeroCopyUTF8String("src2") -> ZeroCopyUTF8String("source-value-2"),
+ ZeroCopyUTF8String("dst") -> ZeroCopyUTF8String("dst-value")
+ )
+ )
+
+ val testKey2 = CustomRangeVectorKey(
+ Map(
+ ZeroCopyUTF8String("src") -> ZeroCopyUTF8String("src-value"),
+ ZeroCopyUTF8String("src1") -> ZeroCopyUTF8String("src1-value"),
+ ZeroCopyUTF8String("src2") -> ZeroCopyUTF8String("src2-value"),
+ ZeroCopyUTF8String("dst") -> ZeroCopyUTF8String("original-dst-value")
+ )
+ )
+
+ val testKey3 = CustomRangeVectorKey(
+ Map(
+ ZeroCopyUTF8String("src") -> ZeroCopyUTF8String("src-value"),
+ ZeroCopyUTF8String("src1") -> ZeroCopyUTF8String("src1-value"),
+ ZeroCopyUTF8String("src2") -> ZeroCopyUTF8String("src2-value")
+ )
+ )
+
+ val testKey4 = CustomRangeVectorKey(
+ Map(
+ ZeroCopyUTF8String("src") -> ZeroCopyUTF8String("src-value"),
+ ZeroCopyUTF8String("src1") -> ZeroCopyUTF8String("src1-value"),
+ ZeroCopyUTF8String("src2") -> ZeroCopyUTF8String("src2-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 sampleWithoutDst: Array[RangeVector] = Array(
+ new RangeVector {
+ override def key: RangeVectorKey = testKey3
+
+ override def rows: Iterator[RowReader] = Seq(
+ new TransientRow(1L, 3.3d),
+ new TransientRow(2L, 5.1d)).iterator
+ },
+ new RangeVector {
+ override def key: RangeVectorKey = testKey4
+
+ override def rows: Iterator[RowReader] = Seq(
+ new TransientRow(3L, 100d),
+ new TransientRow(4L, 200d)).iterator
+ })
+
+ val queryConfig = new QueryConfig(config.getConfig("query"))
+
+ it("label_join joins all src values in order") {
+
+ val expectedLabels = List(Map(ZeroCopyUTF8String("src") -> ZeroCopyUTF8String("source-value"),
+ ZeroCopyUTF8String("src1") -> ZeroCopyUTF8String("source-value-1"),
+ ZeroCopyUTF8String("src2") -> ZeroCopyUTF8String("source-value-2"),
+ ZeroCopyUTF8String("dst") -> ZeroCopyUTF8String("source-value-source-value-1-source-value-2")
+ ),
+ Map(ZeroCopyUTF8String("src") -> ZeroCopyUTF8String("src-value"),
+ ZeroCopyUTF8String("src1") -> ZeroCopyUTF8String("src1-value"),
+ ZeroCopyUTF8String("src2") -> ZeroCopyUTF8String("src2-value"),
+ ZeroCopyUTF8String("dst") -> ZeroCopyUTF8String("src-value-src1-value-src2-value"))
+ )
+
+ val funcParams = Seq("dst", "-", "src", "src1", "src2")
+ val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelJoin, 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
+
+ //label_join should not change rows
+ testSample.map(_.rows.map(_.getDouble(1))).zip(resultRows).foreach {
+ case (ex, res) => {
+ ex.zip(res).foreach {
+ case (val1, val2) =>
+ val1 shouldEqual val2
+ }
+ }
+ }
+ }
+ it("label_join should treat label which is not present as empty string ") {
+
+ val expectedLabels = List(Map(ZeroCopyUTF8String("src") -> ZeroCopyUTF8String("source-value"),
+ ZeroCopyUTF8String("src1") -> ZeroCopyUTF8String("source-value-1"),
+ ZeroCopyUTF8String("src2") -> ZeroCopyUTF8String("source-value-2"),
+ ZeroCopyUTF8String("dst") -> ZeroCopyUTF8String("source-value--source-value-1")
+ ),
+ Map(ZeroCopyUTF8String("src") -> ZeroCopyUTF8String("src-value"),
+ ZeroCopyUTF8String("src1") -> ZeroCopyUTF8String("src1-value"),
+ ZeroCopyUTF8String("src2") -> ZeroCopyUTF8String("src2-value"),
+ ZeroCopyUTF8String("dst") -> ZeroCopyUTF8String("src-value--src1-value"))
+ )
+
+ val funcParams = Seq("dst", "-", "src", "src3", "src1")
+ val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelJoin, 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
+
+ //label_join should not change rows
+ testSample.map(_.rows.map(_.getDouble(1))).zip(resultRows).foreach {
+ case (ex, res) => {
+ ex.zip(res).foreach {
+ case (val1, val2) =>
+ val1 shouldEqual val2
+ }
+ }
+ }
+ }
+
+ it("label_join should overwrite destination label even if resulting dst label is empty string") {
+
+ val expectedLabels = List(Map(ZeroCopyUTF8String("src") -> ZeroCopyUTF8String("source-value"),
+ ZeroCopyUTF8String("src1") -> ZeroCopyUTF8String("source-value-1"),
+ ZeroCopyUTF8String("src2") -> ZeroCopyUTF8String("source-value-2")
+ ),
+ Map(ZeroCopyUTF8String("src") -> ZeroCopyUTF8String("src-value"),
+ ZeroCopyUTF8String("src1") -> ZeroCopyUTF8String("src1-value"),
+ ZeroCopyUTF8String("src2") -> ZeroCopyUTF8String("src2-value")
+ ))
+
+ val funcParams = Seq("dst", "", "emptysrc", "emptysrc1", "emptysrc2")
+ val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelJoin, 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
+
+ //label_join should not change rows
+ testSample.map(_.rows.map(_.getDouble(1))).zip(resultRows).foreach {
+ case (ex, res) => {
+ ex.zip(res).foreach {
+ case (val1, val2) =>
+ val1 shouldEqual val2
+ }
+ }
+ }
+ }
+ it("label_join should create destination label if it is not present") {
+
+ val expectedLabels = List(Map(ZeroCopyUTF8String("src") -> ZeroCopyUTF8String("source-value"),
+ ZeroCopyUTF8String("src1") -> ZeroCopyUTF8String("source-value-1"),
+ ZeroCopyUTF8String("src2") -> ZeroCopyUTF8String("source-value-2"),
+ ZeroCopyUTF8String("dst") -> ZeroCopyUTF8String("source-value-source-value-1-source-value-2")
+ ),
+ Map(ZeroCopyUTF8String("src") -> ZeroCopyUTF8String("src-value"),
+ ZeroCopyUTF8String("src1") -> ZeroCopyUTF8String("src1-value"),
+ ZeroCopyUTF8String("src2") -> ZeroCopyUTF8String("src2-value"),
+ ZeroCopyUTF8String("dst") -> ZeroCopyUTF8String("src-value-src1-value-src2-value"))
+ )
+
+ val funcParams = Seq("dst", "-", "src", "src1", "src2")
+ val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelJoin, 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
+
+ //label_join should not change rows
+ testSample.map(_.rows.map(_.getDouble(1))).zip(resultRows).foreach {
+ case (ex, res) => {
+ ex.zip(res).foreach {
+ case (val1, val2) =>
+ val1 shouldEqual val2
+ }
+ }
+ }
+ }
+
+ it("label_join should remove destination label if source labels are not specified") {
+
+ val expectedLabels = List(Map(ZeroCopyUTF8String("src") -> ZeroCopyUTF8String("source-value"),
+ ZeroCopyUTF8String("src1") -> ZeroCopyUTF8String("source-value-1"),
+ ZeroCopyUTF8String("src2") -> ZeroCopyUTF8String("source-value-2")
+ ),
+ Map(ZeroCopyUTF8String("src") -> ZeroCopyUTF8String("src-value"),
+ ZeroCopyUTF8String("src1") -> ZeroCopyUTF8String("src1-value"),
+ ZeroCopyUTF8String("src2") -> ZeroCopyUTF8String("src2-value")
+ ))
+
+ val funcParams = Seq("dst", "-")
+ val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelJoin, 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
+
+ //label_join should not change rows
+ testSample.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 funcParams1 = Seq("dst", "-", "src$", "src1", "src2")
+ val funcParams2 = Seq("dst#", "-", "src", "src1", "src2")
+
+ the[IllegalArgumentException] thrownBy {
+ val miscellaneousFunctionMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelJoin,
+ funcParams1)
+ miscellaneousFunctionMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema)
+ } should have message "requirement failed: Invalid source label name in label_join()"
+
+ the[IllegalArgumentException] thrownBy {
+ val miscellaneousFunctionMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelJoin,
+ funcParams2)
+ miscellaneousFunctionMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema)
+ } should have message "requirement failed: Invalid destination label name in label_join()"
+
+ the[IllegalArgumentException] thrownBy {
+ val miscellaneousFunctionMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelJoin,
+ Seq("dst"))
+ miscellaneousFunctionMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema)
+ } should have message "requirement failed: expected at least 3 argument(s) in call to label_join"
+ }
+}
diff --git a/scalastyle-config.xml b/scalastyle-config.xml
index 845bbe3cb2..b3edf81229 100644
--- a/scalastyle-config.xml
+++ b/scalastyle-config.xml
@@ -129,7 +129,7 @@
- COLON, IF
+ COLON, IF, EQUALS, COMMA
diff --git a/spark/src/main/scala/filodb.spark/FiloRelation.scala b/spark/src/main/scala/filodb.spark/FiloRelation.scala
index 16a1e9f953..197369b26e 100644
--- a/spark/src/main/scala/filodb.spark/FiloRelation.scala
+++ b/spark/src/main/scala/filodb.spark/FiloRelation.scala
@@ -102,9 +102,9 @@ object FiloRelation extends StrictLogging {
}
val betweenRangePF: PartialFunction[(Column, Seq[Filter]), Option[(Any, Any)]] = {
- case (_, Seq(GreaterThan(_, lVal), LessThan(_, rVal))) => Some((lVal, rVal))
+ case (_, Seq(GreaterThan(_, lVal), LessThan(_, rVal))) => Some((lVal, rVal))
case (_, Seq(GreaterThanOrEqual(_, lVal), LessThan(_, rVal))) => Some((lVal, rVal))
- case (_, Seq(GreaterThan(_, lVal), LessThanOrEqual(_, rVal))) => Some((lVal, rVal))
+ case (_, Seq(GreaterThan(_, lVal), LessThanOrEqual(_, rVal))) => Some((lVal, rVal))
case (_, Seq(GreaterThanOrEqual(_, lVal), LessThanOrEqual(_, rVal))) => Some((lVal, rVal))
}
diff --git a/version.sbt b/version.sbt
index 4c1ac90374..ec2d572313 100644
--- a/version.sbt
+++ b/version.sbt
@@ -1 +1,2 @@
-version in ThisBuild := "0.8.4.4"
+version in ThisBuild := "0.8.5"
+