diff --git a/cassandra/src/test/scala/filodb.cassandra/MemstoreCassandraSinkSpec.scala b/cassandra/src/test/scala/filodb.cassandra/MemstoreCassandraSinkSpec.scala
index b23bb1623a..f11dffa5d9 100644
--- a/cassandra/src/test/scala/filodb.cassandra/MemstoreCassandraSinkSpec.scala
+++ b/cassandra/src/test/scala/filodb.cassandra/MemstoreCassandraSinkSpec.scala
@@ -34,7 +34,7 @@ class MemstoreCassandraSinkSpec extends AllTablesTest {
}
it("should flush MemStore data to C*, and be able to read back data from C* directly") {
- memStore.setup(dataset1.ref, Schemas(dataset1.schema), 0, TestData.storeConf)
+ memStore.setup(dataset1.ref, Schemas(dataset1.schema), 0, TestData.storeConf, 1)
memStore.store.sinkStats.chunksetsWritten.get shouldEqual 0
// Flush every ~50 records
diff --git a/cassandra/src/test/scala/filodb.cassandra/columnstore/OdpSpec.scala b/cassandra/src/test/scala/filodb.cassandra/columnstore/OdpSpec.scala
index 4216d83486..9891688269 100644
--- a/cassandra/src/test/scala/filodb.cassandra/columnstore/OdpSpec.scala
+++ b/cassandra/src/test/scala/filodb.cassandra/columnstore/OdpSpec.scala
@@ -96,7 +96,7 @@ class OdpSpec extends AnyFunSpec with Matchers with BeforeAndAfterAll with Scala
val policy = new FixedMaxPartitionsEvictionPolicy(20)
val memStore = new TimeSeriesMemStore(config, colStore, new InMemoryMetaStore(), Some(policy))
try {
- memStore.setup(dataset.ref, schemas, 0, TestData.storeConf)
+ memStore.setup(dataset.ref, schemas, 0, TestData.storeConf, 1)
memStore.recoverIndex(dataset.ref, 0).futureValue
memStore.refreshIndexForTesting(dataset.ref)
@@ -112,7 +112,7 @@ class OdpSpec extends AnyFunSpec with Matchers with BeforeAndAfterAll with Scala
val policy = new FixedMaxPartitionsEvictionPolicy(20)
val memStore = new TimeSeriesMemStore(config, colStore, new InMemoryMetaStore(), Some(policy))
try {
- memStore.setup(dataset.ref, schemas, 0, TestData.storeConf)
+ memStore.setup(dataset.ref, schemas, 0, TestData.storeConf, 1)
memStore.recoverIndex(dataset.ref, 0).futureValue
memStore.refreshIndexForTesting(dataset.ref)
@@ -134,7 +134,7 @@ class OdpSpec extends AnyFunSpec with Matchers with BeforeAndAfterAll with Scala
val policy = new FixedMaxPartitionsEvictionPolicy(20)
val memStore = new TimeSeriesMemStore(config, colStore, new InMemoryMetaStore(), Some(policy))
try {
- memStore.setup(dataset.ref, schemas, 0, TestData.storeConf)
+ memStore.setup(dataset.ref, schemas, 0, TestData.storeConf, 1)
memStore.recoverIndex(dataset.ref, 0).futureValue
memStore.refreshIndexForTesting(dataset.ref)
@@ -157,7 +157,7 @@ class OdpSpec extends AnyFunSpec with Matchers with BeforeAndAfterAll with Scala
val policy = new FixedMaxPartitionsEvictionPolicy(20)
val memStore = new TimeSeriesMemStore(config, colStore, new InMemoryMetaStore(), Some(policy))
try {
- memStore.setup(dataset.ref, schemas, 0, TestData.storeConf)
+ memStore.setup(dataset.ref, schemas, 0, TestData.storeConf, 1)
memStore.recoverIndex(dataset.ref, 0).futureValue
memStore.refreshIndexForTesting(dataset.ref)
diff --git a/conf/promperf-filodb-server.conf b/conf/promperf-filodb-server.conf
index 9359b8d366..15de43e558 100644
--- a/conf/promperf-filodb-server.conf
+++ b/conf/promperf-filodb-server.conf
@@ -5,8 +5,8 @@ filodb {
"conf/promperf-source.conf"
]
+ min-num-nodes-in-cluster = 1
cluster-discovery {
- num-nodes = 1
failure-detection-interval = 20s
host-list = [
"127.0.0.1:2552"
diff --git a/conf/timeseries-dev-source.conf b/conf/timeseries-dev-source.conf
index ea1a09d11d..da732b47ed 100644
--- a/conf/timeseries-dev-source.conf
+++ b/conf/timeseries-dev-source.conf
@@ -6,7 +6,10 @@
# Should not change once dataset has been set up on the server and data has been persisted to cassandra
num-shards = 4
+ # deprecated in favor of min-num-nodes-in-cluster config in filodb server config
+ # To be removed eventually. There is no reason to set a value for this for each dataset
min-num-nodes = 2
+
# Length of chunks to be written, roughly
sourcefactory = "filodb.kafka.KafkaIngestionStreamFactory"
diff --git a/conf/timeseries-filodb-server.conf b/conf/timeseries-filodb-server.conf
index c36293bd30..c2ce99a7fa 100644
--- a/conf/timeseries-filodb-server.conf
+++ b/conf/timeseries-filodb-server.conf
@@ -1,9 +1,9 @@
dataset-prometheus = { include required("timeseries-dev-source.conf") }
filodb {
+ min-num-nodes-in-cluster = 2
v2-cluster-enabled = false
cluster-discovery {
- num-nodes = 2
failure-detection-interval = 20s
host-list = [
"127.0.0.1:2552",
diff --git a/coordinator/src/main/scala/filodb.coordinator/FilodbSettings.scala b/coordinator/src/main/scala/filodb.coordinator/FilodbSettings.scala
index e543ccf8b8..4b411b61ac 100755
--- a/coordinator/src/main/scala/filodb.coordinator/FilodbSettings.scala
+++ b/coordinator/src/main/scala/filodb.coordinator/FilodbSettings.scala
@@ -48,13 +48,13 @@ final class FilodbSettings(val conf: Config) {
lazy val datasetConfPaths = config.as[Seq[String]]("dataset-configs")
- lazy val numNodes = config.getInt("cluster-discovery.num-nodes")
lazy val k8sHostFormat = config.as[Option[String]]("cluster-discovery.k8s-stateful-sets-hostname-format")
// used for development mode only
lazy val hostList = config.as[Option[Seq[String]]]("cluster-discovery.host-list")
lazy val localhostOrdinal = config.as[Option[Int]]("cluster-discovery.localhost-ordinal")
+ lazy val minNumNodes = config.as[Option[Int]]("min-num-nodes-in-cluster")
/**
* Returns IngestionConfig/dataset configuration from parsing dataset-configs file paths.
diff --git a/coordinator/src/main/scala/filodb.coordinator/IngestionActor.scala b/coordinator/src/main/scala/filodb.coordinator/IngestionActor.scala
index 0c67cedab5..183ca08ec7 100644
--- a/coordinator/src/main/scala/filodb.coordinator/IngestionActor.scala
+++ b/coordinator/src/main/scala/filodb.coordinator/IngestionActor.scala
@@ -37,8 +37,9 @@ object IngestionActor {
source: NodeClusterActor.IngestionSource,
downsample: DownsampleConfig,
storeConfig: StoreConfig,
+ numShards: Int,
statusActor: ActorRef): Props =
- Props(new IngestionActor(ref, schemas, memStore, source, downsample, storeConfig, statusActor))
+ Props(new IngestionActor(ref, schemas, memStore, source, downsample, storeConfig, numShards, statusActor))
}
/**
@@ -62,6 +63,7 @@ private[filodb] final class IngestionActor(ref: DatasetRef,
source: NodeClusterActor.IngestionSource,
downsample: DownsampleConfig,
storeConfig: StoreConfig,
+ numShards: Int,
statusActor: ActorRef) extends BaseActor {
import IngestionActor._
@@ -170,7 +172,7 @@ private[filodb] final class IngestionActor(ref: DatasetRef,
// scalastyle:off method.length
private def startIngestion(shard: Int): Unit = {
- try tsStore.setup(ref, schemas, shard, storeConfig, downsample) catch {
+ try tsStore.setup(ref, schemas, shard, storeConfig, numShards, downsample) catch {
case ShardAlreadySetup(ds, s) =>
logger.warn(s"dataset=$ds shard=$s already setup, skipping....")
return
diff --git a/coordinator/src/main/scala/filodb.coordinator/NodeCoordinatorActor.scala b/coordinator/src/main/scala/filodb.coordinator/NodeCoordinatorActor.scala
index f3abf68bbc..6896d73ae3 100644
--- a/coordinator/src/main/scala/filodb.coordinator/NodeCoordinatorActor.scala
+++ b/coordinator/src/main/scala/filodb.coordinator/NodeCoordinatorActor.scala
@@ -115,6 +115,7 @@ private[filodb] final class NodeCoordinatorActor(metaStore: MetaStore,
setupDataset( dataset,
ingestConfig.storeConfig,
+ ingestConfig.numShards,
IngestionSource(ingestConfig.streamFactoryClass, ingestConfig.sourceConfig),
ingestConfig.downsampleConfig)
}
@@ -141,6 +142,7 @@ private[filodb] final class NodeCoordinatorActor(metaStore: MetaStore,
*/
private def setupDataset(dataset: Dataset,
storeConf: StoreConfig,
+ numShards: Int,
source: IngestionSource,
downsample: DownsampleConfig,
schemaOverride: Boolean = false): Unit = {
@@ -154,7 +156,7 @@ private[filodb] final class NodeCoordinatorActor(metaStore: MetaStore,
val schemas = if (schemaOverride) Schemas(dataset.schema) else settings.schemas
if (schemaOverride) logger.info(s"Overriding schemas from settings: this better be a test!")
val props = IngestionActor.props(dataset.ref, schemas, memStore,
- source, downsample, storeConf, statusActor.get)
+ source, downsample, storeConf, numShards, statusActor.get)
val ingester = context.actorOf(props, s"$Ingestion-${dataset.name}")
context.watch(ingester)
ingesters(ref) = ingester
@@ -187,7 +189,9 @@ private[filodb] final class NodeCoordinatorActor(metaStore: MetaStore,
def ingestHandlers: Receive = LoggingReceive {
case SetupDataset(dataset, resources, source, storeConf, downsample) =>
// used only in unit tests
- if (!(ingesters contains dataset.ref)) { setupDataset(dataset, storeConf, source, downsample, true) }
+ if (!(ingesters contains dataset.ref)) {
+ setupDataset(dataset, storeConf, resources.numShards, source, downsample, true)
+ }
case IngestRows(dataset, shard, rows) =>
withIngester(sender(), dataset) { _ ! IngestionActor.IngestRows(sender(), shard, rows) }
diff --git a/coordinator/src/main/scala/filodb.coordinator/ShardManager.scala b/coordinator/src/main/scala/filodb.coordinator/ShardManager.scala
index 12a68e24b7..dd63e13ab8 100644
--- a/coordinator/src/main/scala/filodb.coordinator/ShardManager.scala
+++ b/coordinator/src/main/scala/filodb.coordinator/ShardManager.scala
@@ -402,7 +402,8 @@ private[coordinator] final class ShardManager(settings: FilodbSettings,
ackTo.foreach(_ ! DatasetExists(dataset.ref))
Map.empty
case None =>
- val resources = DatasetResourceSpec(ingestConfig.numShards, ingestConfig.minNumNodes)
+ val minNumNodes = settings.minNumNodes.getOrElse(ingestConfig.minNumNodes)
+ val resources = DatasetResourceSpec(ingestConfig.numShards, minNumNodes)
val mapper = new ShardMapper(resources.numShards)
_shardMappers(dataset.ref) = mapper
// Access the shardmapper through the HashMap so even if it gets replaced it will update the shard stats
diff --git a/coordinator/src/main/scala/filodb/coordinator/v2/FiloDbClusterDiscovery.scala b/coordinator/src/main/scala/filodb/coordinator/v2/FiloDbClusterDiscovery.scala
index f91968bf45..9db3f19776 100644
--- a/coordinator/src/main/scala/filodb/coordinator/v2/FiloDbClusterDiscovery.scala
+++ b/coordinator/src/main/scala/filodb/coordinator/v2/FiloDbClusterDiscovery.scala
@@ -40,11 +40,13 @@ class FiloDbClusterDiscovery(settings: FilodbSettings,
}
def shardsForOrdinal(ordinal: Int, numShards: Int): Seq[Int] = {
- require(ordinal < settings.numNodes, s"Ordinal $ordinal was not expected. Number of nodes is ${settings.numNodes}")
- val numShardsPerHost = numShards / settings.numNodes
+ require(settings.minNumNodes.isDefined, "Minimum Number of Nodes config not provided")
+ require(ordinal < settings.minNumNodes.get, s"Ordinal $ordinal was not expected. " +
+ s"Number of nodes is ${settings.minNumNodes.get}")
+ val numShardsPerHost = numShards / settings.minNumNodes.get
// Suppose we have a total of 8 shards and 2 hosts, assuming the hostnames are host-0 and host-1, we will map
// host-0 to shard [0,1,2,3] and host-1 to shard [4,5,6,7]
- val numExtraShardsToAssign = numShards % settings.numNodes
+ val numExtraShardsToAssign = numShards % settings.minNumNodes.get
val (firstShardThisNode, numShardsThisHost) = if (numExtraShardsToAssign != 0) {
logger.warn("For stateful shard assignment, numShards should be a multiple of nodes per shard, " +
"using default strategy")
@@ -69,8 +71,9 @@ class FiloDbClusterDiscovery(settings: FilodbSettings,
def shardsForLocalhost(numShards: Int): Seq[Int] = shardsForOrdinal(ordinalOfLocalhost, numShards)
lazy private val hostNames = {
+ require(settings.minNumNodes.isDefined, "Minimum Number of Nodes config not provided")
if (settings.k8sHostFormat.isDefined) {
- (0 until settings.numNodes).map(i => String.format(settings.k8sHostFormat.get, i.toString))
+ (0 until settings.minNumNodes.get).map(i => String.format(settings.k8sHostFormat.get, i.toString))
} else if (settings.hostList.isDefined) {
settings.hostList.get.sorted // sort to make order consistent on all nodes of cluster
} else throw new IllegalArgumentException("Cluster Discovery mechanism not defined")
diff --git a/coordinator/src/main/scala/filodb/coordinator/v2/NewNodeCoordinatorActor.scala b/coordinator/src/main/scala/filodb/coordinator/v2/NewNodeCoordinatorActor.scala
index 6462b356e4..e96b054160 100644
--- a/coordinator/src/main/scala/filodb/coordinator/v2/NewNodeCoordinatorActor.scala
+++ b/coordinator/src/main/scala/filodb/coordinator/v2/NewNodeCoordinatorActor.scala
@@ -84,7 +84,7 @@ private[filodb] final class NewNodeCoordinatorActor(memStore: TimeSeriesStore,
.foreach { downsampleDataset => memStore.store.initialize(downsampleDataset, ingestConfig.numShards) }
setupDataset( dataset,
- ingestConfig.storeConfig,
+ ingestConfig.storeConfig, ingestConfig.numShards,
IngestionSource(ingestConfig.streamFactoryClass, ingestConfig.sourceConfig),
ingestConfig.downsampleConfig)
initShards(dataset, ingestConfig)
@@ -121,6 +121,7 @@ private[filodb] final class NewNodeCoordinatorActor(memStore: TimeSeriesStore,
*/
private def setupDataset(dataset: Dataset,
storeConf: StoreConfig,
+ numShards: Int,
source: IngestionSource,
downsample: DownsampleConfig,
schemaOverride: Boolean = false): Unit = {
@@ -132,7 +133,7 @@ private[filodb] final class NewNodeCoordinatorActor(memStore: TimeSeriesStore,
val schemas = if (schemaOverride) Schemas(dataset.schema) else settings.schemas
if (schemaOverride) logger.info(s"Overriding schemas from settings: this better be a test!")
val props = IngestionActor.props(dataset.ref, schemas, memStore,
- source, downsample, storeConf, self)
+ source, downsample, storeConf, numShards, self)
val ingester = context.actorOf(props, s"$Ingestion-${dataset.name}")
context.watch(ingester)
ingestionActors(ref) = ingester
diff --git a/coordinator/src/test/scala/filodb.coordinator/FiloDbClusterDiscoverySpec.scala b/coordinator/src/test/scala/filodb.coordinator/FiloDbClusterDiscoverySpec.scala
index cade043c61..2c2c729e22 100644
--- a/coordinator/src/test/scala/filodb.coordinator/FiloDbClusterDiscoverySpec.scala
+++ b/coordinator/src/test/scala/filodb.coordinator/FiloDbClusterDiscoverySpec.scala
@@ -11,7 +11,7 @@ class FiloDbClusterDiscoverySpec extends AkkaSpec {
val config = ConfigFactory.parseString(
"""
- |filodb.cluster-discovery.num-nodes = 4
+ |filodb.min-num-nodes-in-cluster = 4
|""".stripMargin)
val settings = new FilodbSettings(config)
@@ -29,7 +29,7 @@ class FiloDbClusterDiscoverySpec extends AkkaSpec {
"Should allocate the extra n shards to first n nodes" in {
val config = ConfigFactory.parseString(
"""
- |filodb.cluster-discovery.num-nodes = 5
+ |filodb.min-num-nodes-in-cluster = 5
|""".stripMargin)
val settings = new FilodbSettings(config)
diff --git a/core/src/main/resources/filodb-defaults.conf b/core/src/main/resources/filodb-defaults.conf
index e2cac29f02..1d92e10964 100644
--- a/core/src/main/resources/filodb-defaults.conf
+++ b/core/src/main/resources/filodb-defaults.conf
@@ -1,11 +1,16 @@
filodb {
v2-cluster-enabled = false
+
+ # Number of nodes in cluster; used to calculate per-shard resources based
+ # on how many shards assigned to node.
+ # Required config if v2-clustering or automatic memory-alloc is enabled
+ # min-num-nodes-in-cluster = 2
+
cluster-discovery {
// set this to a smaller value (like 30s) at thee query entry points
// if FiloDB HTTP API is indeed the query entry point, this should be overridden to small value
// so that failure detection is quick.
failure-detection-interval = 15 minutes
- num-nodes = 2
# one of the two below properties should be enabled
# enable this to use the k8s stateful sets mode and its hostname to extract ordinal
@@ -691,6 +696,41 @@ filodb {
# Note: this memory is shared across all configued datasets on a node.
ingestion-buffer-mem-size = 200MB
+ memory-alloc {
+ # automatic memory allocation is enabled if true
+ automatic-alloc-enabled = false
+
+ # if not provided this is calculated as ContainerOrNodeMemory - os-memory-needs - CurrentJVMHeapMemory
+ # available-memory-bytes = 5GB
+
+ # memory dedicated for proper functioning of OS
+ os-memory-needs = 500MB
+
+ # NOTE: In the three configs below,
+ # lucene-memory-percent + native-memory-manager-percent + block-memory-manager-percent
+ # should equal 100
+ ##############################################################
+ # # # #
+ # LuceneMemPercent # NativeMemPercent # BlockMemPercent #
+ # # # #
+ ##############################################################
+
+ # Memory percent of available-memory reserved for Lucene memory maps.
+ # Note we do not use this config to explicitly allocate space for lucene.
+ # But reserving this space ensures that more of the lucene memory maps are stored in memory
+ lucene-memory-percent = 5
+
+ # memory percent of available-memory reserved for native memory manager
+ # (used for partKeys, chunkMaps, chunkInfos, writeBuffers)
+ native-memory-manager-percent = 24
+
+ # Memory percent of available-memory reserved for block memory manager
+ # (used for storing chunks)
+ # This is divvied amongst datasets on the node per configuration for dataset
+ # The shards of the dataset on the node get even amount of memory from this fraction
+ block-memory-manager-percent = 71
+ }
+
# At the cost of some extra heap memory, we can track queries holding shared lock for a long time
# and starving the exclusive access of lock for eviction
track-queries-holding-eviction-lock = true
diff --git a/core/src/main/scala/filodb.core/Utils.scala b/core/src/main/scala/filodb.core/Utils.scala
index 724c4556fb..7968fcd257 100644
--- a/core/src/main/scala/filodb.core/Utils.scala
+++ b/core/src/main/scala/filodb.core/Utils.scala
@@ -2,6 +2,7 @@ package filodb.core
import java.lang.management.ManagementFactory
+import com.typesafe.config.{Config, ConfigRenderOptions}
import com.typesafe.scalalogging.StrictLogging
object Utils extends StrictLogging {
@@ -13,4 +14,28 @@ object Utils extends StrictLogging {
if (cpuTimeEnabled) threadMbean.getCurrentThreadCpuTime
else System.nanoTime()
}
+
+ def calculateAvailableOffHeapMemory(filodbConfig: Config): Long = {
+ val containerMemory = ManagementFactory.getOperatingSystemMXBean()
+ .asInstanceOf[com.sun.management.OperatingSystemMXBean].getTotalPhysicalMemorySize()
+ val currentJavaHeapMemory = Runtime.getRuntime().maxMemory()
+ val osMemoryNeeds = filodbConfig.getMemorySize("memstore.memory-alloc.os-memory-needs").toBytes
+ logger.info(s"Detected available memory containerMemory=$containerMemory" +
+ s" currentJavaHeapMemory=$currentJavaHeapMemory osMemoryNeeds=$osMemoryNeeds")
+
+ logger.info(s"Memory Alloc Options: " +
+ s"${filodbConfig.getConfig("memstore.memory-alloc").root().render(ConfigRenderOptions.concise())}")
+
+ val availableMem = if (filodbConfig.hasPath("memstore.memory-alloc.available-memory-bytes")) {
+ val avail = filodbConfig.getMemorySize("memstore.memory-alloc.available-memory-bytes").toBytes
+ logger.info(s"Using automatic-memory-config using overridden memory-alloc.available-memory $avail")
+ avail
+ } else {
+ logger.info(s"Using automatic-memory-config using without available memory override")
+ containerMemory - currentJavaHeapMemory - osMemoryNeeds
+ }
+ logger.info(s"Available memory calculated or configured as $availableMem")
+ availableMem
+ }
+
}
diff --git a/core/src/main/scala/filodb.core/binaryrecord2/RecordContainer.scala b/core/src/main/scala/filodb.core/binaryrecord2/RecordContainer.scala
index 86f96146fd..c2da9e672f 100644
--- a/core/src/main/scala/filodb.core/binaryrecord2/RecordContainer.scala
+++ b/core/src/main/scala/filodb.core/binaryrecord2/RecordContainer.scala
@@ -7,7 +7,7 @@ import scala.reflect.ClassTag
import debox.Buffer
import filodb.memory.{BinaryRegionConsumer, BinaryRegionLarge}
-import filodb.memory.format.{RowReader, UnsafeUtils}
+import filodb.memory.format.UnsafeUtils
/**
* A RecordContainer is a binary, wire-compatible container for BinaryRecords V2.
@@ -72,13 +72,13 @@ final class RecordContainer(val base: Any, val offset: Long, maxLength: Int,
* Iterates through each BinaryRecord as a RowReader. Results in two allocations: the Iterator
* as well as a BinaryRecordRowReader.
*/
- final def iterate(schema: RecordSchema): Iterator[RowReader] = new Iterator[RowReader] {
+ final def iterate(schema: RecordSchema): Iterator[BinaryRecordRowReader] = new Iterator[BinaryRecordRowReader] {
val reader = new BinaryRecordRowReader(schema, base)
val endOffset = offset + 4 + numBytes
var curOffset = offset + ContainerHeaderLen
final def hasNext: Boolean = curOffset < endOffset
- final def next: RowReader = {
+ final def next: BinaryRecordRowReader = {
val recordLen = BinaryRegionLarge.numBytes(base, curOffset)
reader.recordOffset = curOffset
curOffset += (recordLen + 7) & ~3 // +4, then aligned/rounded up to next 4 bytes
diff --git a/core/src/main/scala/filodb.core/binaryrecord2/RecordSchema.scala b/core/src/main/scala/filodb.core/binaryrecord2/RecordSchema.scala
index 42dddfd2ca..01a1252f7c 100644
--- a/core/src/main/scala/filodb.core/binaryrecord2/RecordSchema.scala
+++ b/core/src/main/scala/filodb.core/binaryrecord2/RecordSchema.scala
@@ -624,7 +624,12 @@ trait BinaryRecordRowReaderBase extends RowReader {
final class BinaryRecordRowReader(val schema: RecordSchema,
var recordBase: Any = UnsafeUtils.ZeroPointer,
- var recordOffset: Long = 0L) extends BinaryRecordRowReaderBase
+ var recordOffset: Long = 0L) extends BinaryRecordRowReaderBase {
+ def recordLength: Int = {
+ val len = BinaryRegionLarge.numBytes(recordBase, recordOffset)
+ (len + 7) & ~3 // +4, then aligned/rounded up to next 4 bytes
+ }
+}
final class MultiSchemaBRRowReader(var recordBase: Any = UnsafeUtils.ZeroPointer,
var recordOffset: Long = 0L) extends BinaryRecordRowReaderBase {
diff --git a/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesShard.scala b/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesShard.scala
index 7f3aca62e0..66174a3582 100644
--- a/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesShard.scala
+++ b/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesShard.scala
@@ -1,13 +1,14 @@
package filodb.core.downsample
+import java.util
+import java.util.concurrent.atomic.AtomicLong
+
import scala.collection.mutable
import scala.concurrent.{ExecutionContext, Future}
import scala.concurrent.duration._
import com.typesafe.config.Config
import com.typesafe.scalalogging.StrictLogging
-import java.util
-import java.util.concurrent.atomic.AtomicLong
import kamon.Kamon
import kamon.metric.MeasurementUnit
import kamon.tag.TagSet
@@ -16,12 +17,12 @@ import monix.execution.{CancelableFuture, Scheduler, UncaughtExceptionReporter}
import monix.reactive.Observable
import org.apache.lucene.search.CollectionTerminatedException
-import filodb.core.{DatasetRef, Types}
+import filodb.core.{DatasetRef, Types, Utils}
import filodb.core.binaryrecord2.RecordSchema
import filodb.core.memstore._
import filodb.core.memstore.ratelimit.{CardinalityManager, CardinalityRecord, QuotaSource}
import filodb.core.metadata.Schemas
-import filodb.core.query.{ColumnFilter, Filter, QueryContext, QueryLimitException, QuerySession, QueryWarnings}
+import filodb.core.query._
import filodb.core.store._
import filodb.memory.format.{UnsafeUtils, ZeroCopyUTF8String}
import filodb.memory.format.ZeroCopyUTF8String._
@@ -341,6 +342,7 @@ class DownsampledTimeSeriesShard(rawDatasetRef: DatasetRef,
// Second iteration is for query result evaluation. Loading everything to heap
// is expensive, but we do it to handle data sizing for metrics that have
// continuous churn. See capDataScannedPerShardCheck method.
+ val startNs = Utils.currentThreadCpuTimeNanos
val recs = partKeyIndex.partKeyRecordsFromFilters(filters, chunkMethod.startTime, chunkMethod.endTime)
val _schema = recs.headOption.map { pkRec =>
RecordSchema.schemaID(pkRec.partKey, UnsafeUtils.arayOffset)
@@ -353,9 +355,10 @@ class DownsampledTimeSeriesShard(rawDatasetRef: DatasetRef,
val metricGroupBy = deploymentPartitionName +: clusterType +: metricShardKeys.map { col =>
filters.collectFirst {
case ColumnFilter(c, Filter.Equals(filtVal: String)) if c == col => filtVal
- }.getOrElse("unknown")
+ }.getOrElse("multiple")
}.toList
querySession.queryStats.getTimeSeriesScannedCounter(metricGroupBy).addAndGet(recs.length)
+ querySession.queryStats.getCpuNanosCounter(metricGroupBy).addAndGet(Utils.currentThreadCpuTimeNanos - startNs)
val chunksReadCounter = querySession.queryStats.getDataBytesScannedCounter(metricGroupBy)
PartLookupResult(shardNum, chunkMethod, debox.Buffer.empty,
diff --git a/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesStore.scala b/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesStore.scala
index 9702442bec..5f83c06c9c 100644
--- a/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesStore.scala
+++ b/core/src/main/scala/filodb.core/downsample/DownsampledTimeSeriesStore.scala
@@ -59,7 +59,7 @@ extends TimeSeriesStore with StrictLogging {
override def metastore: MetaStore = ??? // Not needed
// TODO: Change the API to return Unit Or ShardAlreadySetup, instead of throwing. Make idempotent.
- def setup(ref: DatasetRef, schemas: Schemas, shard: Int, storeConf: StoreConfig,
+ def setup(ref: DatasetRef, schemas: Schemas, shard: Int, storeConf: StoreConfig, numShards: Int,
downsampleConfig: DownsampleConfig = DownsampleConfig.disabled): Unit = synchronized {
val shards = datasets.getOrElseUpdate(ref, new NonBlockingHashMapLong[DownsampledTimeSeriesShard](32, false))
val quotaSource = quotaSources.getOrElseUpdate(ref,
diff --git a/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala b/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala
index 91ee5c0e70..760fd9e823 100644
--- a/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala
+++ b/core/src/main/scala/filodb.core/memstore/OnDemandPagingShard.scala
@@ -26,6 +26,7 @@ import filodb.memory.NativeMemoryManager
class OnDemandPagingShard(ref: DatasetRef,
schemas: Schemas,
storeConfig: StoreConfig,
+ numShards: Int,
quotaSource: QuotaSource,
shardNum: Int,
bufferMemoryManager: NativeMemoryManager,
@@ -34,7 +35,7 @@ class OnDemandPagingShard(ref: DatasetRef,
evictionPolicy: PartitionEvictionPolicy,
filodbConfig: Config)
(implicit ec: ExecutionContext) extends
-TimeSeriesShard(ref, schemas, storeConfig, quotaSource, shardNum, bufferMemoryManager, rawStore,
+TimeSeriesShard(ref, schemas, storeConfig, numShards, quotaSource, shardNum, bufferMemoryManager, rawStore,
metastore, evictionPolicy, filodbConfig)(ec) {
import TimeSeriesShard._
import FiloSchedulers._
diff --git a/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala b/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala
index 89b5210ed2..d67f640c8d 100644
--- a/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala
+++ b/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala
@@ -46,7 +46,11 @@ import filodb.memory.{BinaryRegionLarge, UTF8StringMedium, UTF8StringShort}
import filodb.memory.format.{UnsafeUtils, ZeroCopyUTF8String => UTF8Str}
object PartKeyLuceneIndex {
- final val PART_ID = "__partId__"
+ // NOTE: these partId fields need to be separate because Lucene 9.7.0 enforces consistent types for document
+ // field values (i.e. a field cannot have both numeric and string values). Additional details can be found
+ // here: https://github.com/apache/lucene/pull/11
+ final val PART_ID_DV = "__partIdDv__"
+ final val PART_ID_FIELD = "__partIdField__"
final val START_TIME = "__startTime__"
final val END_TIME = "__endTime__"
final val PART_KEY = "__partKey__"
@@ -54,7 +58,7 @@ object PartKeyLuceneIndex {
final val FACET_FIELD_PREFIX = "$facet_"
final val LABEL_LIST_FACET = FACET_FIELD_PREFIX + LABEL_LIST
- final val ignoreIndexNames = HashSet(START_TIME, PART_KEY, END_TIME, PART_ID)
+ final val ignoreIndexNames = HashSet(START_TIME, PART_KEY, END_TIME, PART_ID_FIELD, PART_ID_DV)
val MAX_STR_INTERN_ENTRIES = 10000
val MAX_TERMS_TO_ITERATE = 10000
@@ -279,8 +283,8 @@ class PartKeyLuceneIndex(ref: DatasetRef,
var facetsConfig: FacetsConfig = _
val document = new Document()
- private[memstore] val partIdField = new StringField(PART_ID, "0", Store.NO)
- private val partIdDv = new NumericDocValuesField(PART_ID, 0)
+ private[memstore] val partIdField = new StringField(PART_ID_FIELD, "0", Store.NO)
+ private val partIdDv = new NumericDocValuesField(PART_ID_DV, 0)
private val partKeyDv = new BinaryDocValuesField(PART_KEY, new BytesRef())
private val startTimeField = new LongPoint(START_TIME, 0L)
private val startTimeDv = new NumericDocValuesField(START_TIME, 0L)
@@ -328,6 +332,13 @@ class PartKeyLuceneIndex(ref: DatasetRef,
partIdDv.setLongValue(partId)
document.add(partIdDv)
}
+
+ /*
+ * As of this writing, this documentId will be set as one of two values:
+ * - In TimeSeriesShard: the string representation of a partId (e.g. "42")
+ * - In DownsampledTimeSeriesShard: the base64-encoded sha256 of the document ID. This is used to support
+ * persistence of the downsample index; ephemeral partIds cannot be used.
+ */
partIdField.setStringValue(documentId)
startTimeField.setLongValue(startTime)
@@ -468,7 +479,7 @@ class PartKeyLuceneIndex(ref: DatasetRef,
cforRange { 0 until partIds.length } { i =>
terms.add(new BytesRef(partIds(i).toString.getBytes(StandardCharsets.UTF_8)))
}
- indexWriter.deleteDocuments(new TermInSetQuery(PART_ID, terms))
+ indexWriter.deleteDocuments(new TermInSetQuery(PART_ID_FIELD, terms))
}
}
@@ -646,7 +657,7 @@ class PartKeyLuceneIndex(ref: DatasetRef,
s"with startTime=$startTime endTime=$endTime into dataset=$ref shard=$shardNum")
val doc = luceneDocument.get()
val docToAdd = doc.facetsConfig.build(doc.document)
- val term = new Term(PART_ID, documentId)
+ val term = new Term(PART_ID_FIELD, documentId)
indexWriter.updateDocument(term, docToAdd)
}
@@ -711,7 +722,7 @@ class PartKeyLuceneIndex(ref: DatasetRef,
*/
def partKeyFromPartId(partId: Int): Option[BytesRef] = {
val collector = new SinglePartKeyCollector()
- withNewSearcher(s => s.search(new TermQuery(new Term(PART_ID, partId.toString)), collector) )
+ withNewSearcher(s => s.search(new TermQuery(new Term(PART_ID_FIELD, partId.toString)), collector) )
Option(collector.singleResult)
}
@@ -720,7 +731,7 @@ class PartKeyLuceneIndex(ref: DatasetRef,
*/
def startTimeFromPartId(partId: Int): Long = {
val collector = new NumericDocValueCollector(PartKeyLuceneIndex.START_TIME)
- withNewSearcher(s => s.search(new TermQuery(new Term(PART_ID, partId.toString)), collector))
+ withNewSearcher(s => s.search(new TermQuery(new Term(PART_ID_FIELD, partId.toString)), collector))
collector.singleResult
}
@@ -738,7 +749,7 @@ class PartKeyLuceneIndex(ref: DatasetRef,
}
// dont use BooleanQuery which will hit the 1024 term limit. Instead use TermInSetQuery which is
// more efficient within Lucene
- withNewSearcher(s => s.search(new TermInSetQuery(PART_ID, terms), collector))
+ withNewSearcher(s => s.search(new TermInSetQuery(PART_ID_FIELD, terms), collector))
span.tag(s"num-partitions-to-page", terms.size())
val latency = System.nanoTime - startExecute
span.mark(s"index-startTimes-for-odp-lookup-latency=${latency}ns")
@@ -753,7 +764,7 @@ class PartKeyLuceneIndex(ref: DatasetRef,
*/
def endTimeFromPartId(partId: Int): Long = {
val collector = new NumericDocValueCollector(PartKeyLuceneIndex.END_TIME)
- withNewSearcher(s => s.search(new TermQuery(new Term(PART_ID, partId.toString)), collector))
+ withNewSearcher(s => s.search(new TermQuery(new Term(PART_ID_FIELD, partId.toString)), collector))
collector.singleResult
}
@@ -777,7 +788,6 @@ class PartKeyLuceneIndex(ref: DatasetRef,
coll.numHits
}
-
def foreachPartKeyMatchingFilter(columnFilters: Seq[ColumnFilter],
startTime: Long,
endTime: Long, func: (BytesRef) => Unit): Int = {
@@ -805,7 +815,7 @@ class PartKeyLuceneIndex(ref: DatasetRef,
logger.debug(s"Updating document ${partKeyString(documentId, partKeyOnHeapBytes, partKeyBytesRefOffset)} " +
s"with startTime=$startTime endTime=$endTime into dataset=$ref shard=$shardNum")
val docToAdd = doc.facetsConfig.build(doc.document)
- indexWriter.updateDocument(new Term(PART_ID, partId.toString), docToAdd)
+ indexWriter.updateDocument(new Term(PART_ID_FIELD, partId.toString), docToAdd)
}
/**
@@ -817,12 +827,41 @@ class PartKeyLuceneIndex(ref: DatasetRef,
logger.info(s"Refreshed index searchers to make reads consistent for dataset=$ref shard=$shardNum")
}
+ val regexChars = Array('.', '?', '+', '*', '|', '{', '}', '[', ']', '(', ')', '"', '\\')
+ val regexCharsMinusPipe = (regexChars.toSet - '|').toArray
+
+ // scalastyle:off method.length
private def leafFilter(column: String, filter: Filter): Query = {
+
+ def equalsQuery(value: String): Query = {
+ if (value.nonEmpty) new TermQuery(new Term(column, value))
+ else leafFilter(column, NotEqualsRegex(".+")) // value="" means the label is absent or has an empty value.
+ }
+
filter match {
case EqualsRegex(value) =>
val regex = removeRegexAnchors(value.toString)
- if (regex.nonEmpty) new RegexpQuery(new Term(column, regex), RegExp.NONE)
- else leafFilter(column, NotEqualsRegex(".+")) // value="" means the label is absent or has an empty value.
+ if (regex == "") {
+ // if label=~"" then match empty string or label not present condition too
+ leafFilter(column, NotEqualsRegex(".+"))
+ } else if (regex.replaceAll("\\.\\*", "") == "") {
+ // if label=~".*" then match all docs since promQL matches .* with absent label too
+ new MatchAllDocsQuery
+ } else if (regex.forall(c => !regexChars.contains(c))) {
+ // if all regex special chars absent, then treat like Equals
+ equalsQuery(regex)
+ } else if (regex.forall(c => !regexCharsMinusPipe.contains(c))) {
+ // if pipe is only regex special char present, then convert to IN query
+ new TermInSetQuery(column, regex.split('|').map(t => new BytesRef(t)): _*)
+ } else if (regex.endsWith(".*") && regex.length > 2 &&
+ regex.dropRight(2).forall(c => !regexChars.contains(c))) {
+ // if suffix is .* and no regex special chars present in non-empty prefix, then use prefix query
+ new PrefixQuery(new Term(column, regex.dropRight(2)))
+ } else {
+ // regular non-empty regex query
+ new RegexpQuery(new Term(column, regex), RegExp.NONE)
+ }
+
case NotEqualsRegex(value) =>
val term = new Term(column, removeRegexAnchors(value.toString))
val allDocs = new MatchAllDocsQuery
@@ -830,9 +869,10 @@ class PartKeyLuceneIndex(ref: DatasetRef,
booleanQuery.add(allDocs, Occur.FILTER)
booleanQuery.add(new RegexpQuery(term, RegExp.NONE), Occur.MUST_NOT)
booleanQuery.build()
+
case Equals(value) =>
- if (value.toString.nonEmpty) new TermQuery(new Term(column, value.toString))
- else leafFilter(column, NotEqualsRegex(".+")) // value="" means the label is absent or has an empty value.
+ equalsQuery(value.toString)
+
case NotEquals(value) =>
val str = value.toString
val term = new Term(column, str)
@@ -849,22 +889,18 @@ class PartKeyLuceneIndex(ref: DatasetRef,
booleanQuery.build()
case In(values) =>
- if (values.size < 2)
- throw new IllegalArgumentException("In filter should have atleast 2 values")
- val booleanQuery = new BooleanQuery.Builder
- values.foreach { value =>
- booleanQuery.add(new TermQuery(new Term(column, value.toString)), Occur.SHOULD)
- }
- booleanQuery.build()
+ new TermInSetQuery(column, values.toArray.map(t => new BytesRef(t.toString)): _*)
+
case And(lhs, rhs) =>
val andQuery = new BooleanQuery.Builder
andQuery.add(leafFilter(column, lhs), Occur.FILTER)
andQuery.add(leafFilter(column, rhs), Occur.FILTER)
andQuery.build()
+
case _ => throw new UnsupportedOperationException
}
}
-
+ //scalastyle:on method.length
def partIdsFromFilters(columnFilters: Seq[ColumnFilter],
startTime: Long,
endTime: Long): debox.Buffer[Int] = {
@@ -910,7 +946,7 @@ class PartKeyLuceneIndex(ref: DatasetRef,
val startExecute = System.nanoTime()
val span = Kamon.currentSpan()
- val query: BooleanQuery = colFiltersToQuery(columnFilters, startTime, endTime)
+ val query = colFiltersToQuery(columnFilters, startTime, endTime)
logger.debug(s"Querying dataset=$ref shard=$shardNum partKeyIndex with: $query")
withNewSearcher(s => s.search(query, collector))
val latency = System.nanoTime - startExecute
@@ -927,7 +963,7 @@ class PartKeyLuceneIndex(ref: DatasetRef,
booleanQuery.add(LongPoint.newRangeQuery(START_TIME, 0, endTime), Occur.FILTER)
booleanQuery.add(LongPoint.newRangeQuery(END_TIME, startTime, Long.MaxValue), Occur.FILTER)
val query = booleanQuery.build()
- query
+ new ConstantScoreQuery(query) // disable scoring
}
def partIdFromPartKeySlow(partKeyBase: Any,
@@ -1063,7 +1099,7 @@ class SinglePartIdCollector extends SimpleCollector {
// gets called for each segment
override def doSetNextReader(context: LeafReaderContext): Unit = {
- partIdDv = context.reader().getNumericDocValues(PartKeyLuceneIndex.PART_ID)
+ partIdDv = context.reader().getNumericDocValues(PartKeyLuceneIndex.PART_ID_DV)
}
// gets called for each matching document in current segment
@@ -1101,7 +1137,7 @@ class TopKPartIdsCollector(limit: Int) extends Collector with StrictLogging {
def getLeafCollector(context: LeafReaderContext): LeafCollector = {
logger.trace("New segment inspected:" + context.id)
endTimeDv = DocValues.getNumeric(context.reader, END_TIME)
- partIdDv = DocValues.getNumeric(context.reader, PART_ID)
+ partIdDv = DocValues.getNumeric(context.reader, PART_ID_DV)
new LeafCollector() {
override def setScorer(scorer: Scorable): Unit = {}
@@ -1150,7 +1186,7 @@ class PartIdCollector extends SimpleCollector {
override def doSetNextReader(context: LeafReaderContext): Unit = {
//set the subarray of the numeric values for all documents in the context
- partIdDv = context.reader().getNumericDocValues(PartKeyLuceneIndex.PART_ID)
+ partIdDv = context.reader().getNumericDocValues(PartKeyLuceneIndex.PART_ID_DV)
}
override def collect(doc: Int): Unit = {
@@ -1171,7 +1207,7 @@ class PartIdStartTimeCollector extends SimpleCollector {
override def doSetNextReader(context: LeafReaderContext): Unit = {
//set the subarray of the numeric values for all documents in the context
- partIdDv = context.reader().getNumericDocValues(PartKeyLuceneIndex.PART_ID)
+ partIdDv = context.reader().getNumericDocValues(PartKeyLuceneIndex.PART_ID_DV)
startTimeDv = context.reader().getNumericDocValues(PartKeyLuceneIndex.START_TIME)
}
@@ -1220,7 +1256,7 @@ class ActionCollector(action: (Int, BytesRef) => Unit) extends SimpleCollector {
override def scoreMode(): ScoreMode = ScoreMode.COMPLETE_NO_SCORES
override def doSetNextReader(context: LeafReaderContext): Unit = {
- partIdDv = context.reader().getNumericDocValues(PartKeyLuceneIndex.PART_ID)
+ partIdDv = context.reader().getNumericDocValues(PartKeyLuceneIndex.PART_ID_DV)
partKeyDv = context.reader().getBinaryDocValues(PartKeyLuceneIndex.PART_KEY)
}
diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala
index fd3f018b91..3a178dce69 100644
--- a/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala
+++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala
@@ -11,7 +11,7 @@ import monix.execution.{CancelableFuture, Scheduler}
import monix.reactive.Observable
import org.jctools.maps.NonBlockingHashMapLong
-import filodb.core.{DatasetRef, QueryTimeoutException, Response, Types}
+import filodb.core.{DatasetRef, QueryTimeoutException, Response, Types, Utils}
import filodb.core.downsample.DownsampleConfig
import filodb.core.memstore.ratelimit.{CardinalityRecord, ConfigQuotaSource}
import filodb.core.metadata.Schemas
@@ -33,7 +33,6 @@ extends TimeSeriesStore with StrictLogging {
type Shards = NonBlockingHashMapLong[TimeSeriesShard]
private val datasets = new HashMap[DatasetRef, Shards]
- private val datasetMemFactories = new HashMap[DatasetRef, NativeMemoryManager]
private val quotaSources = new HashMap[DatasetRef, ConfigQuotaSource]
val stats = new ChunkSourceStats
@@ -44,7 +43,18 @@ extends TimeSeriesStore with StrictLogging {
private val partEvictionPolicy = evictionPolicy.getOrElse(
new CompositeEvictionPolicy(ensureTspHeadroomPercent, ensureNmmHeadroomPercent))
- private lazy val ingestionMemory = filodbConfig.getMemorySize("memstore.ingestion-buffer-mem-size").toBytes
+ private[memstore] lazy val ingestionMemory = {
+ if (filodbConfig.getBoolean("memstore.memory-alloc.automatic-alloc-enabled")) {
+ val availableMemoryBytes: Long = Utils.calculateAvailableOffHeapMemory(filodbConfig)
+ val nativeMemoryManagerPercent = filodbConfig.getDouble("memstore.memory-alloc.native-memory-manager-percent")
+ val blockMemoryManagerPercent = filodbConfig.getDouble("memstore.memory-alloc.block-memory-manager-percent")
+ val lucenePercent = filodbConfig.getDouble("memstore.memory-alloc.lucene-memory-percent")
+ require(Math.abs(nativeMemoryManagerPercent + blockMemoryManagerPercent + lucenePercent - 100) < 0.001,
+ s"Configured Block($nativeMemoryManagerPercent), Native($blockMemoryManagerPercent) " +
+ s"and Lucene($lucenePercent) memory percents don't sum to 100.0")
+ (availableMemoryBytes * nativeMemoryManagerPercent / 100).toLong
+ } else filodbConfig.getMemorySize("memstore.ingestion-buffer-mem-size").toBytes
+ }
private[this] lazy val ingestionMemFactory: NativeMemoryManager = {
logger.info(s"Allocating $ingestionMemory bytes for WriteBufferPool/PartitionKeys")
@@ -67,7 +77,7 @@ extends TimeSeriesStore with StrictLogging {
}
// TODO: Change the API to return Unit Or ShardAlreadySetup, instead of throwing. Make idempotent.
- def setup(ref: DatasetRef, schemas: Schemas, shard: Int, storeConf: StoreConfig,
+ def setup(ref: DatasetRef, schemas: Schemas, shard: Int, storeConf: StoreConfig, numShards: Int,
downsample: DownsampleConfig = DownsampleConfig.disabled): Unit = synchronized {
val shards = datasets.getOrElseUpdate(ref, new NonBlockingHashMapLong[TimeSeriesShard](32, false))
val quotaSource = quotaSources.getOrElseUpdate(ref,
@@ -75,8 +85,8 @@ extends TimeSeriesStore with StrictLogging {
if (shards.containsKey(shard)) {
throw ShardAlreadySetup(ref, shard)
} else {
- val tsdb = new OnDemandPagingShard(ref, schemas, storeConf, quotaSource, shard, ingestionMemFactory, store,
- metastore, partEvictionPolicy, filodbConfig)
+ val tsdb = new OnDemandPagingShard(ref, schemas, storeConf, numShards, quotaSource, shard,
+ ingestionMemFactory, store, metastore, partEvictionPolicy, filodbConfig)
shards.put(shard, tsdb)
}
}
diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala
index 2c5dc84bad..4293fef734 100644
--- a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala
+++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala
@@ -258,6 +258,7 @@ case class TimeSeriesShardInfo(shardNum: Int,
class TimeSeriesShard(val ref: DatasetRef,
val schemas: Schemas,
val storeConfig: StoreConfig,
+ numShards: Int,
quotaSource: QuotaSource,
val shardNum: Int,
val bufferMemoryManager: NativeMemoryManager,
@@ -365,7 +366,25 @@ class TimeSeriesShard(val ref: DatasetRef,
val ingestSched = Scheduler.singleThread(s"$IngestSchedName-$ref-$shardNum",
reporter = UncaughtExceptionReporter(logger.error("Uncaught Exception in TimeSeriesShard.ingestSched", _)))
- private val blockMemorySize = storeConfig.shardMemSize
+ private[memstore] val blockMemorySize = {
+ val size = if (filodbConfig.getBoolean("memstore.memory-alloc.automatic-alloc-enabled")) {
+ val numNodes = filodbConfig.getInt("min-num-nodes-in-cluster")
+ val availableMemoryBytes: Long = Utils.calculateAvailableOffHeapMemory(filodbConfig)
+ val blockMemoryManagerPercent = filodbConfig.getDouble("memstore.memory-alloc.block-memory-manager-percent")
+ val blockMemForDatasetPercent = storeConfig.shardMemPercent // fraction of block memory for this dataset
+ val numShardsPerNode = Math.ceil(numShards / numNodes.toDouble)
+ logger.info(s"Calculating Block memory size with automatic allocation strategy. " +
+ s"Dataset dataset=$ref has blockMemForDatasetPercent=$blockMemForDatasetPercent " +
+ s"numShardsPerNode=$numShardsPerNode")
+ (availableMemoryBytes * blockMemoryManagerPercent *
+ blockMemForDatasetPercent / 100 / 100 / numShardsPerNode).toLong
+ } else {
+ storeConfig.shardMemSize
+ }
+ logger.info(s"Block Memory for dataset=$ref shard=$shardNum bytesAllocated=$size")
+ size
+ }
+
protected val numGroups = storeConfig.groupsPerShard
private val chunkRetentionHours = (storeConfig.diskTTLSeconds / 3600).toInt
private[memstore] val pagingEnabled = storeConfig.demandPagingEnabled
@@ -915,14 +934,8 @@ class TimeSeriesShard(val ref: DatasetRef,
markPartAsNotIngesting(p, odp = false)
if (storeConfig.meteringEnabled) {
val shardKey = p.schema.partKeySchema.colValues(p.partKeyBase, p.partKeyOffset,
- p.schema.options.shardKeyColumns)
- val newCard = cardTracker.modifyCount(shardKey, 0, -1)
- // TODO remove temporary debugging since we are seeing some negative counts
- if (newCard.exists(_.value.activeTsCount < 0) && p.partID % 100 < 5)
- // log for 5% of the cases to reduce log volume
- logger.error(s"For some reason, activeTs count negative when updating card for " +
- s"partKey: ${p.stringPartition} newCard: $newCard oldActivelyIngestingSize=$oldActivelyIngestingSize " +
- s"newActivelyIngestingSize=${activelyIngesting.size}")
+ p.schema.options.shardKeyColumns)
+ cardTracker.modifyCount(shardKey, 0, -1)
}
}
}
@@ -1756,15 +1769,18 @@ class TimeSeriesShard(val ref: DatasetRef,
startTime: Long,
querySession: QuerySession,
limit: Int): Iterator[ZeroCopyUTF8String] = {
- if (indexFacetingEnabledAllLabels ||
+ val metricShardKeys = schemas.part.options.shardKeyColumns
+ val metricGroupBy = deploymentPartitionName +: clusterType +: shardKeyValuesFromFilter(metricShardKeys, filters)
+ val startNs = Utils.currentThreadCpuTimeNanos
+ val res = if (indexFacetingEnabledAllLabels ||
(indexFacetingEnabledShardKeyLabels && schemas.part.options.shardKeyColumns.contains(label))) {
partKeyIndex.labelValuesEfficient(filters, startTime, endTime, label, limit).iterator.map(_.utf8)
} else {
- val metricShardKeys = schemas.part.options.shardKeyColumns
- val metricGroupBy = deploymentPartitionName +: clusterType +: shardKeyValuesFromFilter(metricShardKeys, filters)
SingleLabelValuesResultIterator(partKeyIndex.partIdsFromFilters(filters, startTime, endTime),
label, querySession, metricGroupBy, limit)
}
+ querySession.queryStats.getCpuNanosCounter(metricGroupBy).addAndGet(startNs - Utils.currentThreadCpuTimeNanos)
+ res
}
/**
@@ -1921,9 +1937,11 @@ class TimeSeriesShard(val ref: DatasetRef,
val chunksReadCounter = querySession.queryStats.getDataBytesScannedCounter(metricGroupBy)
// No matter if there are filters or not, need to run things through Lucene so we can discover potential
// TSPartitions to read back from disk
+ val startNs = Utils.currentThreadCpuTimeNanos
val matches = partKeyIndex.partIdsFromFilters(filters, chunkMethod.startTime, chunkMethod.endTime)
shardStats.queryTimeRangeMins.record((chunkMethod.endTime - chunkMethod.startTime) / 60000 )
querySession.queryStats.getTimeSeriesScannedCounter(metricGroupBy).addAndGet(matches.length)
+ querySession.queryStats.getCpuNanosCounter(metricGroupBy).addAndGet(Utils.currentThreadCpuTimeNanos - startNs)
Kamon.currentSpan().tag(s"num-partitions-from-index-$shardNum", matches.length)
// first find out which partitions are being queried for data not in memory
diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesStore.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesStore.scala
index dfa0f1f0cb..c84a7298c9 100644
--- a/core/src/main/scala/filodb.core/memstore/TimeSeriesStore.scala
+++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesStore.scala
@@ -75,7 +75,7 @@ trait TimeSeriesStore extends ChunkSource {
* @param downsampleConfig configuration for downsampling operation. By default it is disabled.
*/
def setup(ref: DatasetRef, schemas: Schemas, shard: Int,
- storeConf: StoreConfig,
+ storeConf: StoreConfig, numShards: Int,
downsampleConfig: DownsampleConfig = DownsampleConfig.disabled): Unit
/**
diff --git a/core/src/main/scala/filodb.core/memstore/ratelimit/CardinalityManager.scala b/core/src/main/scala/filodb.core/memstore/ratelimit/CardinalityManager.scala
index 92ea92bb19..dfe1d6897c 100644
--- a/core/src/main/scala/filodb.core/memstore/ratelimit/CardinalityManager.scala
+++ b/core/src/main/scala/filodb.core/memstore/ratelimit/CardinalityManager.scala
@@ -32,6 +32,15 @@ class CardinalityManager(datasetRef: DatasetRef,
// physical resources for duplicate calculation
private var isCardTriggered: Boolean = false
+ // number of partKeys to aggregate in memory at any given time, before we write the cardinality information to
+ // rocksDB. This is done to reduce the time taken to calculate cardinality of partitions with high number of
+ // TS by reducing the number writes issued to RocksDB ( From our profiling, high number of RocksDB writes lead to
+ // performance bottlenecks)
+ // For cardFlushCount = 500,000, we conservatively expect to consume - 128 bytes * 500,000 = ~64 MB
+ private val cardFlushCount: Option[Int] =
+ if (filodbConfig.hasPath("card-flush-count")) {
+ Some(filodbConfig.getInt("card-flush-count"))
+ } else None
/**
* `dataset-configs` is an string array where each string is a file path for a dataset config. This function reads
@@ -132,9 +141,6 @@ class CardinalityManager(datasetRef: DatasetRef,
/**
* Triggers cardinalityCount if metering is enabled and the required criteria matches.
- * It creates a new instance of CardinalityTracker and uses the PartKeyLuceneIndex to calculate cardinality count
- * and store data in a local CardinalityStore. We then close the previous instance of CardinalityTracker and switch
- * it with the new one we created in this call.
* @param indexRefreshCount The number of time the indexRefresh has already happened. This is used in the logic of
* shouldTriggerCardinalityCount
*/
@@ -143,34 +149,7 @@ class CardinalityManager(datasetRef: DatasetRef,
try {
if (shouldTriggerCardinalityCount(shardNum, numShardsPerNode, indexRefreshCount)) {
isCardTriggered = true
- val newCardTracker = getNewCardTracker()
- var cardCalculationComplete = false
- try {
- partKeyIndex.calculateCardinality(partSchema, newCardTracker)
- cardCalculationComplete = true
- } catch {
- case ex: Exception =>
- logger.error(s"[CardinalityManager]Error while calculating cardinality using" +
- s" PartKeyLuceneIndex! shardNum=$shardNum indexRefreshCount=$indexRefreshCount", ex)
- // cleanup resources used by the newCardTracker tracker to avoid leaking of resources
- newCardTracker.close()
- }
- if (cardCalculationComplete) {
- try {
- // close the cardinality store and release the physical resources of the current cardinality store
- close()
- cardTracker = Some(newCardTracker)
- logger.info(s"[CardinalityManager] Triggered cardinality count successfully for" +
- s" shardNum=$shardNum indexRefreshCount=$indexRefreshCount")
- } catch {
- case ex: Exception =>
- // Very unlikely scenario, but can happen if the disk call fails.
- logger.error(s"[CardinalityManager]Error closing card tracker! shardNum=$shardNum", ex)
- // setting cardTracker to None in this case, since the exception happened on the close. We
- // can't rely on the card store. The next trigger should re-build the card store and card tracker
- cardTracker = None
- }
- }
+ createNewCardinalityTrackerAndCalculate(indexRefreshCount)
isCardTriggered = false
}
else {
@@ -189,6 +168,50 @@ class CardinalityManager(datasetRef: DatasetRef,
}
}
+ /**
+ * Creates a new instance of CardinalityTracker and uses the PartKeyLuceneIndex to calculate cardinality count
+ * and store data in a local CardinalityStore. We then close the previous instance of CardinalityTracker and switch
+ * it with the new one we created in this call.
+ *
+ * @param indexRefreshCount The number of time the indexRefresh has already happened. This is used in the logic of
+ * shouldTriggerCardinalityCount
+ */
+ private def createNewCardinalityTrackerAndCalculate(indexRefreshCount: Int): Unit = {
+ val newCardTracker = getNewCardTracker()
+ var cardCalculationComplete = false
+ val startTimeMs = System.currentTimeMillis()
+ try {
+ logger.info(s"[CardinalityManager]Triggering cardinality count for shardNum=$shardNum " +
+ s"indexRefreshCount=$indexRefreshCount")
+ partKeyIndex.calculateCardinality(partSchema, newCardTracker)
+ cardCalculationComplete = true
+ } catch {
+ case ex: Exception =>
+ logger.error(s"[CardinalityManager]Error while calculating cardinality using" +
+ s" PartKeyLuceneIndex! shardNum=$shardNum indexRefreshCount=$indexRefreshCount", ex)
+ // cleanup resources used by the newCardTracker tracker to avoid leaking of resources
+ newCardTracker.close()
+ }
+ if (cardCalculationComplete) {
+ try {
+ // close and release the physical resources of the outdated/previous cardinality store
+ close()
+ // reassign the cardTracker with the newly created CardinalityTracker object
+ cardTracker = Some(newCardTracker)
+ val timeTakenInSeconds = ((System.currentTimeMillis() - startTimeMs)/1000.0)
+ logger.info(s"[CardinalityManager] Triggered cardinality count successfully for" +
+ s" shardNum=$shardNum indexRefreshCount=$indexRefreshCount timeTakenInSeconds=$timeTakenInSeconds")
+ } catch {
+ case ex: Exception =>
+ // Very unlikely scenario, but can happen if the disk call fails.
+ logger.error(s"[CardinalityManager]Error closing card tracker! shardNum=$shardNum", ex)
+ // setting cardTracker to None in this case, since the exception happened on the close. We
+ // can't rely on the card store. The next trigger should re-build the card store and card tracker
+ cardTracker = None
+ }
+ }
+ }
+
/**
* Helper method to create a CardinalityTracker instance
*
@@ -197,7 +220,9 @@ class CardinalityManager(datasetRef: DatasetRef,
private def getNewCardTracker(): CardinalityTracker = {
val cardStore = new RocksDbCardinalityStore(datasetRef, shardNum)
val defaultQuota = quotaSource.getDefaults(datasetRef)
- val tracker = new CardinalityTracker(datasetRef, shardNum, shardKeyLen, defaultQuota, cardStore)
+ logger.info(s"[CardinalityManager] Creating new CardinalityTracker with flushCount=$cardFlushCount")
+ val tracker = new CardinalityTracker(datasetRef, shardNum, shardKeyLen, defaultQuota, cardStore,
+ flushCount = cardFlushCount)
quotaSource.getQuotas(datasetRef).foreach { q =>
tracker.setQuota(q.shardKeyPrefix, q.quota)
}
diff --git a/core/src/main/scala/filodb.core/query/PartitionTimeRangeReader.scala b/core/src/main/scala/filodb.core/query/PartitionTimeRangeReader.scala
index a86ad2a56a..e48711b368 100644
--- a/core/src/main/scala/filodb.core/query/PartitionTimeRangeReader.scala
+++ b/core/src/main/scala/filodb.core/query/PartitionTimeRangeReader.scala
@@ -47,8 +47,8 @@ final class PartitionTimeRangeReader(part: ReadablePartition,
s"""${info.debugString} """
}
message += "]"
- logger.error(s"message ${message}")
- throw new IllegalArgumentException(message)
+ logger.error(s"message $message")
+ throw new IllegalArgumentException("requirement length > 0 failed.")
}
}
}
diff --git a/core/src/main/scala/filodb.core/query/RangeVector.scala b/core/src/main/scala/filodb.core/query/RangeVector.scala
index bba7fdd240..b338accf1c 100644
--- a/core/src/main/scala/filodb.core/query/RangeVector.scala
+++ b/core/src/main/scala/filodb.core/query/RangeVector.scala
@@ -213,8 +213,8 @@ sealed trait ScalarSingleValue extends ScalarRangeVector {
else it
}
- // Negligible bytes sent over-the-wire.
- override def estimateSerializedRowBytes: Long = 0
+ // Negligible bytes sent over-the-wire. Don't bother calculating accurately.
+ override def estimateSerializedRowBytes: Long = SerializableRangeVector.SizeOfDouble
}
/**
@@ -399,7 +399,10 @@ final class SerializedRangeVector(val key: RangeVectorKey,
} else it
}
- override def estimateSerializedRowBytes: Long = containers.map(_.numBytes).sum
+ override def estimateSerializedRowBytes: Long =
+ containers.toIterator.flatMap(_.iterate(schema))
+ .slice(startRecordNo, startRecordNo + numRowsSerialized)
+ .foldLeft(0)(_ + _.recordLength)
def containersIterator : Iterator[RecordContainer] = containers.toIterator
@@ -462,7 +465,7 @@ object SerializedRangeVector extends StrictLogging {
val nextRow = rows.next()
// Don't encode empty / NaN data over the wire
if (!canRemoveEmptyRows(rv.outputRange, schema) ||
- schema.columns(1).colType == DoubleColumn && !nextRow.getDouble(1).isNaN ||
+ schema.columns(1).colType == DoubleColumn && !java.lang.Double.isNaN(nextRow.getDouble(1)) ||
schema.columns(1).colType == HistogramColumn && !nextRow.getHistogram(1).isEmpty) {
numRows += 1
builder.addFromReader(nextRow, schema, 0)
@@ -487,11 +490,7 @@ object SerializedRangeVector extends StrictLogging {
case None => builder.allContainers.toList
case Some(firstContainer) => builder.allContainers.dropWhile(_ != firstContainer)
}
- val srv = new SerializedRangeVector(rv.key, numRows, containers, schema, startRecordNo, rv.outputRange)
- val resultSize = srv.estimatedSerializedBytes
- SerializedRangeVector.queryResultBytes.record(resultSize)
- queryStats.getResultBytesCounter(Nil).addAndGet(resultSize)
- srv
+ new SerializedRangeVector(rv.key, numRows, containers, schema, startRecordNo, rv.outputRange)
} finally {
queryStats.getCpuNanosCounter(Nil).addAndGet(Utils.currentThreadCpuTimeNanos - startNs)
}
diff --git a/core/src/main/scala/filodb.core/store/IngestionConfig.scala b/core/src/main/scala/filodb.core/store/IngestionConfig.scala
index 1d530c133c..06802c329e 100644
--- a/core/src/main/scala/filodb.core/store/IngestionConfig.scala
+++ b/core/src/main/scala/filodb.core/store/IngestionConfig.scala
@@ -17,6 +17,7 @@ final case class StoreConfig(flushInterval: FiniteDuration,
maxBlobBufferSize: Int,
// Number of bytes to allocate to chunk storage in each shard
shardMemSize: Long,
+ shardMemPercent: Double,
maxBufferPoolSize: Int,
groupsPerShard: Int,
numPagesPerBlock: Int,
@@ -45,6 +46,7 @@ final case class StoreConfig(flushInterval: FiniteDuration,
"max-chunks-size" -> maxChunksSize,
"max-blob-buffer-size" -> maxBlobBufferSize,
"shard-mem-size" -> shardMemSize,
+ "shard-mem-percent" -> shardMemPercent,
"max-buffer-pool-size" -> maxBufferPoolSize,
"groups-per-shard" -> groupsPerShard,
"max-chunk-time" -> (maxChunkTime.toSeconds + "s"),
@@ -81,6 +83,7 @@ object StoreConfig {
|max-blob-buffer-size = 15000
|max-buffer-pool-size = 10000
|groups-per-shard = 60
+ |shard-mem-percent = 100 # assume only one dataset per node by default
|num-block-pages = 100
|failure-retries = 3
|retry-delay = 15 seconds
@@ -119,6 +122,7 @@ object StoreConfig {
config.getInt("max-chunks-size"),
config.getInt("max-blob-buffer-size"),
config.getMemorySize("shard-mem-size").toBytes,
+ config.getDouble("shard-mem-percent"),
config.getInt("max-buffer-pool-size"),
config.getInt("groups-per-shard"),
config.getInt("num-block-pages"),
diff --git a/core/src/test/resources/application_test.conf b/core/src/test/resources/application_test.conf
index 52e14fbebf..dae84e3bdf 100644
--- a/core/src/test/resources/application_test.conf
+++ b/core/src/test/resources/application_test.conf
@@ -88,6 +88,15 @@ filodb {
spark.dataset-ops-timeout = 15s
memstore {
+ memory-alloc {
+ automatic-alloc-enabled = false
+ available-memory-bytes = 1GB
+ os-memory-needs = 500MB
+ lucene-memory-percent = 20
+ native-memory-manager-percent = 20
+ block-memory-manager-percent = 60
+ }
+
flush-task-parallelism = 1
ensure-block-memory-headroom-percent = 5
ensure-tsp-count-headroom-percent = 5
@@ -97,6 +106,7 @@ filodb {
track-queries-holding-eviction-lock = false
index-faceting-enabled-shard-key-labels = true
index-faceting-enabled-for-all-labels = true
+
}
tasks {
diff --git a/core/src/test/scala/filodb.core/memstore/DemandPagedChunkStoreSpec.scala b/core/src/test/scala/filodb.core/memstore/DemandPagedChunkStoreSpec.scala
index ad13ef95ac..defaf44269 100644
--- a/core/src/test/scala/filodb.core/memstore/DemandPagedChunkStoreSpec.scala
+++ b/core/src/test/scala/filodb.core/memstore/DemandPagedChunkStoreSpec.scala
@@ -26,7 +26,7 @@ class DemandPagedChunkStoreSpec extends AnyFunSpec with AsyncTest {
|shard-mem-size = 200MB""".stripMargin)
.withFallback(TestData.sourceConf.getConfig("store"))
- memStore.setup(dataset1.ref, Schemas(schema1), 0, StoreConfig(sourceConf))
+ memStore.setup(dataset1.ref, Schemas(schema1), 0, StoreConfig(sourceConf), 1)
val onDemandPartMaker = memStore.getShardE(dataset1.ref, 0).partitionMaker
after {
diff --git a/core/src/test/scala/filodb.core/memstore/PartKeyLuceneIndexSpec.scala b/core/src/test/scala/filodb.core/memstore/PartKeyLuceneIndexSpec.scala
index 6eb71613b2..e776700b3f 100644
--- a/core/src/test/scala/filodb.core/memstore/PartKeyLuceneIndexSpec.scala
+++ b/core/src/test/scala/filodb.core/memstore/PartKeyLuceneIndexSpec.scala
@@ -282,7 +282,6 @@ class PartKeyLuceneIndexSpec extends AnyFunSpec with Matchers with BeforeAndAfte
}
-
it("should add part keys and removePartKeys (without partIds) correctly for more than 1024 keys with del count") {
// Identical to test
// it("should add part keys and fetch partIdsEndedBefore and removePartKeys correctly for more than 1024 keys")
@@ -345,7 +344,6 @@ class PartKeyLuceneIndexSpec extends AnyFunSpec with Matchers with BeforeAndAfte
// it is no longer required to have partIds in the index on non unit test setup
}
-
it("should update part keys with endtime and parse filters correctly") {
val start = System.currentTimeMillis()
// Add the first ten keys and row numbers
@@ -447,6 +445,32 @@ class PartKeyLuceneIndexSpec extends AnyFunSpec with Matchers with BeforeAndAfte
partNums2 shouldEqual debox.Buffer.empty[Int]
}
+ it("should be able to convert pipe regex to TermInSetQuery") {
+ // Add the first ten keys and row numbers
+ partKeyFromRecords(dataset6, records(dataset6, readers.take(99)), Some(partBuilder))
+ .zipWithIndex.foreach { case (addr, i) =>
+ keyIndex.addPartKey(partKeyOnHeap(dataset6.partKeySchema, ZeroPointer, addr), i, System.currentTimeMillis())()
+ }
+ keyIndex.refreshReadersBlocking()
+
+ val filters1 = Seq(ColumnFilter("Actor2Code", EqualsRegex("GOV|KHM|LAB|MED".utf8)))
+ val partNums1 = keyIndex.partIdsFromFilters(filters1, 0, Long.MaxValue)
+ partNums1 shouldEqual debox.Buffer(7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 22, 23, 24, 25, 26, 28, 29, 73, 81, 90)
+ }
+
+ it("should be able to convert prefix regex to PrefixQuery") {
+ // Add the first ten keys and row numbers
+ partKeyFromRecords(dataset6, records(dataset6, readers.take(99)), Some(partBuilder))
+ .zipWithIndex.foreach { case (addr, i) =>
+ keyIndex.addPartKey(partKeyOnHeap(dataset6.partKeySchema, ZeroPointer, addr), i, System.currentTimeMillis())()
+ }
+ keyIndex.refreshReadersBlocking()
+
+ val filters1 = Seq(ColumnFilter("Actor2Name", EqualsRegex("C.*".utf8)))
+ val partNums1 = keyIndex.partIdsFromFilters(filters1, 0, Long.MaxValue)
+ partNums1 shouldEqual debox.Buffer(3, 12, 22, 23, 24, 31, 59, 60, 66, 69, 72, 78, 79, 80, 88, 89)
+ }
+
it("should ignore unsupported columns and return empty filter") {
val index2 = new PartKeyLuceneIndex(dataset1.ref, dataset1.schema.partition, true, true, 0, 1.hour.toMillis)
partKeyFromRecords(dataset1, records(dataset1, readers.take(10))).zipWithIndex.foreach { case (addr, i) =>
@@ -1008,4 +1032,61 @@ class PartKeyLuceneIndexSpec extends AnyFunSpec with Matchers with BeforeAndAfte
// close CardinalityTracker to avoid leaking of resources
cardTracker.close()
}
+
+ it("should match records without label when .* is provided on a non existent label") {
+
+ val pkrs = partKeyFromRecords(dataset6, records(dataset6, readers.take(10)), Some(partBuilder))
+ .zipWithIndex.map { case (addr, i) =>
+ val pk = partKeyOnHeap(dataset6.schema.partKeySchema, ZeroPointer, addr)
+ keyIndex.addPartKey(pk, i, i, i + 10)()
+ PartKeyLuceneIndexRecord(pk, i, i + 10)
+ }
+ keyIndex.refreshReadersBlocking()
+
+
+ // Query with just the existing Label name
+ val filter1 = ColumnFilter("Actor2Code", Equals("GOV".utf8))
+ val result1 = keyIndex.partKeyRecordsFromFilters(Seq(filter1), 0, Long.MaxValue)
+ val expected1 = Seq(pkrs(7), pkrs(8), pkrs(9))
+
+ result1.map(_.partKey.toSeq) shouldEqual expected1.map(_.partKey.toSeq)
+ result1.map(p => (p.startTime, p.endTime)) shouldEqual expected1.map(p => (p.startTime, p.endTime))
+
+ // Query with non existent label name with an empty regex
+ val filter2 = ColumnFilter("dummy", EqualsRegex(".*".utf8))
+ val filter3 = ColumnFilter("Actor2Code", Equals("GOV".utf8))
+ val result2 = keyIndex.partKeyRecordsFromFilters(Seq(filter2, filter3), 0, Long.MaxValue)
+ val expected2 = Seq(pkrs(7), pkrs(8), pkrs(9))
+
+ result2.map(_.partKey.toSeq) shouldEqual expected2.map(_.partKey.toSeq)
+ result2.map(p => (p.startTime, p.endTime)) shouldEqual expected2.map(p => (p.startTime, p.endTime))
+
+ // Query with non existent label name with an regex matching at least 1 character
+ val filter4 = ColumnFilter("dummy", EqualsRegex(".+".utf8))
+ val filter5 = ColumnFilter("Actor2Code", Equals("GOV".utf8))
+ val result3 = keyIndex.partKeyRecordsFromFilters(Seq(filter4, filter5), 0, Long.MaxValue)
+ result3 shouldEqual Seq()
+
+ // Query with non existent label name with an empty regex
+ val filter6 = ColumnFilter("dummy", EqualsRegex("".utf8))
+ val filter7 = ColumnFilter("Actor2Code", Equals("GOV".utf8))
+ val result4 = keyIndex.partKeyRecordsFromFilters(Seq(filter6, filter7), 0, Long.MaxValue)
+ val expected4 = Seq(pkrs(7), pkrs(8), pkrs(9))
+ result4.map(_.partKey.toSeq) shouldEqual expected4.map(_.partKey.toSeq)
+ result4.map(p => (p.startTime, p.endTime)) shouldEqual expected4.map(p => (p.startTime, p.endTime))
+
+ // Query with non existent label name with an empty equals
+ val filter8 = ColumnFilter("dummy", Equals("".utf8))
+ val filter9 = ColumnFilter("Actor2Code", Equals("GOV".utf8))
+ val result5 = keyIndex.partKeyRecordsFromFilters(Seq(filter8, filter9), 0, Long.MaxValue)
+ val expected5 = Seq(pkrs(7), pkrs(8), pkrs(9))
+ result5.map(_.partKey.toSeq) shouldEqual expected5.map(_.partKey.toSeq)
+ result5.map(p => (p.startTime, p.endTime)) shouldEqual expected5.map(p => (p.startTime, p.endTime))
+
+
+ val filter10 = ColumnFilter("Actor2Code", EqualsRegex(".*".utf8))
+ val result10= keyIndex.partKeyRecordsFromFilters(Seq(filter10), 0, Long.MaxValue)
+ result10.map(_.partKey.toSeq) shouldEqual pkrs.map(_.partKey.toSeq)
+ result10.map(p => (p.startTime, p.endTime)) shouldEqual pkrs.map(p => (p.startTime, p.endTime))
+ }
}
\ No newline at end of file
diff --git a/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreForMetadataSpec.scala b/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreForMetadataSpec.scala
index 8a3797557b..f1500e0c1e 100644
--- a/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreForMetadataSpec.scala
+++ b/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreForMetadataSpec.scala
@@ -39,7 +39,7 @@ class TimeSeriesMemStoreForMetadataSpec extends AnyFunSpec with Matchers with Sc
val container = createRecordContainer(0, 10)
override def beforeAll(): Unit = {
- memStore.setup(timeseriesDataset.ref, Schemas(timeseriesSchema), 0, TestData.storeConf)
+ memStore.setup(timeseriesDataset.ref, Schemas(timeseriesSchema), 0, TestData.storeConf, 1)
memStore.ingest(timeseriesDataset.ref, 0, SomeData(container, 0))
memStore.refreshIndexForTesting(timeseriesDataset.ref)
}
diff --git a/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala b/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala
index b199600fbd..1654c81c51 100644
--- a/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala
+++ b/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala
@@ -46,9 +46,9 @@ class TimeSeriesMemStoreSpec extends AnyFunSpec with Matchers with BeforeAndAfte
val schemas1 = Schemas(schema1)
it("should detect duplicate setup") {
- memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf)
+ memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf, 1)
try {
- memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf)
+ memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf, 1)
fail()
} catch {
case e: ShardAlreadySetup => { } // expected
@@ -58,7 +58,7 @@ class TimeSeriesMemStoreSpec extends AnyFunSpec with Matchers with BeforeAndAfte
// Look mama! Real-time time series ingestion and querying across multiple partitions!
it("should ingest into multiple series and be able to query across all partitions in real time") {
- memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf)
+ memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf, 1)
val rawData = multiSeriesData().take(20)
val data = records(dataset1, rawData) // 2 records per series x 10 series
memStore.ingest(dataset1.ref, 0, data)
@@ -81,7 +81,7 @@ class TimeSeriesMemStoreSpec extends AnyFunSpec with Matchers with BeforeAndAfte
}
it("should ingest into multiple series and query across partitions") {
- memStore.setup(dataset1.ref, schemas1, 1, TestData.storeConf)
+ memStore.setup(dataset1.ref, schemas1, 1, TestData.storeConf, 1)
val data = records(dataset1, linearMultiSeries().take(20)) // 2 records per series x 10 series
memStore.ingest(dataset1.ref, 1, data)
@@ -97,7 +97,7 @@ class TimeSeriesMemStoreSpec extends AnyFunSpec with Matchers with BeforeAndAfte
}
it("should ingest map/tags column as partition key and aggregate") {
- memStore.setup(dataset2.ref, schemas2h, 0, TestData.storeConf)
+ memStore.setup(dataset2.ref, schemas2h, 0, TestData.storeConf, 1)
val data = records(dataset2, withMap(linearMultiSeries().take(20))) // 2 records per series x 10 series
memStore.ingest(dataset2.ref, 0, data)
@@ -109,7 +109,7 @@ class TimeSeriesMemStoreSpec extends AnyFunSpec with Matchers with BeforeAndAfte
}
it("should ingest histograms and read them back properly") {
- memStore.setup(histDataset.ref, schemas2h, 0, TestData.storeConf)
+ memStore.setup(histDataset.ref, schemas2h, 0, TestData.storeConf, 1)
val data = linearHistSeries().take(40)
memStore.ingest(histDataset.ref, 0, records(histDataset, data))
memStore.refreshIndexForTesting(histDataset.ref)
@@ -137,7 +137,7 @@ class TimeSeriesMemStoreSpec extends AnyFunSpec with Matchers with BeforeAndAfte
it("should ingest multiple schemas simultaneously into one shard") {
val ref = dataset2.ref
- memStore.setup(ref, schemas2h, 0, TestData.storeConf)
+ memStore.setup(ref, schemas2h, 0, TestData.storeConf, 1)
val data = linearHistSeries().take(40)
memStore.ingest(ref, 0, records(histDataset, data))
val data2 = records(dataset2, withMap(linearMultiSeries()).take(30)) // 3 records per series x 10 series
@@ -151,14 +151,14 @@ class TimeSeriesMemStoreSpec extends AnyFunSpec with Matchers with BeforeAndAfte
}
it("should be able to handle nonexistent partition keys") {
- memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf)
+ memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf, 1)
val q = memStore.scanRows(dataset1, Seq(1), SinglePartitionScan(Array[Byte]()))
q.toBuffer.length should equal (0)
}
it("should ingest into multiple series and be able to query on one partition in real time") {
- memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf)
+ memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf, 1)
val data = multiSeriesData().take(20) // 2 records per series x 10 series
memStore.ingest(dataset1.ref, 0, records(dataset1, data))
@@ -174,7 +174,7 @@ class TimeSeriesMemStoreSpec extends AnyFunSpec with Matchers with BeforeAndAfte
}
it("should query on multiple partitions using filters") {
- memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf)
+ memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf, 1)
val data = records(dataset1, linearMultiSeries().take(20)) // 2 records per series x 10 series
memStore.ingest(dataset1.ref, 0, data)
memStore.refreshIndexForTesting(dataset1.ref)
@@ -186,8 +186,8 @@ class TimeSeriesMemStoreSpec extends AnyFunSpec with Matchers with BeforeAndAfte
}
it("should ingest into multiple shards, getScanSplits, query, get index info from shards") {
- memStore.setup(dataset2.ref, schemas2h, 0, TestData.storeConf)
- memStore.setup(dataset2.ref, schemas2h, 1, TestData.storeConf)
+ memStore.setup(dataset2.ref, schemas2h, 0, TestData.storeConf, 2)
+ memStore.setup(dataset2.ref, schemas2h, 1, TestData.storeConf, 2)
val data = records(dataset2, withMap(linearMultiSeries()).take(20)) // 2 records per series x 10 series
memStore.ingest(dataset2.ref, 0, data)
val data2 = records(dataset2, withMap(linearMultiSeries(200000L, 6), 6).take(20)) // 5 series only
@@ -214,7 +214,7 @@ class TimeSeriesMemStoreSpec extends AnyFunSpec with Matchers with BeforeAndAfte
}
it("should handle errors from ingestStream") {
- memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf)
+ memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf, 1)
val errStream = Observable.fromIterable(groupedRecords(dataset1, linearMultiSeries()))
.endWithError(new NumberFormatException)
val fut = memStore.startIngestion(dataset1.ref, 0, errStream, s)
@@ -224,7 +224,7 @@ class TimeSeriesMemStoreSpec extends AnyFunSpec with Matchers with BeforeAndAfte
}
it("should ingestStream and flush on interval") {
- memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf)
+ memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf, 1)
val initChunksWritten = chunksetsWritten
val stream = Observable.fromIterable(groupedRecords(dataset1, linearMultiSeries()))
@@ -245,7 +245,7 @@ class TimeSeriesMemStoreSpec extends AnyFunSpec with Matchers with BeforeAndAfte
it("should flush dirty part keys during start-ingestion, end-ingestion and re-ingestion") {
memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf.copy(groupsPerShard = 2,
diskTTLSeconds = 1.hour.toSeconds.toInt,
- flushInterval = 10.minutes))
+ flushInterval = 10.minutes), 1)
Thread sleep 1000
val numPartKeysWritten = partKeysWritten
val tsShard = memStore.asInstanceOf[TimeSeriesMemStore].getShard(dataset1.ref, 0).get
@@ -302,6 +302,54 @@ class TimeSeriesMemStoreSpec extends AnyFunSpec with Matchers with BeforeAndAfte
10.until(20).foreach {i => tsShard.activelyIngesting(i) shouldEqual false}
}
+ it("should configure memory automatically when enabled") {
+ val colStore = new NullColumnStore()
+
+ val config1 = ConfigFactory.parseString(
+ """
+ |min-num-nodes-in-cluster = 32
+ |memstore {
+ | memory-alloc {
+ | automatic-alloc-enabled = true
+ | available-memory-bytes = 1GB
+ | lucene-memory-percent = 10
+ | native-memory-manager-percent = 30
+ | block-memory-manager-percent = 60
+ | }
+ |}
+ |""".stripMargin).withFallback(config)
+
+ val memStore1 = new TimeSeriesMemStore(config1, colStore, new InMemoryMetaStore())
+ memStore1.setup(dataset1.ref, schemas1, 0, TestData.storeConf.copy(groupsPerShard = 2,
+ diskTTLSeconds = 1.hour.toSeconds.toInt,
+ flushInterval = 10.minutes, shardMemPercent = 40), 256)
+
+ memStore1.ingestionMemory shouldEqual 300000000 // 1GB * 30%
+ val tsShard = memStore1.getShard(dataset1.ref, 0).get
+ tsShard.blockMemorySize shouldEqual 30000000 // 1GB * 60% (for block memory) * 40% (for dataset memory) / ceil(256/32)
+
+ memStore1.shutdown()
+
+ // Expand cluster by only changing min-num-nodes-in-cluster
+ // now each shard should get more memory since fewer shards per node
+
+ val config2 = ConfigFactory.parseString(
+ """
+ |min-num-nodes-in-cluster = 45
+ |""".stripMargin).withFallback(config1)
+
+ val memStore2 = new TimeSeriesMemStore(config2, colStore, new InMemoryMetaStore())
+ memStore2.setup(dataset1.ref, schemas1, 0, TestData.storeConf.copy(groupsPerShard = 2,
+ diskTTLSeconds = 1.hour.toSeconds.toInt,
+ flushInterval = 10.minutes, shardMemPercent = 40), 256)
+
+ memStore2.ingestionMemory shouldEqual 300000000 // 1GB * 30%
+ val tsShard2 = memStore2.getShard(dataset1.ref, 0).get
+ tsShard2.blockMemorySize shouldEqual 40000000 // 1GB * 60% (for block memory) * 40% (for dataset memory) / ceil(256/45)
+
+ memStore2.shutdown()
+ }
+
it("should recover index data from col store correctly") {
val partBuilder = new RecordBuilder(TestData.nativeMem)
@@ -323,7 +371,7 @@ class TimeSeriesMemStoreSpec extends AnyFunSpec with Matchers with BeforeAndAfte
val memStore = new TimeSeriesMemStore(config, colStore, new InMemoryMetaStore())
memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf.copy(groupsPerShard = 2,
diskTTLSeconds = 1.hour.toSeconds.toInt,
- flushInterval = 10.minutes))
+ flushInterval = 10.minutes), 1)
Thread sleep 1000
val tsShard = memStore.asInstanceOf[TimeSeriesMemStore].getShard(dataset1.ref, 0).get
@@ -346,7 +394,7 @@ class TimeSeriesMemStoreSpec extends AnyFunSpec with Matchers with BeforeAndAfte
}
it("should lookupPartitions and return correct PartLookupResult") {
- memStore.setup(dataset2.ref, schemas2h, 0, TestData.storeConf)
+ memStore.setup(dataset2.ref, schemas2h, 0, TestData.storeConf, 1)
val data = records(dataset2, withMap(linearMultiSeries().take(20))) // 2 records per series x 10 series
memStore.ingest(dataset2.ref, 0, data)
@@ -372,7 +420,7 @@ class TimeSeriesMemStoreSpec extends AnyFunSpec with Matchers with BeforeAndAfte
// 0 -> 2L, 1 -> 4L, 2 -> 6L, 3 -> 8L
// A whole bunch of records should be skipped. However, remember that each group of 5 records gets one offset.
- memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf)
+ memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf, 1)
val initChunksWritten = chunksetsWritten
val checkpoints = Map(0 -> 2L, 1 -> 21L, 2 -> 6L, 3 -> 8L)
@@ -396,7 +444,7 @@ class TimeSeriesMemStoreSpec extends AnyFunSpec with Matchers with BeforeAndAfte
}
it("should recoverStream after timeout, returns endOffset to start normal ingestion") {
- memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf)
+ memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf, 1)
val checkpoints = Map(0 -> 2L, 1 -> 21L, 2 -> 6L, 3 -> 8L)
val stream = Observable.never // "never" to mimic no data in stream source
@@ -410,7 +458,7 @@ class TimeSeriesMemStoreSpec extends AnyFunSpec with Matchers with BeforeAndAfte
}
it("should truncate shards properly") {
- memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf)
+ memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf, 1)
val data = records(dataset1, multiSeriesData().take(20)) // 2 records per series x 10 series
memStore.ingest(dataset1.ref, 0, data)
@@ -437,7 +485,7 @@ class TimeSeriesMemStoreSpec extends AnyFunSpec with Matchers with BeforeAndAfte
}
it("should be able to evict partitions properly on ingestion and on ODP") {
- memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf)
+ memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf, 1)
val shard = memStore.getShardE(dataset1.ref, 0)
// Ingest normal multi series data with 10 partitions. Should have 10 partitions.
@@ -526,7 +574,7 @@ class TimeSeriesMemStoreSpec extends AnyFunSpec with Matchers with BeforeAndAfte
it("should assign same previously assigned partId using bloom filter when evicted series starts re-ingesting") {
- memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf)
+ memStore.setup(dataset1.ref, schemas1, 0, TestData.storeConf, 1)
val shard0 = memStore.getShardE(dataset1.ref, 0)
// Ingest normal multi series data with 10 partitions. Should have 10 partitions.
@@ -568,7 +616,7 @@ class TimeSeriesMemStoreSpec extends AnyFunSpec with Matchers with BeforeAndAfte
try {
// Ingest >250 partitions. Note how much memory is left after all the allocations
- store2.setup(dataset1.ref, schemas1, 0, TestData.storeConf)
+ store2.setup(dataset1.ref, schemas1, 0, TestData.storeConf, 1)
val shard = store2.getShardE(dataset1.ref, 0)
// Ingest normal multi series data with 10 partitions. Should have 10 partitions.
diff --git a/core/src/test/scala/filodb.core/query/SerializedRangeVectorSpec.scala b/core/src/test/scala/filodb.core/query/SerializedRangeVectorSpec.scala
index 46915cc20b..7ab7aa8e6e 100644
--- a/core/src/test/scala/filodb.core/query/SerializedRangeVectorSpec.scala
+++ b/core/src/test/scala/filodb.core/query/SerializedRangeVectorSpec.scala
@@ -51,9 +51,9 @@ class SerializedRangeVectorSpec extends AnyFunSpec with Matchers {
val queryStats = QueryStats()
val srv = SerializedRangeVector.apply(rv, builder, recSchema, "someExecPlan", queryStats)
queryStats.getCpuNanosCounter(Nil).get() > 0 shouldEqual true
- queryStats.getResultBytesCounter(Nil).get() shouldEqual 108
srv.numRows shouldEqual Some(11)
srv.numRowsSerialized shouldEqual 4
+ srv.estimateSerializedRowBytes shouldEqual 80 // 4 non nan records each of 20 bytes
val res = srv.rows.map(r => (r.getLong(0), r.getDouble(1))).toList
res.length shouldEqual 11
res.map(_._1) shouldEqual (0 to 1000 by 100)
@@ -77,9 +77,9 @@ class SerializedRangeVectorSpec extends AnyFunSpec with Matchers {
val queryStats = QueryStats()
val srv = SerializedRangeVector.apply(rv, builder, recSchema, "someExecPlan", queryStats)
queryStats.getCpuNanosCounter(Nil).get() > 0 shouldEqual true
- queryStats.getResultBytesCounter(Nil).get() shouldEqual 248
srv.numRows shouldEqual Some(11)
srv.numRowsSerialized shouldEqual 11
+ srv.estimateSerializedRowBytes shouldEqual 220
val res = srv.rows.map(r => (r.getLong(0), r.getDouble(1))).toList
res.length shouldEqual 11
res.map(_._1) shouldEqual (0 to 1000 by 100)
@@ -105,7 +105,6 @@ class SerializedRangeVectorSpec extends AnyFunSpec with Matchers {
val queryStats = QueryStats()
val srv = SerializedRangeVector.apply(rv, builder, recSchema, "someExecPlan", queryStats)
queryStats.getCpuNanosCounter(Nil).get() > 0 shouldEqual true
- queryStats.getResultBytesCounter(Nil).get() shouldEqual 188
srv.numRows shouldEqual Some(11)
srv.numRowsSerialized shouldEqual 4
val res = srv.rows.map(r => (r.getLong(0), r.getHistogram(1))).toList
@@ -114,4 +113,26 @@ class SerializedRangeVectorSpec extends AnyFunSpec with Matchers {
res.map(_._2).filterNot(_.isEmpty) shouldEqual Seq(h1, h1, h1, h1)
}
+ it("should calculate estimateSerializedRowBytes correctly when builder is used for several SRVs") {
+ val builder = SerializedRangeVector.newBuilder()
+ val recSchema = new RecordSchema(Seq(ColumnInfo("time", ColumnType.TimestampColumn),
+ ColumnInfo("value", ColumnType.DoubleColumn)))
+ val keysMap = Map(UTF8Str("key1") -> UTF8Str("val1"),
+ UTF8Str("key2") -> UTF8Str("val2"))
+ val key = CustomRangeVectorKey(keysMap)
+
+ (0 to 200).foreach { i =>
+ val rv = toRv(Seq((0, Double.NaN), (100, 1.0), (200, Double.NaN),
+ (300, 3.0), (400, Double.NaN),
+ (500, 5.0), (600, 6.0),
+ (700, Double.NaN), (800, Double.NaN),
+ (900, Double.NaN), (1000, Double.NaN)), key,
+ RvRange(1000, 100, 1000))
+ val queryStats = QueryStats()
+ val srv = SerializedRangeVector.apply(rv, builder, recSchema, "someExecPlan", queryStats)
+ srv.numRowsSerialized shouldEqual 11
+ srv.estimateSerializedRowBytes shouldEqual 220
+ }
+ }
+
}
diff --git a/core/src/test/scala/filodb.core/store/ColumnStoreSpec.scala b/core/src/test/scala/filodb.core/store/ColumnStoreSpec.scala
index 57a15afecf..8257cb8691 100644
--- a/core/src/test/scala/filodb.core/store/ColumnStoreSpec.scala
+++ b/core/src/test/scala/filodb.core/store/ColumnStoreSpec.scala
@@ -126,7 +126,7 @@ with BeforeAndAfter with BeforeAndAfterAll with ScalaFutures {
// TODO: FilteredPartitionScan() for ColumnStores does not work without an index right now
ignore should "filter rows written with single partition key" in {
import GdeltTestData._
- memStore.setup(dataset2.ref, schemas, 0, TestData.storeConf)
+ memStore.setup(dataset2.ref, schemas, 0, TestData.storeConf, 1)
val stream = Observable.now(records(dataset2))
// Force flush of all groups at end
memStore.startIngestion(dataset2.ref, 0, stream, s, Task {}).futureValue
@@ -143,7 +143,7 @@ with BeforeAndAfter with BeforeAndAfterAll with ScalaFutures {
// "rangeVectors api" should "return Range Vectors for given filter and read all rows" in {
ignore should "return Range Vectors for given filter and read all rows" in {
import GdeltTestData._
- memStore.setup(dataset2.ref, schemas, 0, TestData.storeConf)
+ memStore.setup(dataset2.ref, schemas, 0, TestData.storeConf, 1)
val stream = Observable.now(records(dataset2))
// Force flush of all groups at end
memStore.startIngestion(dataset2.ref, 0, stream, s, Task {}).futureValue
diff --git a/jmh/src/main/scala/filodb.jmh/HistogramIngestBenchmark.scala b/jmh/src/main/scala/filodb.jmh/HistogramIngestBenchmark.scala
index e4eb963d14..1f5437952b 100644
--- a/jmh/src/main/scala/filodb.jmh/HistogramIngestBenchmark.scala
+++ b/jmh/src/main/scala/filodb.jmh/HistogramIngestBenchmark.scala
@@ -68,8 +68,8 @@ class HistogramIngestBenchmark {
val policy = new FixedMaxPartitionsEvictionPolicy(1000)
val memStore = new TimeSeriesMemStore(config, new NullColumnStore, new InMemoryMetaStore(), Some(policy))
val ingestConf = TestData.storeConf.copy(shardMemSize = 512 * 1024 * 1024, maxChunksSize = 100)
- memStore.setup(histDataset.ref, Schemas(histDataset.schema), 0, ingestConf)
- memStore.setup(promDataset.ref, Schemas(promDataset.schema), 0, ingestConf)
+ memStore.setup(histDataset.ref, Schemas(histDataset.schema), 0, ingestConf, 1)
+ memStore.setup(promDataset.ref, Schemas(promDataset.schema), 0, ingestConf, 1)
val hShard = memStore.getShardE(histDataset.ref, 0)
val pShard = memStore.getShardE(promDataset.ref, 0)
diff --git a/jmh/src/main/scala/filodb.jmh/HistogramQueryBenchmark.scala b/jmh/src/main/scala/filodb.jmh/HistogramQueryBenchmark.scala
index d87b2b7ca8..545027a85a 100644
--- a/jmh/src/main/scala/filodb.jmh/HistogramQueryBenchmark.scala
+++ b/jmh/src/main/scala/filodb.jmh/HistogramQueryBenchmark.scala
@@ -53,7 +53,7 @@ class HistogramQueryBenchmark {
histSchemaData.take(10 * 180).foreach(histSchemaBuilder.addFromReader(_, histDataset.schema))
val histSchemas = Schemas(histDataset.schema)
- memStore.setup(histDataset.ref, histSchemas, 0, ingestConf)
+ memStore.setup(histDataset.ref, histSchemas, 0, ingestConf, 1)
val hShard = memStore.getShardE(histDataset.ref, 0)
histSchemaBuilder.allContainers.foreach { c => hShard.ingest(c, 0) }
memStore.refreshIndexForTesting(histDataset.ref) // commit lucene index
@@ -66,7 +66,7 @@ class HistogramQueryBenchmark {
val promBuilder = new RecordBuilder(MemFactory.onHeapFactory, 4200000)
promData.take(10*66*180).foreach(promBuilder.addFromReader(_, promDataset.schema))
- memStore.setup(promDataset.ref, promSchemas, 0, ingestConf)
+ memStore.setup(promDataset.ref, promSchemas, 0, ingestConf, 1)
val pShard = memStore.getShardE(promDataset.ref, 0)
promBuilder.allContainers.foreach { c => pShard.ingest(c, 0) }
memStore.refreshIndexForTesting(promDataset.ref) // commit lucene index
diff --git a/jmh/src/main/scala/filodb.jmh/IngestionBenchmark.scala b/jmh/src/main/scala/filodb.jmh/IngestionBenchmark.scala
index 286e607c5e..2d3c86b129 100644
--- a/jmh/src/main/scala/filodb.jmh/IngestionBenchmark.scala
+++ b/jmh/src/main/scala/filodb.jmh/IngestionBenchmark.scala
@@ -61,7 +61,7 @@ class IngestionBenchmark {
val policy = new FixedMaxPartitionsEvictionPolicy(100)
val memStore = new TimeSeriesMemStore(config, new NullColumnStore, new InMemoryMetaStore(), Some(policy))
val ingestConf = TestData.storeConf.copy(shardMemSize = 512 * 1024 * 1024, maxChunksSize = 200)
- memStore.setup(dataset1.ref, Schemas(dataset1.schema), 0, ingestConf)
+ memStore.setup(dataset1.ref, Schemas(dataset1.schema), 0, ingestConf, 1)
val shard = memStore.getShardE(dataset1.ref, 0)
diff --git a/jmh/src/main/scala/filodb.jmh/PartKeyIndexBenchmark.scala b/jmh/src/main/scala/filodb.jmh/PartKeyIndexBenchmark.scala
index 5fd16ab097..41b036bf0c 100644
--- a/jmh/src/main/scala/filodb.jmh/PartKeyIndexBenchmark.scala
+++ b/jmh/src/main/scala/filodb.jmh/PartKeyIndexBenchmark.scala
@@ -74,7 +74,7 @@ class PartKeyIndexBenchmark {
partKeyIndex.partIdsFromFilters(
Seq(ColumnFilter("_ns_", Filter.Equals(s"App-$i")),
ColumnFilter("_ws_", Filter.Equals("demo")),
- ColumnFilter("host", Filter.EqualsRegex("H0")),
+ ColumnFilter("host", Filter.Equals("H0")),
ColumnFilter("_metric_", Filter.Equals("heap_usage0"))),
now,
now + 1000)
@@ -90,7 +90,7 @@ class PartKeyIndexBenchmark {
partKeyIndex.partIdsFromFilters(
Seq(ColumnFilter("_ns_", Filter.Equals(s"App-${i + 200}")),
ColumnFilter("_ws_", Filter.Equals("demo")),
- ColumnFilter("host", Filter.EqualsRegex("H0")),
+ ColumnFilter("host", Filter.Equals("H0")),
ColumnFilter("_metric_", Filter.Equals("heap_usage0"))),
now,
now + 1000)
@@ -101,7 +101,7 @@ class PartKeyIndexBenchmark {
@BenchmarkMode(Array(Mode.Throughput))
@OutputTimeUnit(TimeUnit.SECONDS)
@OperationsPerInvocation(8)
- def partIdsLookupWithSuffixRegexFilters(): Unit = {
+ def partIdsLookupWithPrefixRegexFilters(): Unit = {
cforRange ( 0 until 8 ) { i =>
partKeyIndex.partIdsFromFilters(
Seq(ColumnFilter("_ns_", Filter.Equals(s"App-$i")),
@@ -117,7 +117,7 @@ class PartKeyIndexBenchmark {
@BenchmarkMode(Array(Mode.Throughput))
@OutputTimeUnit(TimeUnit.SECONDS)
@OperationsPerInvocation(8)
- def partIdsLookupWithPrefixRegexFilters(): Unit = {
+ def partIdsLookupWithSuffixRegexFilters(): Unit = {
cforRange ( 0 until 8 ) { i =>
partKeyIndex.partIdsFromFilters(
Seq(ColumnFilter("_ns_", Filter.Equals(s"App-$i")),
@@ -129,6 +129,25 @@ class PartKeyIndexBenchmark {
}
}
+ @Benchmark
+ @BenchmarkMode(Array(Mode.Throughput))
+ @OutputTimeUnit(TimeUnit.SECONDS)
+ @OperationsPerInvocation(8)
+ def partIdsLookupWithEnumRegexFilter(): Unit = {
+ cforRange(0 until 8) { i =>
+ val c = partKeyIndex.partIdsFromFilters(
+ Seq(ColumnFilter("_ns_", Filter.Equals(s"App-0")),
+ ColumnFilter("_ws_", Filter.Equals("demo")),
+ ColumnFilter("_metric_", Filter.Equals("heap_usage0")),
+ ColumnFilter("instance",
+ Filter.EqualsRegex("Instance-1|Instance-2|Instance-3|Instance-4|Instance-5|Instance-6|Instance-7|Instance-8|Instance-9|Instance-10|" +
+ "Instance-11|Instance-12|Instance-13|Instance-14|Instance-15|Instance-16|Instance-17|Instance-18|Instance-19|Instance-20|" +
+ "Instance-21|Instance-22|Instance-23|Instance-24|Instance-25|Instance-26|Instance-27|Instance-28|Instance-29|Instance-30"))),
+ now,
+ now + 1000).length
+ }
+ }
+
@Benchmark
@BenchmarkMode(Array(Mode.Throughput))
@OutputTimeUnit(TimeUnit.SECONDS)
diff --git a/kafka/src/main/scala/filodb/kafka/KafkaIngestionStream.scala b/kafka/src/main/scala/filodb/kafka/KafkaIngestionStream.scala
index 66c5bb2ae9..07f9e0b409 100644
--- a/kafka/src/main/scala/filodb/kafka/KafkaIngestionStream.scala
+++ b/kafka/src/main/scala/filodb/kafka/KafkaIngestionStream.scala
@@ -59,6 +59,7 @@ class KafkaIngestionStream(config: Config,
if (sourceConfig.LogConfig) logger.info(s"Consumer properties: $props")
blocking {
+ props.put("client.id", s"${props.get("group.id")}.${System.getenv("INSTANCE_ID")}.$shard")
val consumer = new KafkaConsumer(props)
consumer.assign(List(topicPartition).asJava)
offset.foreach { off => consumer.seek(topicPartition, off) }
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 e5db5d6e4a..f749ee236f 100644
--- a/memory/src/main/scala/filodb.memory/format/vectors/DoubleVector.scala
+++ b/memory/src/main/scala/filodb.memory/format/vectors/DoubleVector.scala
@@ -242,7 +242,7 @@ object DoubleVectorDataReader64 extends DoubleVectorDataReader {
val nextDbl = acc.getDouble(addr)
// There are many possible values of NaN. Use a function to ignore them reliably.
if (!java.lang.Double.isNaN(nextDbl)) {
- if (sum.isNaN) sum = 0d
+ if (java.lang.Double.isNaN(sum)) sum = 0d
sum += nextDbl
}
addr += 8
@@ -323,7 +323,7 @@ extends DoubleVectorDataReader {
var last = Double.MinValue
cforRange { 0 until len } { pos =>
var nextVal = it.next
- if (nextVal.isNaN) nextVal = 0 // explicit counter reset due to end of time series marker
+ if (java.lang.Double.isNaN(nextVal)) nextVal = 0 // explicit counter reset due to end of time series marker
if (nextVal < last) { // reset!
_correction += last
_drops += pos
@@ -443,9 +443,9 @@ class DoubleCounterAppender(addr: BinaryRegion.NativePointer, maxBytes: Int, dis
extends DoubleAppendingVector(addr, maxBytes, dispose) {
private var last = Double.MinValue
override final def addData(data: Double): AddResponse = {
- if (data.isNaN || data < last)
+ if (java.lang.Double.isNaN(data) || data < last)
PrimitiveVectorReader.markDrop(MemoryAccessor.nativePtrAccessor, addr)
- if (!data.isNaN)
+ if (!java.lang.Double.isNaN(data))
last = data
super.addData(data)
}
@@ -544,7 +544,7 @@ object DoubleLongWrapDataReader extends DoubleVectorDataReader {
final def changes(acc: MemoryReader, vector: BinaryVectorPtr, start: Int, end: Int,
prev: Double, ignorePrev: Boolean = false):
(Double, Double) = {
- val ignorePrev = if (prev.isNaN) true
+ val ignorePrev = if (java.lang.Double.isNaN(prev)) true
else false
val changes = LongBinaryVector(acc, vector).changes(acc, vector, start, end, prev.toLong, ignorePrev)
(changes._1.toDouble, changes._2.toDouble)
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 4ebb711d6c..3e8bd77a8b 100644
--- a/memory/src/main/scala/filodb.memory/format/vectors/Histogram.scala
+++ b/memory/src/main/scala/filodb.memory/format/vectors/Histogram.scala
@@ -228,7 +228,7 @@ final case class MutableHistogram(buckets: HistogramBuckets, values: Array[Doubl
// Allow addition when type of bucket is different
if (buckets.similarForMath(other.buckets)) {
// If it was NaN before, reset to 0 to sum another hist
- if (values(0).isNaN) java.util.Arrays.fill(values, 0.0)
+ if (java.lang.Double.isNaN(values(0))) java.util.Arrays.fill(values, 0.0)
cforRange { 0 until numBuckets } { b =>
values(b) += other.bucketValue(b)
}
@@ -272,7 +272,7 @@ final case class MutableHistogram(buckets: HistogramBuckets, values: Array[Doubl
cforRange { 0 until values.size } { b =>
// When bucket no longer used NaN will be seen. Non-increasing values can be seen when
// newer buckets are introduced and not all instances are updated with that bucket.
- if (values(b) < max || values(b).isNaN) values(b) = max // assign previous max
+ if (values(b) < max || java.lang.Double.isNaN(values(b))) values(b) = max // assign previous max
else if (values(b) > max) max = values(b) // update max
}
}
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 2a6f230e86..d9ca52558c 100644
--- a/memory/src/main/scala/filodb.memory/format/vectors/HistogramVector.scala
+++ b/memory/src/main/scala/filodb.memory/format/vectors/HistogramVector.scala
@@ -583,7 +583,6 @@ class SectDeltaHistogramReader(acc2: MemoryReader, histVect: Ptr.U8)
throw EmptyHistogramException(s"""length = $length memory=${toHexString(acc, histVect.addr)}""")
}
- require(length > 0)
val histPtr = locate(index)
// Just return the base histogram if we are at start of section
diff --git a/prometheus/src/main/java/filodb/prometheus/antlr/PromQL.g4 b/prometheus/src/main/java/filodb/prometheus/antlr/PromQL.g4
index aadb1a36f8..d5ccc62656 100644
--- a/prometheus/src/main/java/filodb/prometheus/antlr/PromQL.g4
+++ b/prometheus/src/main/java/filodb/prometheus/antlr/PromQL.g4
@@ -20,7 +20,7 @@ vectorExpression
| vectorExpression compareOp grouping? vectorExpression #binaryOperation
| vectorExpression andUnlessOp grouping? vectorExpression #binaryOperation
| vectorExpression orOp grouping? vectorExpression #binaryOperation
- | vectorExpression subquery offset? #subqueryOperation
+ | vectorExpression subquery modifier? #subqueryOperation
| vectorExpression limit #limitOperation
| vector #vectorOperation
;
@@ -44,7 +44,7 @@ vector
parens: '(' vectorExpression ')';
// TODO: Make offset applicable to any expression.
-instantOrRangeSelector: instantSelector window? offset?;
+instantOrRangeSelector: instantSelector window? modifier?;
instantSelector
: metricName ('{' labelMatcherList? '}')?
@@ -55,6 +55,10 @@ window: '[' DURATION ']';
offset: OFFSET DURATION;
+atModifier: AT(NUMBER | START | END);
+
+modifier: offset | atModifier | offset atModifier | atModifier offset;
+
limit: LIMIT NUMBER;
subquery: '[' DURATION ':' DURATION? ']';
@@ -149,6 +153,8 @@ LE: '<=';
RE: '=~';
NRE: '!~';
+AT: '@';
+
// See section below: "Magic for case-insensitive matching."
AND: A N D;
OR: O R;
@@ -162,6 +168,8 @@ GROUP_RIGHT: G R O U P '_' R I G H T;
OFFSET: O F F S E T;
LIMIT: L I M I T;
BOOL: B O O L;
+START: S T A R T '(' ')';
+END: E N D '(' ')';
// See section below: "Magic for case-insensitive matching."
AGGREGATION_OP
diff --git a/prometheus/src/main/java/filodb/prometheus/antlr/PromQL.interp b/prometheus/src/main/java/filodb/prometheus/antlr/PromQL.interp
index 1e1b24d02b..8b6d82b11b 100644
--- a/prometheus/src/main/java/filodb/prometheus/antlr/PromQL.interp
+++ b/prometheus/src/main/java/filodb/prometheus/antlr/PromQL.interp
@@ -25,6 +25,9 @@ null
'<='
'=~'
'!~'
+'@'
+null
+null
null
null
null
@@ -71,6 +74,7 @@ GE
LE
RE
NRE
+AT
AND
OR
UNLESS
@@ -83,6 +87,8 @@ GROUP_RIGHT
OFFSET
LIMIT
BOOL
+START
+END
AGGREGATION_OP
DURATION
IDENTIFIER
@@ -106,6 +112,8 @@ instantOrRangeSelector
instantSelector
window
offset
+atModifier
+modifier
limit
subquery
labelMatcher
@@ -131,4 +139,4 @@ literal
atn:
-[3, 24715, 42794, 33075, 47597, 16764, 15335, 30598, 22884, 3, 45, 317, 4, 2, 9, 2, 4, 3, 9, 3, 4, 4, 9, 4, 4, 5, 9, 5, 4, 6, 9, 6, 4, 7, 9, 7, 4, 8, 9, 8, 4, 9, 9, 9, 4, 10, 9, 10, 4, 11, 9, 11, 4, 12, 9, 12, 4, 13, 9, 13, 4, 14, 9, 14, 4, 15, 9, 15, 4, 16, 9, 16, 4, 17, 9, 17, 4, 18, 9, 18, 4, 19, 9, 19, 4, 20, 9, 20, 4, 21, 9, 21, 4, 22, 9, 22, 4, 23, 9, 23, 4, 24, 9, 24, 4, 25, 9, 25, 4, 26, 9, 26, 4, 27, 9, 27, 4, 28, 9, 28, 4, 29, 9, 29, 4, 30, 9, 30, 4, 31, 9, 31, 4, 32, 9, 32, 4, 33, 9, 33, 4, 34, 9, 34, 4, 35, 9, 35, 4, 36, 9, 36, 4, 37, 9, 37, 4, 38, 9, 38, 3, 2, 3, 2, 3, 2, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 5, 3, 85, 10, 3, 3, 3, 3, 3, 3, 3, 5, 3, 90, 10, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 5, 3, 97, 10, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 5, 3, 104, 10, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 5, 3, 111, 10, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 5, 3, 118, 10, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 5, 3, 125, 10, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 5, 3, 132, 10, 3, 3, 3, 3, 3, 7, 3, 136, 10, 3, 12, 3, 14, 3, 139, 11, 3, 3, 4, 3, 4, 3, 5, 3, 5, 3, 6, 3, 6, 3, 7, 3, 7, 3, 8, 3, 8, 5, 8, 151, 10, 8, 3, 9, 3, 9, 3, 10, 3, 10, 3, 11, 3, 11, 3, 11, 3, 11, 3, 11, 5, 11, 162, 10, 11, 3, 12, 3, 12, 3, 12, 3, 12, 3, 13, 3, 13, 5, 13, 170, 10, 13, 3, 13, 5, 13, 173, 10, 13, 3, 14, 3, 14, 3, 14, 5, 14, 178, 10, 14, 3, 14, 5, 14, 181, 10, 14, 3, 14, 3, 14, 3, 14, 3, 14, 5, 14, 187, 10, 14, 3, 15, 3, 15, 3, 15, 3, 15, 3, 16, 3, 16, 3, 16, 3, 17, 3, 17, 3, 17, 3, 18, 3, 18, 3, 18, 3, 18, 5, 18, 203, 10, 18, 3, 18, 3, 18, 3, 19, 3, 19, 3, 19, 3, 19, 3, 20, 3, 20, 3, 21, 3, 21, 3, 21, 7, 21, 216, 10, 21, 12, 21, 14, 21, 219, 11, 21, 3, 22, 3, 22, 3, 22, 3, 23, 3, 23, 5, 23, 226, 10, 23, 3, 24, 3, 24, 3, 24, 3, 24, 7, 24, 232, 10, 24, 12, 24, 14, 24, 235, 11, 24, 5, 24, 237, 10, 24, 3, 24, 3, 24, 3, 25, 3, 25, 3, 25, 3, 25, 3, 25, 5, 25, 246, 10, 25, 3, 25, 3, 25, 3, 25, 3, 25, 3, 25, 3, 25, 5, 25, 254, 10, 25, 5, 25, 256, 10, 25, 3, 26, 3, 26, 3, 26, 3, 27, 3, 27, 3, 27, 3, 28, 3, 28, 5, 28, 266, 10, 28, 3, 28, 3, 28, 5, 28, 270, 10, 28, 3, 29, 3, 29, 3, 29, 3, 30, 3, 30, 3, 30, 3, 31, 3, 31, 5, 31, 280, 10, 31, 3, 32, 3, 32, 5, 32, 284, 10, 32, 3, 33, 3, 33, 3, 33, 5, 33, 289, 10, 33, 3, 34, 3, 34, 3, 35, 3, 35, 5, 35, 295, 10, 35, 3, 36, 3, 36, 3, 36, 3, 36, 7, 36, 301, 10, 36, 12, 36, 14, 36, 304, 11, 36, 5, 36, 306, 10, 36, 3, 36, 5, 36, 309, 10, 36, 3, 36, 3, 36, 3, 37, 3, 37, 3, 38, 3, 38, 3, 38, 2, 3, 4, 39, 2, 4, 6, 8, 10, 12, 14, 16, 18, 20, 22, 24, 26, 28, 30, 32, 34, 36, 38, 40, 42, 44, 46, 48, 50, 52, 54, 56, 58, 60, 62, 64, 66, 68, 70, 72, 74, 2, 10, 3, 2, 13, 14, 3, 2, 15, 17, 3, 2, 20, 25, 4, 2, 28, 28, 30, 30, 5, 2, 19, 19, 21, 21, 26, 27, 5, 2, 28, 32, 37, 38, 40, 40, 3, 2, 28, 40, 3, 2, 11, 12, 2, 325, 2, 76, 3, 2, 2, 2, 4, 84, 3, 2, 2, 2, 6, 140, 3, 2, 2, 2, 8, 142, 3, 2, 2, 2, 10, 144, 3, 2, 2, 2, 12, 146, 3, 2, 2, 2, 14, 148, 3, 2, 2, 2, 16, 152, 3, 2, 2, 2, 18, 154, 3, 2, 2, 2, 20, 161, 3, 2, 2, 2, 22, 163, 3, 2, 2, 2, 24, 167, 3, 2, 2, 2, 26, 186, 3, 2, 2, 2, 28, 188, 3, 2, 2, 2, 30, 192, 3, 2, 2, 2, 32, 195, 3, 2, 2, 2, 34, 198, 3, 2, 2, 2, 36, 206, 3, 2, 2, 2, 38, 210, 3, 2, 2, 2, 40, 212, 3, 2, 2, 2, 42, 220, 3, 2, 2, 2, 44, 225, 3, 2, 2, 2, 46, 227, 3, 2, 2, 2, 48, 255, 3, 2, 2, 2, 50, 257, 3, 2, 2, 2, 52, 260, 3, 2, 2, 2, 54, 265, 3, 2, 2, 2, 56, 271, 3, 2, 2, 2, 58, 274, 3, 2, 2, 2, 60, 277, 3, 2, 2, 2, 62, 281, 3, 2, 2, 2, 64, 288, 3, 2, 2, 2, 66, 290, 3, 2, 2, 2, 68, 294, 3, 2, 2, 2, 70, 296, 3, 2, 2, 2, 72, 312, 3, 2, 2, 2, 74, 314, 3, 2, 2, 2, 76, 77, 5, 4, 3, 2, 77, 78, 7, 2, 2, 3, 78, 3, 3, 2, 2, 2, 79, 80, 8, 3, 1, 2, 80, 81, 5, 6, 4, 2, 81, 82, 5, 4, 3, 11, 82, 85, 3, 2, 2, 2, 83, 85, 5, 20, 11, 2, 84, 79, 3, 2, 2, 2, 84, 83, 3, 2, 2, 2, 85, 137, 3, 2, 2, 2, 86, 87, 12, 12, 2, 2, 87, 89, 5, 8, 5, 2, 88, 90, 5, 54, 28, 2, 89, 88, 3, 2, 2, 2, 89, 90, 3, 2, 2, 2, 90, 91, 3, 2, 2, 2, 91, 92, 5, 4, 3, 12, 92, 136, 3, 2, 2, 2, 93, 94, 12, 10, 2, 2, 94, 96, 5, 10, 6, 2, 95, 97, 5, 54, 28, 2, 96, 95, 3, 2, 2, 2, 96, 97, 3, 2, 2, 2, 97, 98, 3, 2, 2, 2, 98, 99, 5, 4, 3, 11, 99, 136, 3, 2, 2, 2, 100, 101, 12, 9, 2, 2, 101, 103, 5, 12, 7, 2, 102, 104, 5, 54, 28, 2, 103, 102, 3, 2, 2, 2, 103, 104, 3, 2, 2, 2, 104, 105, 3, 2, 2, 2, 105, 106, 5, 4, 3, 10, 106, 136, 3, 2, 2, 2, 107, 108, 12, 8, 2, 2, 108, 110, 5, 14, 8, 2, 109, 111, 5, 54, 28, 2, 110, 109, 3, 2, 2, 2, 110, 111, 3, 2, 2, 2, 111, 112, 3, 2, 2, 2, 112, 113, 5, 4, 3, 9, 113, 136, 3, 2, 2, 2, 114, 115, 12, 7, 2, 2, 115, 117, 5, 16, 9, 2, 116, 118, 5, 54, 28, 2, 117, 116, 3, 2, 2, 2, 117, 118, 3, 2, 2, 2, 118, 119, 3, 2, 2, 2, 119, 120, 5, 4, 3, 8, 120, 136, 3, 2, 2, 2, 121, 122, 12, 6, 2, 2, 122, 124, 5, 18, 10, 2, 123, 125, 5, 54, 28, 2, 124, 123, 3, 2, 2, 2, 124, 125, 3, 2, 2, 2, 125, 126, 3, 2, 2, 2, 126, 127, 5, 4, 3, 7, 127, 136, 3, 2, 2, 2, 128, 129, 12, 5, 2, 2, 129, 131, 5, 34, 18, 2, 130, 132, 5, 30, 16, 2, 131, 130, 3, 2, 2, 2, 131, 132, 3, 2, 2, 2, 132, 136, 3, 2, 2, 2, 133, 134, 12, 4, 2, 2, 134, 136, 5, 32, 17, 2, 135, 86, 3, 2, 2, 2, 135, 93, 3, 2, 2, 2, 135, 100, 3, 2, 2, 2, 135, 107, 3, 2, 2, 2, 135, 114, 3, 2, 2, 2, 135, 121, 3, 2, 2, 2, 135, 128, 3, 2, 2, 2, 135, 133, 3, 2, 2, 2, 136, 139, 3, 2, 2, 2, 137, 135, 3, 2, 2, 2, 137, 138, 3, 2, 2, 2, 138, 5, 3, 2, 2, 2, 139, 137, 3, 2, 2, 2, 140, 141, 9, 2, 2, 2, 141, 7, 3, 2, 2, 2, 142, 143, 7, 18, 2, 2, 143, 9, 3, 2, 2, 2, 144, 145, 9, 3, 2, 2, 145, 11, 3, 2, 2, 2, 146, 147, 9, 2, 2, 2, 147, 13, 3, 2, 2, 2, 148, 150, 9, 4, 2, 2, 149, 151, 7, 39, 2, 2, 150, 149, 3, 2, 2, 2, 150, 151, 3, 2, 2, 2, 151, 15, 3, 2, 2, 2, 152, 153, 9, 5, 2, 2, 153, 17, 3, 2, 2, 2, 154, 155, 7, 29, 2, 2, 155, 19, 3, 2, 2, 2, 156, 162, 5, 42, 22, 2, 157, 162, 5, 48, 25, 2, 158, 162, 5, 24, 13, 2, 159, 162, 5, 74, 38, 2, 160, 162, 5, 22, 12, 2, 161, 156, 3, 2, 2, 2, 161, 157, 3, 2, 2, 2, 161, 158, 3, 2, 2, 2, 161, 159, 3, 2, 2, 2, 161, 160, 3, 2, 2, 2, 162, 21, 3, 2, 2, 2, 163, 164, 7, 3, 2, 2, 164, 165, 5, 4, 3, 2, 165, 166, 7, 4, 2, 2, 166, 23, 3, 2, 2, 2, 167, 169, 5, 26, 14, 2, 168, 170, 5, 28, 15, 2, 169, 168, 3, 2, 2, 2, 169, 170, 3, 2, 2, 2, 170, 172, 3, 2, 2, 2, 171, 173, 5, 30, 16, 2, 172, 171, 3, 2, 2, 2, 172, 173, 3, 2, 2, 2, 173, 25, 3, 2, 2, 2, 174, 180, 5, 64, 33, 2, 175, 177, 7, 5, 2, 2, 176, 178, 5, 40, 21, 2, 177, 176, 3, 2, 2, 2, 177, 178, 3, 2, 2, 2, 178, 179, 3, 2, 2, 2, 179, 181, 7, 6, 2, 2, 180, 175, 3, 2, 2, 2, 180, 181, 3, 2, 2, 2, 181, 187, 3, 2, 2, 2, 182, 183, 7, 5, 2, 2, 183, 184, 5, 40, 21, 2, 184, 185, 7, 6, 2, 2, 185, 187, 3, 2, 2, 2, 186, 174, 3, 2, 2, 2, 186, 182, 3, 2, 2, 2, 187, 27, 3, 2, 2, 2, 188, 189, 7, 7, 2, 2, 189, 190, 7, 41, 2, 2, 190, 191, 7, 8, 2, 2, 191, 29, 3, 2, 2, 2, 192, 193, 7, 37, 2, 2, 193, 194, 7, 41, 2, 2, 194, 31, 3, 2, 2, 2, 195, 196, 7, 38, 2, 2, 196, 197, 7, 11, 2, 2, 197, 33, 3, 2, 2, 2, 198, 199, 7, 7, 2, 2, 199, 200, 7, 41, 2, 2, 200, 202, 7, 9, 2, 2, 201, 203, 7, 41, 2, 2, 202, 201, 3, 2, 2, 2, 202, 203, 3, 2, 2, 2, 203, 204, 3, 2, 2, 2, 204, 205, 7, 8, 2, 2, 205, 35, 3, 2, 2, 2, 206, 207, 5, 68, 35, 2, 207, 208, 5, 38, 20, 2, 208, 209, 7, 12, 2, 2, 209, 37, 3, 2, 2, 2, 210, 211, 9, 6, 2, 2, 211, 39, 3, 2, 2, 2, 212, 217, 5, 36, 19, 2, 213, 214, 7, 10, 2, 2, 214, 216, 5, 36, 19, 2, 215, 213, 3, 2, 2, 2, 216, 219, 3, 2, 2, 2, 217, 215, 3, 2, 2, 2, 217, 218, 3, 2, 2, 2, 218, 41, 3, 2, 2, 2, 219, 217, 3, 2, 2, 2, 220, 221, 7, 42, 2, 2, 221, 222, 5, 46, 24, 2, 222, 43, 3, 2, 2, 2, 223, 226, 5, 74, 38, 2, 224, 226, 5, 4, 3, 2, 225, 223, 3, 2, 2, 2, 225, 224, 3, 2, 2, 2, 226, 45, 3, 2, 2, 2, 227, 236, 7, 3, 2, 2, 228, 233, 5, 44, 23, 2, 229, 230, 7, 10, 2, 2, 230, 232, 5, 44, 23, 2, 231, 229, 3, 2, 2, 2, 232, 235, 3, 2, 2, 2, 233, 231, 3, 2, 2, 2, 233, 234, 3, 2, 2, 2, 234, 237, 3, 2, 2, 2, 235, 233, 3, 2, 2, 2, 236, 228, 3, 2, 2, 2, 236, 237, 3, 2, 2, 2, 237, 238, 3, 2, 2, 2, 238, 239, 7, 4, 2, 2, 239, 47, 3, 2, 2, 2, 240, 241, 7, 40, 2, 2, 241, 256, 5, 46, 24, 2, 242, 245, 7, 40, 2, 2, 243, 246, 5, 50, 26, 2, 244, 246, 5, 52, 27, 2, 245, 243, 3, 2, 2, 2, 245, 244, 3, 2, 2, 2, 246, 247, 3, 2, 2, 2, 247, 248, 5, 46, 24, 2, 248, 256, 3, 2, 2, 2, 249, 250, 7, 40, 2, 2, 250, 253, 5, 46, 24, 2, 251, 254, 5, 50, 26, 2, 252, 254, 5, 52, 27, 2, 253, 251, 3, 2, 2, 2, 253, 252, 3, 2, 2, 2, 254, 256, 3, 2, 2, 2, 255, 240, 3, 2, 2, 2, 255, 242, 3, 2, 2, 2, 255, 249, 3, 2, 2, 2, 256, 49, 3, 2, 2, 2, 257, 258, 7, 31, 2, 2, 258, 259, 5, 70, 36, 2, 259, 51, 3, 2, 2, 2, 260, 261, 7, 32, 2, 2, 261, 262, 5, 70, 36, 2, 262, 53, 3, 2, 2, 2, 263, 266, 5, 56, 29, 2, 264, 266, 5, 58, 30, 2, 265, 263, 3, 2, 2, 2, 265, 264, 3, 2, 2, 2, 266, 269, 3, 2, 2, 2, 267, 270, 5, 60, 31, 2, 268, 270, 5, 62, 32, 2, 269, 267, 3, 2, 2, 2, 269, 268, 3, 2, 2, 2, 269, 270, 3, 2, 2, 2, 270, 55, 3, 2, 2, 2, 271, 272, 7, 33, 2, 2, 272, 273, 5, 70, 36, 2, 273, 57, 3, 2, 2, 2, 274, 275, 7, 34, 2, 2, 275, 276, 5, 70, 36, 2, 276, 59, 3, 2, 2, 2, 277, 279, 7, 35, 2, 2, 278, 280, 5, 70, 36, 2, 279, 278, 3, 2, 2, 2, 279, 280, 3, 2, 2, 2, 280, 61, 3, 2, 2, 2, 281, 283, 7, 36, 2, 2, 282, 284, 5, 70, 36, 2, 283, 282, 3, 2, 2, 2, 283, 284, 3, 2, 2, 2, 284, 63, 3, 2, 2, 2, 285, 289, 5, 66, 34, 2, 286, 289, 7, 42, 2, 2, 287, 289, 7, 43, 2, 2, 288, 285, 3, 2, 2, 2, 288, 286, 3, 2, 2, 2, 288, 287, 3, 2, 2, 2, 289, 65, 3, 2, 2, 2, 290, 291, 9, 7, 2, 2, 291, 67, 3, 2, 2, 2, 292, 295, 5, 72, 37, 2, 293, 295, 7, 42, 2, 2, 294, 292, 3, 2, 2, 2, 294, 293, 3, 2, 2, 2, 295, 69, 3, 2, 2, 2, 296, 305, 7, 3, 2, 2, 297, 302, 5, 68, 35, 2, 298, 299, 7, 10, 2, 2, 299, 301, 5, 68, 35, 2, 300, 298, 3, 2, 2, 2, 301, 304, 3, 2, 2, 2, 302, 300, 3, 2, 2, 2, 302, 303, 3, 2, 2, 2, 303, 306, 3, 2, 2, 2, 304, 302, 3, 2, 2, 2, 305, 297, 3, 2, 2, 2, 305, 306, 3, 2, 2, 2, 306, 308, 3, 2, 2, 2, 307, 309, 7, 10, 2, 2, 308, 307, 3, 2, 2, 2, 308, 309, 3, 2, 2, 2, 309, 310, 3, 2, 2, 2, 310, 311, 7, 4, 2, 2, 311, 71, 3, 2, 2, 2, 312, 313, 9, 8, 2, 2, 313, 73, 3, 2, 2, 2, 314, 315, 9, 9, 2, 2, 315, 75, 3, 2, 2, 2, 36, 84, 89, 96, 103, 110, 117, 124, 131, 135, 137, 150, 161, 169, 172, 177, 180, 186, 202, 217, 225, 233, 236, 245, 253, 255, 265, 269, 279, 283, 288, 294, 302, 305, 308]
\ No newline at end of file
+[3, 24715, 42794, 33075, 47597, 16764, 15335, 30598, 22884, 3, 48, 334, 4, 2, 9, 2, 4, 3, 9, 3, 4, 4, 9, 4, 4, 5, 9, 5, 4, 6, 9, 6, 4, 7, 9, 7, 4, 8, 9, 8, 4, 9, 9, 9, 4, 10, 9, 10, 4, 11, 9, 11, 4, 12, 9, 12, 4, 13, 9, 13, 4, 14, 9, 14, 4, 15, 9, 15, 4, 16, 9, 16, 4, 17, 9, 17, 4, 18, 9, 18, 4, 19, 9, 19, 4, 20, 9, 20, 4, 21, 9, 21, 4, 22, 9, 22, 4, 23, 9, 23, 4, 24, 9, 24, 4, 25, 9, 25, 4, 26, 9, 26, 4, 27, 9, 27, 4, 28, 9, 28, 4, 29, 9, 29, 4, 30, 9, 30, 4, 31, 9, 31, 4, 32, 9, 32, 4, 33, 9, 33, 4, 34, 9, 34, 4, 35, 9, 35, 4, 36, 9, 36, 4, 37, 9, 37, 4, 38, 9, 38, 4, 39, 9, 39, 4, 40, 9, 40, 3, 2, 3, 2, 3, 2, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 5, 3, 89, 10, 3, 3, 3, 3, 3, 3, 3, 5, 3, 94, 10, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 5, 3, 101, 10, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 5, 3, 108, 10, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 5, 3, 115, 10, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 5, 3, 122, 10, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 5, 3, 129, 10, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 5, 3, 136, 10, 3, 3, 3, 3, 3, 7, 3, 140, 10, 3, 12, 3, 14, 3, 143, 11, 3, 3, 4, 3, 4, 3, 5, 3, 5, 3, 6, 3, 6, 3, 7, 3, 7, 3, 8, 3, 8, 5, 8, 155, 10, 8, 3, 9, 3, 9, 3, 10, 3, 10, 3, 11, 3, 11, 3, 11, 3, 11, 3, 11, 5, 11, 166, 10, 11, 3, 12, 3, 12, 3, 12, 3, 12, 3, 13, 3, 13, 5, 13, 174, 10, 13, 3, 13, 5, 13, 177, 10, 13, 3, 14, 3, 14, 3, 14, 5, 14, 182, 10, 14, 3, 14, 5, 14, 185, 10, 14, 3, 14, 3, 14, 3, 14, 3, 14, 5, 14, 191, 10, 14, 3, 15, 3, 15, 3, 15, 3, 15, 3, 16, 3, 16, 3, 16, 3, 17, 3, 17, 3, 17, 3, 18, 3, 18, 3, 18, 3, 18, 3, 18, 3, 18, 3, 18, 3, 18, 5, 18, 211, 10, 18, 3, 19, 3, 19, 3, 19, 3, 20, 3, 20, 3, 20, 3, 20, 5, 20, 220, 10, 20, 3, 20, 3, 20, 3, 21, 3, 21, 3, 21, 3, 21, 3, 22, 3, 22, 3, 23, 3, 23, 3, 23, 7, 23, 233, 10, 23, 12, 23, 14, 23, 236, 11, 23, 3, 24, 3, 24, 3, 24, 3, 25, 3, 25, 5, 25, 243, 10, 25, 3, 26, 3, 26, 3, 26, 3, 26, 7, 26, 249, 10, 26, 12, 26, 14, 26, 252, 11, 26, 5, 26, 254, 10, 26, 3, 26, 3, 26, 3, 27, 3, 27, 3, 27, 3, 27, 3, 27, 5, 27, 263, 10, 27, 3, 27, 3, 27, 3, 27, 3, 27, 3, 27, 3, 27, 5, 27, 271, 10, 27, 5, 27, 273, 10, 27, 3, 28, 3, 28, 3, 28, 3, 29, 3, 29, 3, 29, 3, 30, 3, 30, 5, 30, 283, 10, 30, 3, 30, 3, 30, 5, 30, 287, 10, 30, 3, 31, 3, 31, 3, 31, 3, 32, 3, 32, 3, 32, 3, 33, 3, 33, 5, 33, 297, 10, 33, 3, 34, 3, 34, 5, 34, 301, 10, 34, 3, 35, 3, 35, 3, 35, 5, 35, 306, 10, 35, 3, 36, 3, 36, 3, 37, 3, 37, 5, 37, 312, 10, 37, 3, 38, 3, 38, 3, 38, 3, 38, 7, 38, 318, 10, 38, 12, 38, 14, 38, 321, 11, 38, 5, 38, 323, 10, 38, 3, 38, 5, 38, 326, 10, 38, 3, 38, 3, 38, 3, 39, 3, 39, 3, 40, 3, 40, 3, 40, 2, 3, 4, 41, 2, 4, 6, 8, 10, 12, 14, 16, 18, 20, 22, 24, 26, 28, 30, 32, 34, 36, 38, 40, 42, 44, 46, 48, 50, 52, 54, 56, 58, 60, 62, 64, 66, 68, 70, 72, 74, 76, 78, 2, 11, 3, 2, 13, 14, 3, 2, 15, 17, 3, 2, 20, 25, 4, 2, 29, 29, 31, 31, 4, 2, 11, 11, 41, 42, 5, 2, 19, 19, 21, 21, 26, 27, 5, 2, 29, 33, 38, 39, 43, 43, 4, 2, 29, 40, 43, 43, 3, 2, 11, 12, 2, 343, 2, 80, 3, 2, 2, 2, 4, 88, 3, 2, 2, 2, 6, 144, 3, 2, 2, 2, 8, 146, 3, 2, 2, 2, 10, 148, 3, 2, 2, 2, 12, 150, 3, 2, 2, 2, 14, 152, 3, 2, 2, 2, 16, 156, 3, 2, 2, 2, 18, 158, 3, 2, 2, 2, 20, 165, 3, 2, 2, 2, 22, 167, 3, 2, 2, 2, 24, 171, 3, 2, 2, 2, 26, 190, 3, 2, 2, 2, 28, 192, 3, 2, 2, 2, 30, 196, 3, 2, 2, 2, 32, 199, 3, 2, 2, 2, 34, 210, 3, 2, 2, 2, 36, 212, 3, 2, 2, 2, 38, 215, 3, 2, 2, 2, 40, 223, 3, 2, 2, 2, 42, 227, 3, 2, 2, 2, 44, 229, 3, 2, 2, 2, 46, 237, 3, 2, 2, 2, 48, 242, 3, 2, 2, 2, 50, 244, 3, 2, 2, 2, 52, 272, 3, 2, 2, 2, 54, 274, 3, 2, 2, 2, 56, 277, 3, 2, 2, 2, 58, 282, 3, 2, 2, 2, 60, 288, 3, 2, 2, 2, 62, 291, 3, 2, 2, 2, 64, 294, 3, 2, 2, 2, 66, 298, 3, 2, 2, 2, 68, 305, 3, 2, 2, 2, 70, 307, 3, 2, 2, 2, 72, 311, 3, 2, 2, 2, 74, 313, 3, 2, 2, 2, 76, 329, 3, 2, 2, 2, 78, 331, 3, 2, 2, 2, 80, 81, 5, 4, 3, 2, 81, 82, 7, 2, 2, 3, 82, 3, 3, 2, 2, 2, 83, 84, 8, 3, 1, 2, 84, 85, 5, 6, 4, 2, 85, 86, 5, 4, 3, 11, 86, 89, 3, 2, 2, 2, 87, 89, 5, 20, 11, 2, 88, 83, 3, 2, 2, 2, 88, 87, 3, 2, 2, 2, 89, 141, 3, 2, 2, 2, 90, 91, 12, 12, 2, 2, 91, 93, 5, 8, 5, 2, 92, 94, 5, 58, 30, 2, 93, 92, 3, 2, 2, 2, 93, 94, 3, 2, 2, 2, 94, 95, 3, 2, 2, 2, 95, 96, 5, 4, 3, 12, 96, 140, 3, 2, 2, 2, 97, 98, 12, 10, 2, 2, 98, 100, 5, 10, 6, 2, 99, 101, 5, 58, 30, 2, 100, 99, 3, 2, 2, 2, 100, 101, 3, 2, 2, 2, 101, 102, 3, 2, 2, 2, 102, 103, 5, 4, 3, 11, 103, 140, 3, 2, 2, 2, 104, 105, 12, 9, 2, 2, 105, 107, 5, 12, 7, 2, 106, 108, 5, 58, 30, 2, 107, 106, 3, 2, 2, 2, 107, 108, 3, 2, 2, 2, 108, 109, 3, 2, 2, 2, 109, 110, 5, 4, 3, 10, 110, 140, 3, 2, 2, 2, 111, 112, 12, 8, 2, 2, 112, 114, 5, 14, 8, 2, 113, 115, 5, 58, 30, 2, 114, 113, 3, 2, 2, 2, 114, 115, 3, 2, 2, 2, 115, 116, 3, 2, 2, 2, 116, 117, 5, 4, 3, 9, 117, 140, 3, 2, 2, 2, 118, 119, 12, 7, 2, 2, 119, 121, 5, 16, 9, 2, 120, 122, 5, 58, 30, 2, 121, 120, 3, 2, 2, 2, 121, 122, 3, 2, 2, 2, 122, 123, 3, 2, 2, 2, 123, 124, 5, 4, 3, 8, 124, 140, 3, 2, 2, 2, 125, 126, 12, 6, 2, 2, 126, 128, 5, 18, 10, 2, 127, 129, 5, 58, 30, 2, 128, 127, 3, 2, 2, 2, 128, 129, 3, 2, 2, 2, 129, 130, 3, 2, 2, 2, 130, 131, 5, 4, 3, 7, 131, 140, 3, 2, 2, 2, 132, 133, 12, 5, 2, 2, 133, 135, 5, 38, 20, 2, 134, 136, 5, 34, 18, 2, 135, 134, 3, 2, 2, 2, 135, 136, 3, 2, 2, 2, 136, 140, 3, 2, 2, 2, 137, 138, 12, 4, 2, 2, 138, 140, 5, 36, 19, 2, 139, 90, 3, 2, 2, 2, 139, 97, 3, 2, 2, 2, 139, 104, 3, 2, 2, 2, 139, 111, 3, 2, 2, 2, 139, 118, 3, 2, 2, 2, 139, 125, 3, 2, 2, 2, 139, 132, 3, 2, 2, 2, 139, 137, 3, 2, 2, 2, 140, 143, 3, 2, 2, 2, 141, 139, 3, 2, 2, 2, 141, 142, 3, 2, 2, 2, 142, 5, 3, 2, 2, 2, 143, 141, 3, 2, 2, 2, 144, 145, 9, 2, 2, 2, 145, 7, 3, 2, 2, 2, 146, 147, 7, 18, 2, 2, 147, 9, 3, 2, 2, 2, 148, 149, 9, 3, 2, 2, 149, 11, 3, 2, 2, 2, 150, 151, 9, 2, 2, 2, 151, 13, 3, 2, 2, 2, 152, 154, 9, 4, 2, 2, 153, 155, 7, 40, 2, 2, 154, 153, 3, 2, 2, 2, 154, 155, 3, 2, 2, 2, 155, 15, 3, 2, 2, 2, 156, 157, 9, 5, 2, 2, 157, 17, 3, 2, 2, 2, 158, 159, 7, 30, 2, 2, 159, 19, 3, 2, 2, 2, 160, 166, 5, 46, 24, 2, 161, 166, 5, 52, 27, 2, 162, 166, 5, 24, 13, 2, 163, 166, 5, 78, 40, 2, 164, 166, 5, 22, 12, 2, 165, 160, 3, 2, 2, 2, 165, 161, 3, 2, 2, 2, 165, 162, 3, 2, 2, 2, 165, 163, 3, 2, 2, 2, 165, 164, 3, 2, 2, 2, 166, 21, 3, 2, 2, 2, 167, 168, 7, 3, 2, 2, 168, 169, 5, 4, 3, 2, 169, 170, 7, 4, 2, 2, 170, 23, 3, 2, 2, 2, 171, 173, 5, 26, 14, 2, 172, 174, 5, 28, 15, 2, 173, 172, 3, 2, 2, 2, 173, 174, 3, 2, 2, 2, 174, 176, 3, 2, 2, 2, 175, 177, 5, 34, 18, 2, 176, 175, 3, 2, 2, 2, 176, 177, 3, 2, 2, 2, 177, 25, 3, 2, 2, 2, 178, 184, 5, 68, 35, 2, 179, 181, 7, 5, 2, 2, 180, 182, 5, 44, 23, 2, 181, 180, 3, 2, 2, 2, 181, 182, 3, 2, 2, 2, 182, 183, 3, 2, 2, 2, 183, 185, 7, 6, 2, 2, 184, 179, 3, 2, 2, 2, 184, 185, 3, 2, 2, 2, 185, 191, 3, 2, 2, 2, 186, 187, 7, 5, 2, 2, 187, 188, 5, 44, 23, 2, 188, 189, 7, 6, 2, 2, 189, 191, 3, 2, 2, 2, 190, 178, 3, 2, 2, 2, 190, 186, 3, 2, 2, 2, 191, 27, 3, 2, 2, 2, 192, 193, 7, 7, 2, 2, 193, 194, 7, 44, 2, 2, 194, 195, 7, 8, 2, 2, 195, 29, 3, 2, 2, 2, 196, 197, 7, 38, 2, 2, 197, 198, 7, 44, 2, 2, 198, 31, 3, 2, 2, 2, 199, 200, 7, 28, 2, 2, 200, 201, 9, 6, 2, 2, 201, 33, 3, 2, 2, 2, 202, 211, 5, 30, 16, 2, 203, 211, 5, 32, 17, 2, 204, 205, 5, 30, 16, 2, 205, 206, 5, 32, 17, 2, 206, 211, 3, 2, 2, 2, 207, 208, 5, 32, 17, 2, 208, 209, 5, 30, 16, 2, 209, 211, 3, 2, 2, 2, 210, 202, 3, 2, 2, 2, 210, 203, 3, 2, 2, 2, 210, 204, 3, 2, 2, 2, 210, 207, 3, 2, 2, 2, 211, 35, 3, 2, 2, 2, 212, 213, 7, 39, 2, 2, 213, 214, 7, 11, 2, 2, 214, 37, 3, 2, 2, 2, 215, 216, 7, 7, 2, 2, 216, 217, 7, 44, 2, 2, 217, 219, 7, 9, 2, 2, 218, 220, 7, 44, 2, 2, 219, 218, 3, 2, 2, 2, 219, 220, 3, 2, 2, 2, 220, 221, 3, 2, 2, 2, 221, 222, 7, 8, 2, 2, 222, 39, 3, 2, 2, 2, 223, 224, 5, 72, 37, 2, 224, 225, 5, 42, 22, 2, 225, 226, 7, 12, 2, 2, 226, 41, 3, 2, 2, 2, 227, 228, 9, 7, 2, 2, 228, 43, 3, 2, 2, 2, 229, 234, 5, 40, 21, 2, 230, 231, 7, 10, 2, 2, 231, 233, 5, 40, 21, 2, 232, 230, 3, 2, 2, 2, 233, 236, 3, 2, 2, 2, 234, 232, 3, 2, 2, 2, 234, 235, 3, 2, 2, 2, 235, 45, 3, 2, 2, 2, 236, 234, 3, 2, 2, 2, 237, 238, 7, 45, 2, 2, 238, 239, 5, 50, 26, 2, 239, 47, 3, 2, 2, 2, 240, 243, 5, 78, 40, 2, 241, 243, 5, 4, 3, 2, 242, 240, 3, 2, 2, 2, 242, 241, 3, 2, 2, 2, 243, 49, 3, 2, 2, 2, 244, 253, 7, 3, 2, 2, 245, 250, 5, 48, 25, 2, 246, 247, 7, 10, 2, 2, 247, 249, 5, 48, 25, 2, 248, 246, 3, 2, 2, 2, 249, 252, 3, 2, 2, 2, 250, 248, 3, 2, 2, 2, 250, 251, 3, 2, 2, 2, 251, 254, 3, 2, 2, 2, 252, 250, 3, 2, 2, 2, 253, 245, 3, 2, 2, 2, 253, 254, 3, 2, 2, 2, 254, 255, 3, 2, 2, 2, 255, 256, 7, 4, 2, 2, 256, 51, 3, 2, 2, 2, 257, 258, 7, 43, 2, 2, 258, 273, 5, 50, 26, 2, 259, 262, 7, 43, 2, 2, 260, 263, 5, 54, 28, 2, 261, 263, 5, 56, 29, 2, 262, 260, 3, 2, 2, 2, 262, 261, 3, 2, 2, 2, 263, 264, 3, 2, 2, 2, 264, 265, 5, 50, 26, 2, 265, 273, 3, 2, 2, 2, 266, 267, 7, 43, 2, 2, 267, 270, 5, 50, 26, 2, 268, 271, 5, 54, 28, 2, 269, 271, 5, 56, 29, 2, 270, 268, 3, 2, 2, 2, 270, 269, 3, 2, 2, 2, 271, 273, 3, 2, 2, 2, 272, 257, 3, 2, 2, 2, 272, 259, 3, 2, 2, 2, 272, 266, 3, 2, 2, 2, 273, 53, 3, 2, 2, 2, 274, 275, 7, 32, 2, 2, 275, 276, 5, 74, 38, 2, 276, 55, 3, 2, 2, 2, 277, 278, 7, 33, 2, 2, 278, 279, 5, 74, 38, 2, 279, 57, 3, 2, 2, 2, 280, 283, 5, 60, 31, 2, 281, 283, 5, 62, 32, 2, 282, 280, 3, 2, 2, 2, 282, 281, 3, 2, 2, 2, 283, 286, 3, 2, 2, 2, 284, 287, 5, 64, 33, 2, 285, 287, 5, 66, 34, 2, 286, 284, 3, 2, 2, 2, 286, 285, 3, 2, 2, 2, 286, 287, 3, 2, 2, 2, 287, 59, 3, 2, 2, 2, 288, 289, 7, 34, 2, 2, 289, 290, 5, 74, 38, 2, 290, 61, 3, 2, 2, 2, 291, 292, 7, 35, 2, 2, 292, 293, 5, 74, 38, 2, 293, 63, 3, 2, 2, 2, 294, 296, 7, 36, 2, 2, 295, 297, 5, 74, 38, 2, 296, 295, 3, 2, 2, 2, 296, 297, 3, 2, 2, 2, 297, 65, 3, 2, 2, 2, 298, 300, 7, 37, 2, 2, 299, 301, 5, 74, 38, 2, 300, 299, 3, 2, 2, 2, 300, 301, 3, 2, 2, 2, 301, 67, 3, 2, 2, 2, 302, 306, 5, 70, 36, 2, 303, 306, 7, 45, 2, 2, 304, 306, 7, 46, 2, 2, 305, 302, 3, 2, 2, 2, 305, 303, 3, 2, 2, 2, 305, 304, 3, 2, 2, 2, 306, 69, 3, 2, 2, 2, 307, 308, 9, 8, 2, 2, 308, 71, 3, 2, 2, 2, 309, 312, 5, 76, 39, 2, 310, 312, 7, 45, 2, 2, 311, 309, 3, 2, 2, 2, 311, 310, 3, 2, 2, 2, 312, 73, 3, 2, 2, 2, 313, 322, 7, 3, 2, 2, 314, 319, 5, 72, 37, 2, 315, 316, 7, 10, 2, 2, 316, 318, 5, 72, 37, 2, 317, 315, 3, 2, 2, 2, 318, 321, 3, 2, 2, 2, 319, 317, 3, 2, 2, 2, 319, 320, 3, 2, 2, 2, 320, 323, 3, 2, 2, 2, 321, 319, 3, 2, 2, 2, 322, 314, 3, 2, 2, 2, 322, 323, 3, 2, 2, 2, 323, 325, 3, 2, 2, 2, 324, 326, 7, 10, 2, 2, 325, 324, 3, 2, 2, 2, 325, 326, 3, 2, 2, 2, 326, 327, 3, 2, 2, 2, 327, 328, 7, 4, 2, 2, 328, 75, 3, 2, 2, 2, 329, 330, 9, 9, 2, 2, 330, 77, 3, 2, 2, 2, 331, 332, 9, 10, 2, 2, 332, 79, 3, 2, 2, 2, 37, 88, 93, 100, 107, 114, 121, 128, 135, 139, 141, 154, 165, 173, 176, 181, 184, 190, 210, 219, 234, 242, 250, 253, 262, 270, 272, 282, 286, 296, 300, 305, 311, 319, 322, 325]
\ No newline at end of file
diff --git a/prometheus/src/main/java/filodb/prometheus/antlr/PromQL.tokens b/prometheus/src/main/java/filodb/prometheus/antlr/PromQL.tokens
index 4a4589c50f..993fc6e637 100644
--- a/prometheus/src/main/java/filodb/prometheus/antlr/PromQL.tokens
+++ b/prometheus/src/main/java/filodb/prometheus/antlr/PromQL.tokens
@@ -23,24 +23,27 @@ GE=22
LE=23
RE=24
NRE=25
-AND=26
-OR=27
-UNLESS=28
-BY=29
-WITHOUT=30
-ON=31
-IGNORING=32
-GROUP_LEFT=33
-GROUP_RIGHT=34
-OFFSET=35
-LIMIT=36
-BOOL=37
-AGGREGATION_OP=38
-DURATION=39
-IDENTIFIER=40
-IDENTIFIER_EXTENDED=41
-WS=42
-COMMENT=43
+AT=26
+AND=27
+OR=28
+UNLESS=29
+BY=30
+WITHOUT=31
+ON=32
+IGNORING=33
+GROUP_LEFT=34
+GROUP_RIGHT=35
+OFFSET=36
+LIMIT=37
+BOOL=38
+START=39
+END=40
+AGGREGATION_OP=41
+DURATION=42
+IDENTIFIER=43
+IDENTIFIER_EXTENDED=44
+WS=45
+COMMENT=46
'('=1
')'=2
'{'=3
@@ -64,3 +67,4 @@ COMMENT=43
'<='=23
'=~'=24
'!~'=25
+'@'=26
diff --git a/prometheus/src/main/java/filodb/prometheus/antlr/PromQLBaseVisitor.java b/prometheus/src/main/java/filodb/prometheus/antlr/PromQLBaseVisitor.java
index 2cbb681816..8d82e55a76 100644
--- a/prometheus/src/main/java/filodb/prometheus/antlr/PromQLBaseVisitor.java
+++ b/prometheus/src/main/java/filodb/prometheus/antlr/PromQLBaseVisitor.java
@@ -1,4 +1,4 @@
-// Generated from PromQL.g4 by ANTLR 4.9.3
+// Generated from PromQL.g4 by ANTLR 4.9.1
package filodb.prometheus.antlr;
import org.antlr.v4.runtime.tree.AbstractParseTreeVisitor;
@@ -144,6 +144,20 @@ public class PromQLBaseVisitor
The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.
+ */ + @Override public T visitAtModifier(PromQLParser.AtModifierContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.
+ */ + @Override public T visitModifier(PromQLParser.ModifierContext ctx) { return visitChildren(ctx); } /** * {@inheritDoc} * diff --git a/prometheus/src/main/java/filodb/prometheus/antlr/PromQLLexer.interp b/prometheus/src/main/java/filodb/prometheus/antlr/PromQLLexer.interp index e8e27fcc1f..85b41731e3 100644 --- a/prometheus/src/main/java/filodb/prometheus/antlr/PromQLLexer.interp +++ b/prometheus/src/main/java/filodb/prometheus/antlr/PromQLLexer.interp @@ -25,6 +25,9 @@ null '<=' '=~' '!~' +'@' +null +null null null null @@ -71,6 +74,7 @@ GE LE RE NRE +AT AND OR UNLESS @@ -83,6 +87,8 @@ GROUP_RIGHT OFFSET LIMIT BOOL +START +END AGGREGATION_OP DURATION IDENTIFIER @@ -116,6 +122,7 @@ GE LE RE NRE +AT AND OR UNLESS @@ -128,6 +135,8 @@ GROUP_RIGHT OFFSET LIMIT BOOL +START +END AGGREGATION_OP DURATION IDENTIFIER @@ -169,4 +178,4 @@ mode names: DEFAULT_MODE atn: -[3, 24715, 42794, 33075, 47597, 16764, 15335, 30598, 22884, 2, 45, 518, 8, 1, 4, 2, 9, 2, 4, 3, 9, 3, 4, 4, 9, 4, 4, 5, 9, 5, 4, 6, 9, 6, 4, 7, 9, 7, 4, 8, 9, 8, 4, 9, 9, 9, 4, 10, 9, 10, 4, 11, 9, 11, 4, 12, 9, 12, 4, 13, 9, 13, 4, 14, 9, 14, 4, 15, 9, 15, 4, 16, 9, 16, 4, 17, 9, 17, 4, 18, 9, 18, 4, 19, 9, 19, 4, 20, 9, 20, 4, 21, 9, 21, 4, 22, 9, 22, 4, 23, 9, 23, 4, 24, 9, 24, 4, 25, 9, 25, 4, 26, 9, 26, 4, 27, 9, 27, 4, 28, 9, 28, 4, 29, 9, 29, 4, 30, 9, 30, 4, 31, 9, 31, 4, 32, 9, 32, 4, 33, 9, 33, 4, 34, 9, 34, 4, 35, 9, 35, 4, 36, 9, 36, 4, 37, 9, 37, 4, 38, 9, 38, 4, 39, 9, 39, 4, 40, 9, 40, 4, 41, 9, 41, 4, 42, 9, 42, 4, 43, 9, 43, 4, 44, 9, 44, 4, 45, 9, 45, 4, 46, 9, 46, 4, 47, 9, 47, 4, 48, 9, 48, 4, 49, 9, 49, 4, 50, 9, 50, 4, 51, 9, 51, 4, 52, 9, 52, 4, 53, 9, 53, 4, 54, 9, 54, 4, 55, 9, 55, 4, 56, 9, 56, 4, 57, 9, 57, 4, 58, 9, 58, 4, 59, 9, 59, 4, 60, 9, 60, 4, 61, 9, 61, 4, 62, 9, 62, 4, 63, 9, 63, 4, 64, 9, 64, 4, 65, 9, 65, 4, 66, 9, 66, 4, 67, 9, 67, 4, 68, 9, 68, 4, 69, 9, 69, 4, 70, 9, 70, 3, 2, 3, 2, 3, 3, 3, 3, 3, 4, 3, 4, 3, 5, 3, 5, 3, 6, 3, 6, 3, 7, 3, 7, 3, 8, 3, 8, 3, 9, 3, 9, 3, 10, 7, 10, 159, 10, 10, 12, 10, 14, 10, 162, 11, 10, 3, 10, 5, 10, 165, 10, 10, 3, 10, 6, 10, 168, 10, 10, 13, 10, 14, 10, 169, 3, 10, 3, 10, 5, 10, 174, 10, 10, 3, 10, 6, 10, 177, 10, 10, 13, 10, 14, 10, 178, 5, 10, 181, 10, 10, 3, 10, 6, 10, 184, 10, 10, 13, 10, 14, 10, 185, 3, 10, 3, 10, 3, 10, 3, 10, 6, 10, 192, 10, 10, 13, 10, 14, 10, 193, 5, 10, 196, 10, 10, 3, 11, 3, 11, 3, 11, 3, 11, 7, 11, 202, 10, 11, 12, 11, 14, 11, 205, 11, 11, 3, 11, 3, 11, 3, 11, 3, 11, 3, 11, 7, 11, 212, 10, 11, 12, 11, 14, 11, 215, 11, 11, 3, 11, 5, 11, 218, 10, 11, 3, 12, 3, 12, 3, 13, 3, 13, 3, 14, 3, 14, 3, 15, 3, 15, 3, 16, 3, 16, 3, 17, 3, 17, 3, 18, 3, 18, 3, 19, 3, 19, 3, 19, 3, 20, 3, 20, 3, 20, 3, 21, 3, 21, 3, 22, 3, 22, 3, 23, 3, 23, 3, 23, 3, 24, 3, 24, 3, 24, 3, 25, 3, 25, 3, 25, 3, 26, 3, 26, 3, 26, 3, 27, 3, 27, 3, 27, 3, 27, 3, 28, 3, 28, 3, 28, 3, 29, 3, 29, 3, 29, 3, 29, 3, 29, 3, 29, 3, 29, 3, 30, 3, 30, 3, 30, 3, 31, 3, 31, 3, 31, 3, 31, 3, 31, 3, 31, 3, 31, 3, 31, 3, 32, 3, 32, 3, 32, 3, 33, 3, 33, 3, 33, 3, 33, 3, 33, 3, 33, 3, 33, 3, 33, 3, 33, 3, 34, 3, 34, 3, 34, 3, 34, 3, 34, 3, 34, 3, 34, 3, 34, 3, 34, 3, 34, 3, 34, 3, 35, 3, 35, 3, 35, 3, 35, 3, 35, 3, 35, 3, 35, 3, 35, 3, 35, 3, 35, 3, 35, 3, 35, 3, 36, 3, 36, 3, 36, 3, 36, 3, 36, 3, 36, 3, 36, 3, 37, 3, 37, 3, 37, 3, 37, 3, 37, 3, 37, 3, 38, 3, 38, 3, 38, 3, 38, 3, 38, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 5, 39, 417, 10, 39, 3, 40, 3, 40, 3, 40, 6, 40, 422, 10, 40, 13, 40, 14, 40, 423, 3, 40, 3, 40, 3, 40, 5, 40, 429, 10, 40, 3, 41, 3, 41, 7, 41, 433, 10, 41, 12, 41, 14, 41, 436, 11, 41, 3, 42, 7, 42, 439, 10, 42, 12, 42, 14, 42, 442, 11, 42, 3, 42, 3, 42, 7, 42, 446, 10, 42, 12, 42, 14, 42, 449, 11, 42, 3, 43, 3, 43, 3, 44, 3, 44, 3, 45, 3, 45, 3, 46, 3, 46, 3, 47, 3, 47, 3, 48, 3, 48, 3, 49, 3, 49, 3, 50, 3, 50, 3, 51, 3, 51, 3, 52, 3, 52, 3, 53, 3, 53, 3, 54, 3, 54, 3, 55, 3, 55, 3, 56, 3, 56, 3, 57, 3, 57, 3, 58, 3, 58, 3, 59, 3, 59, 3, 60, 3, 60, 3, 61, 3, 61, 3, 62, 3, 62, 3, 63, 3, 63, 3, 64, 3, 64, 3, 65, 3, 65, 3, 66, 3, 66, 3, 67, 3, 67, 3, 68, 3, 68, 3, 69, 6, 69, 504, 10, 69, 13, 69, 14, 69, 505, 3, 69, 3, 69, 3, 70, 3, 70, 7, 70, 512, 10, 70, 12, 70, 14, 70, 515, 11, 70, 3, 70, 3, 70, 2, 2, 71, 3, 3, 5, 4, 7, 5, 9, 6, 11, 7, 13, 8, 15, 9, 17, 10, 19, 11, 21, 12, 23, 13, 25, 14, 27, 15, 29, 16, 31, 17, 33, 18, 35, 19, 37, 20, 39, 21, 41, 22, 43, 23, 45, 24, 47, 25, 49, 26, 51, 27, 53, 28, 55, 29, 57, 30, 59, 31, 61, 32, 63, 33, 65, 34, 67, 35, 69, 36, 71, 37, 73, 38, 75, 39, 77, 40, 79, 41, 81, 42, 83, 43, 85, 2, 87, 2, 89, 2, 91, 2, 93, 2, 95, 2, 97, 2, 99, 2, 101, 2, 103, 2, 105, 2, 107, 2, 109, 2, 111, 2, 113, 2, 115, 2, 117, 2, 119, 2, 121, 2, 123, 2, 125, 2, 127, 2, 129, 2, 131, 2, 133, 2, 135, 2, 137, 44, 139, 45, 3, 2, 41, 3, 2, 50, 59, 4, 2, 71, 71, 103, 103, 4, 2, 45, 45, 47, 47, 4, 2, 90, 90, 122, 122, 5, 2, 50, 59, 67, 72, 99, 104, 4, 2, 41, 41, 94, 94, 4, 2, 36, 36, 94, 94, 8, 2, 102, 102, 106, 106, 111, 111, 117, 117, 121, 121, 123, 123, 5, 2, 67, 92, 97, 97, 99, 124, 6, 2, 50, 59, 67, 92, 97, 97, 99, 124, 4, 2, 60, 60, 97, 97, 4, 2, 67, 92, 99, 124, 7, 2, 47, 48, 50, 60, 67, 92, 97, 97, 99, 124, 4, 2, 67, 67, 99, 99, 4, 2, 68, 68, 100, 100, 4, 2, 69, 69, 101, 101, 4, 2, 70, 70, 102, 102, 4, 2, 72, 72, 104, 104, 4, 2, 73, 73, 105, 105, 4, 2, 74, 74, 106, 106, 4, 2, 75, 75, 107, 107, 4, 2, 76, 76, 108, 108, 4, 2, 77, 77, 109, 109, 4, 2, 78, 78, 110, 110, 4, 2, 79, 79, 111, 111, 4, 2, 80, 80, 112, 112, 4, 2, 81, 81, 113, 113, 4, 2, 82, 82, 114, 114, 4, 2, 83, 83, 115, 115, 4, 2, 84, 84, 116, 116, 4, 2, 85, 85, 117, 117, 4, 2, 86, 86, 118, 118, 4, 2, 87, 87, 119, 119, 4, 2, 88, 88, 120, 120, 4, 2, 89, 89, 121, 121, 4, 2, 91, 91, 123, 123, 4, 2, 92, 92, 124, 124, 5, 2, 11, 12, 15, 15, 34, 34, 4, 2, 12, 12, 15, 15, 2, 525, 2, 3, 3, 2, 2, 2, 2, 5, 3, 2, 2, 2, 2, 7, 3, 2, 2, 2, 2, 9, 3, 2, 2, 2, 2, 11, 3, 2, 2, 2, 2, 13, 3, 2, 2, 2, 2, 15, 3, 2, 2, 2, 2, 17, 3, 2, 2, 2, 2, 19, 3, 2, 2, 2, 2, 21, 3, 2, 2, 2, 2, 23, 3, 2, 2, 2, 2, 25, 3, 2, 2, 2, 2, 27, 3, 2, 2, 2, 2, 29, 3, 2, 2, 2, 2, 31, 3, 2, 2, 2, 2, 33, 3, 2, 2, 2, 2, 35, 3, 2, 2, 2, 2, 37, 3, 2, 2, 2, 2, 39, 3, 2, 2, 2, 2, 41, 3, 2, 2, 2, 2, 43, 3, 2, 2, 2, 2, 45, 3, 2, 2, 2, 2, 47, 3, 2, 2, 2, 2, 49, 3, 2, 2, 2, 2, 51, 3, 2, 2, 2, 2, 53, 3, 2, 2, 2, 2, 55, 3, 2, 2, 2, 2, 57, 3, 2, 2, 2, 2, 59, 3, 2, 2, 2, 2, 61, 3, 2, 2, 2, 2, 63, 3, 2, 2, 2, 2, 65, 3, 2, 2, 2, 2, 67, 3, 2, 2, 2, 2, 69, 3, 2, 2, 2, 2, 71, 3, 2, 2, 2, 2, 73, 3, 2, 2, 2, 2, 75, 3, 2, 2, 2, 2, 77, 3, 2, 2, 2, 2, 79, 3, 2, 2, 2, 2, 81, 3, 2, 2, 2, 2, 83, 3, 2, 2, 2, 2, 137, 3, 2, 2, 2, 2, 139, 3, 2, 2, 2, 3, 141, 3, 2, 2, 2, 5, 143, 3, 2, 2, 2, 7, 145, 3, 2, 2, 2, 9, 147, 3, 2, 2, 2, 11, 149, 3, 2, 2, 2, 13, 151, 3, 2, 2, 2, 15, 153, 3, 2, 2, 2, 17, 155, 3, 2, 2, 2, 19, 195, 3, 2, 2, 2, 21, 217, 3, 2, 2, 2, 23, 219, 3, 2, 2, 2, 25, 221, 3, 2, 2, 2, 27, 223, 3, 2, 2, 2, 29, 225, 3, 2, 2, 2, 31, 227, 3, 2, 2, 2, 33, 229, 3, 2, 2, 2, 35, 231, 3, 2, 2, 2, 37, 233, 3, 2, 2, 2, 39, 236, 3, 2, 2, 2, 41, 239, 3, 2, 2, 2, 43, 241, 3, 2, 2, 2, 45, 243, 3, 2, 2, 2, 47, 246, 3, 2, 2, 2, 49, 249, 3, 2, 2, 2, 51, 252, 3, 2, 2, 2, 53, 255, 3, 2, 2, 2, 55, 259, 3, 2, 2, 2, 57, 262, 3, 2, 2, 2, 59, 269, 3, 2, 2, 2, 61, 272, 3, 2, 2, 2, 63, 280, 3, 2, 2, 2, 65, 283, 3, 2, 2, 2, 67, 292, 3, 2, 2, 2, 69, 303, 3, 2, 2, 2, 71, 315, 3, 2, 2, 2, 73, 322, 3, 2, 2, 2, 75, 328, 3, 2, 2, 2, 77, 416, 3, 2, 2, 2, 79, 428, 3, 2, 2, 2, 81, 430, 3, 2, 2, 2, 83, 440, 3, 2, 2, 2, 85, 450, 3, 2, 2, 2, 87, 452, 3, 2, 2, 2, 89, 454, 3, 2, 2, 2, 91, 456, 3, 2, 2, 2, 93, 458, 3, 2, 2, 2, 95, 460, 3, 2, 2, 2, 97, 462, 3, 2, 2, 2, 99, 464, 3, 2, 2, 2, 101, 466, 3, 2, 2, 2, 103, 468, 3, 2, 2, 2, 105, 470, 3, 2, 2, 2, 107, 472, 3, 2, 2, 2, 109, 474, 3, 2, 2, 2, 111, 476, 3, 2, 2, 2, 113, 478, 3, 2, 2, 2, 115, 480, 3, 2, 2, 2, 117, 482, 3, 2, 2, 2, 119, 484, 3, 2, 2, 2, 121, 486, 3, 2, 2, 2, 123, 488, 3, 2, 2, 2, 125, 490, 3, 2, 2, 2, 127, 492, 3, 2, 2, 2, 129, 494, 3, 2, 2, 2, 131, 496, 3, 2, 2, 2, 133, 498, 3, 2, 2, 2, 135, 500, 3, 2, 2, 2, 137, 503, 3, 2, 2, 2, 139, 509, 3, 2, 2, 2, 141, 142, 7, 42, 2, 2, 142, 4, 3, 2, 2, 2, 143, 144, 7, 43, 2, 2, 144, 6, 3, 2, 2, 2, 145, 146, 7, 125, 2, 2, 146, 8, 3, 2, 2, 2, 147, 148, 7, 127, 2, 2, 148, 10, 3, 2, 2, 2, 149, 150, 7, 93, 2, 2, 150, 12, 3, 2, 2, 2, 151, 152, 7, 95, 2, 2, 152, 14, 3, 2, 2, 2, 153, 154, 7, 60, 2, 2, 154, 16, 3, 2, 2, 2, 155, 156, 7, 46, 2, 2, 156, 18, 3, 2, 2, 2, 157, 159, 9, 2, 2, 2, 158, 157, 3, 2, 2, 2, 159, 162, 3, 2, 2, 2, 160, 158, 3, 2, 2, 2, 160, 161, 3, 2, 2, 2, 161, 164, 3, 2, 2, 2, 162, 160, 3, 2, 2, 2, 163, 165, 7, 48, 2, 2, 164, 163, 3, 2, 2, 2, 164, 165, 3, 2, 2, 2, 165, 167, 3, 2, 2, 2, 166, 168, 9, 2, 2, 2, 167, 166, 3, 2, 2, 2, 168, 169, 3, 2, 2, 2, 169, 167, 3, 2, 2, 2, 169, 170, 3, 2, 2, 2, 170, 180, 3, 2, 2, 2, 171, 173, 9, 3, 2, 2, 172, 174, 9, 4, 2, 2, 173, 172, 3, 2, 2, 2, 173, 174, 3, 2, 2, 2, 174, 176, 3, 2, 2, 2, 175, 177, 9, 2, 2, 2, 176, 175, 3, 2, 2, 2, 177, 178, 3, 2, 2, 2, 178, 176, 3, 2, 2, 2, 178, 179, 3, 2, 2, 2, 179, 181, 3, 2, 2, 2, 180, 171, 3, 2, 2, 2, 180, 181, 3, 2, 2, 2, 181, 196, 3, 2, 2, 2, 182, 184, 9, 2, 2, 2, 183, 182, 3, 2, 2, 2, 184, 185, 3, 2, 2, 2, 185, 183, 3, 2, 2, 2, 185, 186, 3, 2, 2, 2, 186, 187, 3, 2, 2, 2, 187, 196, 7, 48, 2, 2, 188, 189, 7, 50, 2, 2, 189, 191, 9, 5, 2, 2, 190, 192, 9, 6, 2, 2, 191, 190, 3, 2, 2, 2, 192, 193, 3, 2, 2, 2, 193, 191, 3, 2, 2, 2, 193, 194, 3, 2, 2, 2, 194, 196, 3, 2, 2, 2, 195, 160, 3, 2, 2, 2, 195, 183, 3, 2, 2, 2, 195, 188, 3, 2, 2, 2, 196, 20, 3, 2, 2, 2, 197, 203, 7, 41, 2, 2, 198, 202, 10, 7, 2, 2, 199, 200, 7, 94, 2, 2, 200, 202, 11, 2, 2, 2, 201, 198, 3, 2, 2, 2, 201, 199, 3, 2, 2, 2, 202, 205, 3, 2, 2, 2, 203, 201, 3, 2, 2, 2, 203, 204, 3, 2, 2, 2, 204, 206, 3, 2, 2, 2, 205, 203, 3, 2, 2, 2, 206, 218, 7, 41, 2, 2, 207, 213, 7, 36, 2, 2, 208, 212, 10, 8, 2, 2, 209, 210, 7, 94, 2, 2, 210, 212, 11, 2, 2, 2, 211, 208, 3, 2, 2, 2, 211, 209, 3, 2, 2, 2, 212, 215, 3, 2, 2, 2, 213, 211, 3, 2, 2, 2, 213, 214, 3, 2, 2, 2, 214, 216, 3, 2, 2, 2, 215, 213, 3, 2, 2, 2, 216, 218, 7, 36, 2, 2, 217, 197, 3, 2, 2, 2, 217, 207, 3, 2, 2, 2, 218, 22, 3, 2, 2, 2, 219, 220, 7, 45, 2, 2, 220, 24, 3, 2, 2, 2, 221, 222, 7, 47, 2, 2, 222, 26, 3, 2, 2, 2, 223, 224, 7, 44, 2, 2, 224, 28, 3, 2, 2, 2, 225, 226, 7, 49, 2, 2, 226, 30, 3, 2, 2, 2, 227, 228, 7, 39, 2, 2, 228, 32, 3, 2, 2, 2, 229, 230, 7, 96, 2, 2, 230, 34, 3, 2, 2, 2, 231, 232, 7, 63, 2, 2, 232, 36, 3, 2, 2, 2, 233, 234, 7, 63, 2, 2, 234, 235, 7, 63, 2, 2, 235, 38, 3, 2, 2, 2, 236, 237, 7, 35, 2, 2, 237, 238, 7, 63, 2, 2, 238, 40, 3, 2, 2, 2, 239, 240, 7, 64, 2, 2, 240, 42, 3, 2, 2, 2, 241, 242, 7, 62, 2, 2, 242, 44, 3, 2, 2, 2, 243, 244, 7, 64, 2, 2, 244, 245, 7, 63, 2, 2, 245, 46, 3, 2, 2, 2, 246, 247, 7, 62, 2, 2, 247, 248, 7, 63, 2, 2, 248, 48, 3, 2, 2, 2, 249, 250, 7, 63, 2, 2, 250, 251, 7, 128, 2, 2, 251, 50, 3, 2, 2, 2, 252, 253, 7, 35, 2, 2, 253, 254, 7, 128, 2, 2, 254, 52, 3, 2, 2, 2, 255, 256, 5, 85, 43, 2, 256, 257, 5, 111, 56, 2, 257, 258, 5, 91, 46, 2, 258, 54, 3, 2, 2, 2, 259, 260, 5, 113, 57, 2, 260, 261, 5, 119, 60, 2, 261, 56, 3, 2, 2, 2, 262, 263, 5, 125, 63, 2, 263, 264, 5, 111, 56, 2, 264, 265, 5, 107, 54, 2, 265, 266, 5, 93, 47, 2, 266, 267, 5, 121, 61, 2, 267, 268, 5, 121, 61, 2, 268, 58, 3, 2, 2, 2, 269, 270, 5, 87, 44, 2, 270, 271, 5, 133, 67, 2, 271, 60, 3, 2, 2, 2, 272, 273, 5, 129, 65, 2, 273, 274, 5, 101, 51, 2, 274, 275, 5, 123, 62, 2, 275, 276, 5, 99, 50, 2, 276, 277, 5, 113, 57, 2, 277, 278, 5, 125, 63, 2, 278, 279, 5, 123, 62, 2, 279, 62, 3, 2, 2, 2, 280, 281, 5, 113, 57, 2, 281, 282, 5, 111, 56, 2, 282, 64, 3, 2, 2, 2, 283, 284, 5, 101, 51, 2, 284, 285, 5, 97, 49, 2, 285, 286, 5, 111, 56, 2, 286, 287, 5, 113, 57, 2, 287, 288, 5, 119, 60, 2, 288, 289, 5, 101, 51, 2, 289, 290, 5, 111, 56, 2, 290, 291, 5, 97, 49, 2, 291, 66, 3, 2, 2, 2, 292, 293, 5, 97, 49, 2, 293, 294, 5, 119, 60, 2, 294, 295, 5, 113, 57, 2, 295, 296, 5, 125, 63, 2, 296, 297, 5, 115, 58, 2, 297, 298, 7, 97, 2, 2, 298, 299, 5, 107, 54, 2, 299, 300, 5, 93, 47, 2, 300, 301, 5, 95, 48, 2, 301, 302, 5, 123, 62, 2, 302, 68, 3, 2, 2, 2, 303, 304, 5, 97, 49, 2, 304, 305, 5, 119, 60, 2, 305, 306, 5, 113, 57, 2, 306, 307, 5, 125, 63, 2, 307, 308, 5, 115, 58, 2, 308, 309, 7, 97, 2, 2, 309, 310, 5, 119, 60, 2, 310, 311, 5, 101, 51, 2, 311, 312, 5, 97, 49, 2, 312, 313, 5, 99, 50, 2, 313, 314, 5, 123, 62, 2, 314, 70, 3, 2, 2, 2, 315, 316, 5, 113, 57, 2, 316, 317, 5, 95, 48, 2, 317, 318, 5, 95, 48, 2, 318, 319, 5, 121, 61, 2, 319, 320, 5, 93, 47, 2, 320, 321, 5, 123, 62, 2, 321, 72, 3, 2, 2, 2, 322, 323, 5, 107, 54, 2, 323, 324, 5, 101, 51, 2, 324, 325, 5, 109, 55, 2, 325, 326, 5, 101, 51, 2, 326, 327, 5, 123, 62, 2, 327, 74, 3, 2, 2, 2, 328, 329, 5, 87, 44, 2, 329, 330, 5, 113, 57, 2, 330, 331, 5, 113, 57, 2, 331, 332, 5, 107, 54, 2, 332, 76, 3, 2, 2, 2, 333, 334, 5, 121, 61, 2, 334, 335, 5, 125, 63, 2, 335, 336, 5, 109, 55, 2, 336, 417, 3, 2, 2, 2, 337, 338, 5, 109, 55, 2, 338, 339, 5, 101, 51, 2, 339, 340, 5, 111, 56, 2, 340, 417, 3, 2, 2, 2, 341, 342, 5, 109, 55, 2, 342, 343, 5, 85, 43, 2, 343, 344, 5, 131, 66, 2, 344, 417, 3, 2, 2, 2, 345, 346, 5, 85, 43, 2, 346, 347, 5, 127, 64, 2, 347, 348, 5, 97, 49, 2, 348, 417, 3, 2, 2, 2, 349, 350, 5, 97, 49, 2, 350, 351, 5, 119, 60, 2, 351, 352, 5, 113, 57, 2, 352, 353, 5, 125, 63, 2, 353, 354, 5, 115, 58, 2, 354, 417, 3, 2, 2, 2, 355, 356, 5, 121, 61, 2, 356, 357, 5, 123, 62, 2, 357, 358, 5, 91, 46, 2, 358, 359, 5, 91, 46, 2, 359, 360, 5, 93, 47, 2, 360, 361, 5, 127, 64, 2, 361, 417, 3, 2, 2, 2, 362, 363, 5, 121, 61, 2, 363, 364, 5, 123, 62, 2, 364, 365, 5, 91, 46, 2, 365, 366, 5, 127, 64, 2, 366, 367, 5, 85, 43, 2, 367, 368, 5, 119, 60, 2, 368, 417, 3, 2, 2, 2, 369, 370, 5, 89, 45, 2, 370, 371, 5, 113, 57, 2, 371, 372, 5, 125, 63, 2, 372, 373, 5, 111, 56, 2, 373, 374, 5, 123, 62, 2, 374, 417, 3, 2, 2, 2, 375, 376, 5, 89, 45, 2, 376, 377, 5, 113, 57, 2, 377, 378, 5, 125, 63, 2, 378, 379, 5, 111, 56, 2, 379, 380, 5, 123, 62, 2, 380, 381, 7, 97, 2, 2, 381, 382, 5, 127, 64, 2, 382, 383, 5, 85, 43, 2, 383, 384, 5, 107, 54, 2, 384, 385, 5, 125, 63, 2, 385, 386, 5, 93, 47, 2, 386, 387, 5, 121, 61, 2, 387, 417, 3, 2, 2, 2, 388, 389, 5, 87, 44, 2, 389, 390, 5, 113, 57, 2, 390, 391, 5, 123, 62, 2, 391, 392, 5, 123, 62, 2, 392, 393, 5, 113, 57, 2, 393, 394, 5, 109, 55, 2, 394, 395, 5, 105, 53, 2, 395, 417, 3, 2, 2, 2, 396, 397, 5, 123, 62, 2, 397, 398, 5, 113, 57, 2, 398, 399, 5, 115, 58, 2, 399, 400, 5, 105, 53, 2, 400, 417, 3, 2, 2, 2, 401, 402, 5, 117, 59, 2, 402, 403, 5, 125, 63, 2, 403, 404, 5, 85, 43, 2, 404, 405, 5, 111, 56, 2, 405, 406, 5, 123, 62, 2, 406, 407, 5, 101, 51, 2, 407, 408, 5, 107, 54, 2, 408, 409, 5, 93, 47, 2, 409, 417, 3, 2, 2, 2, 410, 411, 5, 97, 49, 2, 411, 412, 5, 119, 60, 2, 412, 413, 5, 113, 57, 2, 413, 414, 5, 125, 63, 2, 414, 415, 5, 115, 58, 2, 415, 417, 3, 2, 2, 2, 416, 333, 3, 2, 2, 2, 416, 337, 3, 2, 2, 2, 416, 341, 3, 2, 2, 2, 416, 345, 3, 2, 2, 2, 416, 349, 3, 2, 2, 2, 416, 355, 3, 2, 2, 2, 416, 362, 3, 2, 2, 2, 416, 369, 3, 2, 2, 2, 416, 375, 3, 2, 2, 2, 416, 388, 3, 2, 2, 2, 416, 396, 3, 2, 2, 2, 416, 401, 3, 2, 2, 2, 416, 410, 3, 2, 2, 2, 417, 78, 3, 2, 2, 2, 418, 419, 5, 19, 10, 2, 419, 420, 9, 9, 2, 2, 420, 422, 3, 2, 2, 2, 421, 418, 3, 2, 2, 2, 422, 423, 3, 2, 2, 2, 423, 421, 3, 2, 2, 2, 423, 424, 3, 2, 2, 2, 424, 429, 3, 2, 2, 2, 425, 426, 5, 19, 10, 2, 426, 427, 7, 107, 2, 2, 427, 429, 3, 2, 2, 2, 428, 421, 3, 2, 2, 2, 428, 425, 3, 2, 2, 2, 429, 80, 3, 2, 2, 2, 430, 434, 9, 10, 2, 2, 431, 433, 9, 11, 2, 2, 432, 431, 3, 2, 2, 2, 433, 436, 3, 2, 2, 2, 434, 432, 3, 2, 2, 2, 434, 435, 3, 2, 2, 2, 435, 82, 3, 2, 2, 2, 436, 434, 3, 2, 2, 2, 437, 439, 9, 12, 2, 2, 438, 437, 3, 2, 2, 2, 439, 442, 3, 2, 2, 2, 440, 438, 3, 2, 2, 2, 440, 441, 3, 2, 2, 2, 441, 443, 3, 2, 2, 2, 442, 440, 3, 2, 2, 2, 443, 447, 9, 13, 2, 2, 444, 446, 9, 14, 2, 2, 445, 444, 3, 2, 2, 2, 446, 449, 3, 2, 2, 2, 447, 445, 3, 2, 2, 2, 447, 448, 3, 2, 2, 2, 448, 84, 3, 2, 2, 2, 449, 447, 3, 2, 2, 2, 450, 451, 9, 15, 2, 2, 451, 86, 3, 2, 2, 2, 452, 453, 9, 16, 2, 2, 453, 88, 3, 2, 2, 2, 454, 455, 9, 17, 2, 2, 455, 90, 3, 2, 2, 2, 456, 457, 9, 18, 2, 2, 457, 92, 3, 2, 2, 2, 458, 459, 9, 3, 2, 2, 459, 94, 3, 2, 2, 2, 460, 461, 9, 19, 2, 2, 461, 96, 3, 2, 2, 2, 462, 463, 9, 20, 2, 2, 463, 98, 3, 2, 2, 2, 464, 465, 9, 21, 2, 2, 465, 100, 3, 2, 2, 2, 466, 467, 9, 22, 2, 2, 467, 102, 3, 2, 2, 2, 468, 469, 9, 23, 2, 2, 469, 104, 3, 2, 2, 2, 470, 471, 9, 24, 2, 2, 471, 106, 3, 2, 2, 2, 472, 473, 9, 25, 2, 2, 473, 108, 3, 2, 2, 2, 474, 475, 9, 26, 2, 2, 475, 110, 3, 2, 2, 2, 476, 477, 9, 27, 2, 2, 477, 112, 3, 2, 2, 2, 478, 479, 9, 28, 2, 2, 479, 114, 3, 2, 2, 2, 480, 481, 9, 29, 2, 2, 481, 116, 3, 2, 2, 2, 482, 483, 9, 30, 2, 2, 483, 118, 3, 2, 2, 2, 484, 485, 9, 31, 2, 2, 485, 120, 3, 2, 2, 2, 486, 487, 9, 32, 2, 2, 487, 122, 3, 2, 2, 2, 488, 489, 9, 33, 2, 2, 489, 124, 3, 2, 2, 2, 490, 491, 9, 34, 2, 2, 491, 126, 3, 2, 2, 2, 492, 493, 9, 35, 2, 2, 493, 128, 3, 2, 2, 2, 494, 495, 9, 36, 2, 2, 495, 130, 3, 2, 2, 2, 496, 497, 9, 5, 2, 2, 497, 132, 3, 2, 2, 2, 498, 499, 9, 37, 2, 2, 499, 134, 3, 2, 2, 2, 500, 501, 9, 38, 2, 2, 501, 136, 3, 2, 2, 2, 502, 504, 9, 39, 2, 2, 503, 502, 3, 2, 2, 2, 504, 505, 3, 2, 2, 2, 505, 503, 3, 2, 2, 2, 505, 506, 3, 2, 2, 2, 506, 507, 3, 2, 2, 2, 507, 508, 8, 69, 2, 2, 508, 138, 3, 2, 2, 2, 509, 513, 7, 37, 2, 2, 510, 512, 10, 40, 2, 2, 511, 510, 3, 2, 2, 2, 512, 515, 3, 2, 2, 2, 513, 511, 3, 2, 2, 2, 513, 514, 3, 2, 2, 2, 514, 516, 3, 2, 2, 2, 515, 513, 3, 2, 2, 2, 516, 517, 8, 70, 2, 2, 517, 140, 3, 2, 2, 2, 25, 2, 160, 164, 169, 173, 178, 180, 185, 193, 195, 201, 203, 211, 213, 217, 416, 423, 428, 434, 440, 447, 505, 513, 3, 8, 2, 2] \ No newline at end of file +[3, 24715, 42794, 33075, 47597, 16764, 15335, 30598, 22884, 2, 48, 540, 8, 1, 4, 2, 9, 2, 4, 3, 9, 3, 4, 4, 9, 4, 4, 5, 9, 5, 4, 6, 9, 6, 4, 7, 9, 7, 4, 8, 9, 8, 4, 9, 9, 9, 4, 10, 9, 10, 4, 11, 9, 11, 4, 12, 9, 12, 4, 13, 9, 13, 4, 14, 9, 14, 4, 15, 9, 15, 4, 16, 9, 16, 4, 17, 9, 17, 4, 18, 9, 18, 4, 19, 9, 19, 4, 20, 9, 20, 4, 21, 9, 21, 4, 22, 9, 22, 4, 23, 9, 23, 4, 24, 9, 24, 4, 25, 9, 25, 4, 26, 9, 26, 4, 27, 9, 27, 4, 28, 9, 28, 4, 29, 9, 29, 4, 30, 9, 30, 4, 31, 9, 31, 4, 32, 9, 32, 4, 33, 9, 33, 4, 34, 9, 34, 4, 35, 9, 35, 4, 36, 9, 36, 4, 37, 9, 37, 4, 38, 9, 38, 4, 39, 9, 39, 4, 40, 9, 40, 4, 41, 9, 41, 4, 42, 9, 42, 4, 43, 9, 43, 4, 44, 9, 44, 4, 45, 9, 45, 4, 46, 9, 46, 4, 47, 9, 47, 4, 48, 9, 48, 4, 49, 9, 49, 4, 50, 9, 50, 4, 51, 9, 51, 4, 52, 9, 52, 4, 53, 9, 53, 4, 54, 9, 54, 4, 55, 9, 55, 4, 56, 9, 56, 4, 57, 9, 57, 4, 58, 9, 58, 4, 59, 9, 59, 4, 60, 9, 60, 4, 61, 9, 61, 4, 62, 9, 62, 4, 63, 9, 63, 4, 64, 9, 64, 4, 65, 9, 65, 4, 66, 9, 66, 4, 67, 9, 67, 4, 68, 9, 68, 4, 69, 9, 69, 4, 70, 9, 70, 4, 71, 9, 71, 4, 72, 9, 72, 4, 73, 9, 73, 3, 2, 3, 2, 3, 3, 3, 3, 3, 4, 3, 4, 3, 5, 3, 5, 3, 6, 3, 6, 3, 7, 3, 7, 3, 8, 3, 8, 3, 9, 3, 9, 3, 10, 7, 10, 165, 10, 10, 12, 10, 14, 10, 168, 11, 10, 3, 10, 5, 10, 171, 10, 10, 3, 10, 6, 10, 174, 10, 10, 13, 10, 14, 10, 175, 3, 10, 3, 10, 5, 10, 180, 10, 10, 3, 10, 6, 10, 183, 10, 10, 13, 10, 14, 10, 184, 5, 10, 187, 10, 10, 3, 10, 6, 10, 190, 10, 10, 13, 10, 14, 10, 191, 3, 10, 3, 10, 3, 10, 3, 10, 6, 10, 198, 10, 10, 13, 10, 14, 10, 199, 5, 10, 202, 10, 10, 3, 11, 3, 11, 3, 11, 3, 11, 7, 11, 208, 10, 11, 12, 11, 14, 11, 211, 11, 11, 3, 11, 3, 11, 3, 11, 3, 11, 3, 11, 7, 11, 218, 10, 11, 12, 11, 14, 11, 221, 11, 11, 3, 11, 5, 11, 224, 10, 11, 3, 12, 3, 12, 3, 13, 3, 13, 3, 14, 3, 14, 3, 15, 3, 15, 3, 16, 3, 16, 3, 17, 3, 17, 3, 18, 3, 18, 3, 19, 3, 19, 3, 19, 3, 20, 3, 20, 3, 20, 3, 21, 3, 21, 3, 22, 3, 22, 3, 23, 3, 23, 3, 23, 3, 24, 3, 24, 3, 24, 3, 25, 3, 25, 3, 25, 3, 26, 3, 26, 3, 26, 3, 27, 3, 27, 3, 28, 3, 28, 3, 28, 3, 28, 3, 29, 3, 29, 3, 29, 3, 30, 3, 30, 3, 30, 3, 30, 3, 30, 3, 30, 3, 30, 3, 31, 3, 31, 3, 31, 3, 32, 3, 32, 3, 32, 3, 32, 3, 32, 3, 32, 3, 32, 3, 32, 3, 33, 3, 33, 3, 33, 3, 34, 3, 34, 3, 34, 3, 34, 3, 34, 3, 34, 3, 34, 3, 34, 3, 34, 3, 35, 3, 35, 3, 35, 3, 35, 3, 35, 3, 35, 3, 35, 3, 35, 3, 35, 3, 35, 3, 35, 3, 36, 3, 36, 3, 36, 3, 36, 3, 36, 3, 36, 3, 36, 3, 36, 3, 36, 3, 36, 3, 36, 3, 36, 3, 37, 3, 37, 3, 37, 3, 37, 3, 37, 3, 37, 3, 37, 3, 38, 3, 38, 3, 38, 3, 38, 3, 38, 3, 38, 3, 39, 3, 39, 3, 39, 3, 39, 3, 39, 3, 40, 3, 40, 3, 40, 3, 40, 3, 40, 3, 40, 3, 40, 3, 40, 3, 41, 3, 41, 3, 41, 3, 41, 3, 41, 3, 41, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 3, 42, 5, 42, 439, 10, 42, 3, 43, 3, 43, 3, 43, 6, 43, 444, 10, 43, 13, 43, 14, 43, 445, 3, 43, 3, 43, 3, 43, 5, 43, 451, 10, 43, 3, 44, 3, 44, 7, 44, 455, 10, 44, 12, 44, 14, 44, 458, 11, 44, 3, 45, 7, 45, 461, 10, 45, 12, 45, 14, 45, 464, 11, 45, 3, 45, 3, 45, 7, 45, 468, 10, 45, 12, 45, 14, 45, 471, 11, 45, 3, 46, 3, 46, 3, 47, 3, 47, 3, 48, 3, 48, 3, 49, 3, 49, 3, 50, 3, 50, 3, 51, 3, 51, 3, 52, 3, 52, 3, 53, 3, 53, 3, 54, 3, 54, 3, 55, 3, 55, 3, 56, 3, 56, 3, 57, 3, 57, 3, 58, 3, 58, 3, 59, 3, 59, 3, 60, 3, 60, 3, 61, 3, 61, 3, 62, 3, 62, 3, 63, 3, 63, 3, 64, 3, 64, 3, 65, 3, 65, 3, 66, 3, 66, 3, 67, 3, 67, 3, 68, 3, 68, 3, 69, 3, 69, 3, 70, 3, 70, 3, 71, 3, 71, 3, 72, 6, 72, 526, 10, 72, 13, 72, 14, 72, 527, 3, 72, 3, 72, 3, 73, 3, 73, 7, 73, 534, 10, 73, 12, 73, 14, 73, 537, 11, 73, 3, 73, 3, 73, 2, 2, 74, 3, 3, 5, 4, 7, 5, 9, 6, 11, 7, 13, 8, 15, 9, 17, 10, 19, 11, 21, 12, 23, 13, 25, 14, 27, 15, 29, 16, 31, 17, 33, 18, 35, 19, 37, 20, 39, 21, 41, 22, 43, 23, 45, 24, 47, 25, 49, 26, 51, 27, 53, 28, 55, 29, 57, 30, 59, 31, 61, 32, 63, 33, 65, 34, 67, 35, 69, 36, 71, 37, 73, 38, 75, 39, 77, 40, 79, 41, 81, 42, 83, 43, 85, 44, 87, 45, 89, 46, 91, 2, 93, 2, 95, 2, 97, 2, 99, 2, 101, 2, 103, 2, 105, 2, 107, 2, 109, 2, 111, 2, 113, 2, 115, 2, 117, 2, 119, 2, 121, 2, 123, 2, 125, 2, 127, 2, 129, 2, 131, 2, 133, 2, 135, 2, 137, 2, 139, 2, 141, 2, 143, 47, 145, 48, 3, 2, 41, 3, 2, 50, 59, 4, 2, 71, 71, 103, 103, 4, 2, 45, 45, 47, 47, 4, 2, 90, 90, 122, 122, 5, 2, 50, 59, 67, 72, 99, 104, 4, 2, 41, 41, 94, 94, 4, 2, 36, 36, 94, 94, 8, 2, 102, 102, 106, 106, 111, 111, 117, 117, 121, 121, 123, 123, 5, 2, 67, 92, 97, 97, 99, 124, 6, 2, 50, 59, 67, 92, 97, 97, 99, 124, 4, 2, 60, 60, 97, 97, 4, 2, 67, 92, 99, 124, 7, 2, 47, 48, 50, 60, 67, 92, 97, 97, 99, 124, 4, 2, 67, 67, 99, 99, 4, 2, 68, 68, 100, 100, 4, 2, 69, 69, 101, 101, 4, 2, 70, 70, 102, 102, 4, 2, 72, 72, 104, 104, 4, 2, 73, 73, 105, 105, 4, 2, 74, 74, 106, 106, 4, 2, 75, 75, 107, 107, 4, 2, 76, 76, 108, 108, 4, 2, 77, 77, 109, 109, 4, 2, 78, 78, 110, 110, 4, 2, 79, 79, 111, 111, 4, 2, 80, 80, 112, 112, 4, 2, 81, 81, 113, 113, 4, 2, 82, 82, 114, 114, 4, 2, 83, 83, 115, 115, 4, 2, 84, 84, 116, 116, 4, 2, 85, 85, 117, 117, 4, 2, 86, 86, 118, 118, 4, 2, 87, 87, 119, 119, 4, 2, 88, 88, 120, 120, 4, 2, 89, 89, 121, 121, 4, 2, 91, 91, 123, 123, 4, 2, 92, 92, 124, 124, 5, 2, 11, 12, 15, 15, 34, 34, 4, 2, 12, 12, 15, 15, 2, 547, 2, 3, 3, 2, 2, 2, 2, 5, 3, 2, 2, 2, 2, 7, 3, 2, 2, 2, 2, 9, 3, 2, 2, 2, 2, 11, 3, 2, 2, 2, 2, 13, 3, 2, 2, 2, 2, 15, 3, 2, 2, 2, 2, 17, 3, 2, 2, 2, 2, 19, 3, 2, 2, 2, 2, 21, 3, 2, 2, 2, 2, 23, 3, 2, 2, 2, 2, 25, 3, 2, 2, 2, 2, 27, 3, 2, 2, 2, 2, 29, 3, 2, 2, 2, 2, 31, 3, 2, 2, 2, 2, 33, 3, 2, 2, 2, 2, 35, 3, 2, 2, 2, 2, 37, 3, 2, 2, 2, 2, 39, 3, 2, 2, 2, 2, 41, 3, 2, 2, 2, 2, 43, 3, 2, 2, 2, 2, 45, 3, 2, 2, 2, 2, 47, 3, 2, 2, 2, 2, 49, 3, 2, 2, 2, 2, 51, 3, 2, 2, 2, 2, 53, 3, 2, 2, 2, 2, 55, 3, 2, 2, 2, 2, 57, 3, 2, 2, 2, 2, 59, 3, 2, 2, 2, 2, 61, 3, 2, 2, 2, 2, 63, 3, 2, 2, 2, 2, 65, 3, 2, 2, 2, 2, 67, 3, 2, 2, 2, 2, 69, 3, 2, 2, 2, 2, 71, 3, 2, 2, 2, 2, 73, 3, 2, 2, 2, 2, 75, 3, 2, 2, 2, 2, 77, 3, 2, 2, 2, 2, 79, 3, 2, 2, 2, 2, 81, 3, 2, 2, 2, 2, 83, 3, 2, 2, 2, 2, 85, 3, 2, 2, 2, 2, 87, 3, 2, 2, 2, 2, 89, 3, 2, 2, 2, 2, 143, 3, 2, 2, 2, 2, 145, 3, 2, 2, 2, 3, 147, 3, 2, 2, 2, 5, 149, 3, 2, 2, 2, 7, 151, 3, 2, 2, 2, 9, 153, 3, 2, 2, 2, 11, 155, 3, 2, 2, 2, 13, 157, 3, 2, 2, 2, 15, 159, 3, 2, 2, 2, 17, 161, 3, 2, 2, 2, 19, 201, 3, 2, 2, 2, 21, 223, 3, 2, 2, 2, 23, 225, 3, 2, 2, 2, 25, 227, 3, 2, 2, 2, 27, 229, 3, 2, 2, 2, 29, 231, 3, 2, 2, 2, 31, 233, 3, 2, 2, 2, 33, 235, 3, 2, 2, 2, 35, 237, 3, 2, 2, 2, 37, 239, 3, 2, 2, 2, 39, 242, 3, 2, 2, 2, 41, 245, 3, 2, 2, 2, 43, 247, 3, 2, 2, 2, 45, 249, 3, 2, 2, 2, 47, 252, 3, 2, 2, 2, 49, 255, 3, 2, 2, 2, 51, 258, 3, 2, 2, 2, 53, 261, 3, 2, 2, 2, 55, 263, 3, 2, 2, 2, 57, 267, 3, 2, 2, 2, 59, 270, 3, 2, 2, 2, 61, 277, 3, 2, 2, 2, 63, 280, 3, 2, 2, 2, 65, 288, 3, 2, 2, 2, 67, 291, 3, 2, 2, 2, 69, 300, 3, 2, 2, 2, 71, 311, 3, 2, 2, 2, 73, 323, 3, 2, 2, 2, 75, 330, 3, 2, 2, 2, 77, 336, 3, 2, 2, 2, 79, 341, 3, 2, 2, 2, 81, 349, 3, 2, 2, 2, 83, 438, 3, 2, 2, 2, 85, 450, 3, 2, 2, 2, 87, 452, 3, 2, 2, 2, 89, 462, 3, 2, 2, 2, 91, 472, 3, 2, 2, 2, 93, 474, 3, 2, 2, 2, 95, 476, 3, 2, 2, 2, 97, 478, 3, 2, 2, 2, 99, 480, 3, 2, 2, 2, 101, 482, 3, 2, 2, 2, 103, 484, 3, 2, 2, 2, 105, 486, 3, 2, 2, 2, 107, 488, 3, 2, 2, 2, 109, 490, 3, 2, 2, 2, 111, 492, 3, 2, 2, 2, 113, 494, 3, 2, 2, 2, 115, 496, 3, 2, 2, 2, 117, 498, 3, 2, 2, 2, 119, 500, 3, 2, 2, 2, 121, 502, 3, 2, 2, 2, 123, 504, 3, 2, 2, 2, 125, 506, 3, 2, 2, 2, 127, 508, 3, 2, 2, 2, 129, 510, 3, 2, 2, 2, 131, 512, 3, 2, 2, 2, 133, 514, 3, 2, 2, 2, 135, 516, 3, 2, 2, 2, 137, 518, 3, 2, 2, 2, 139, 520, 3, 2, 2, 2, 141, 522, 3, 2, 2, 2, 143, 525, 3, 2, 2, 2, 145, 531, 3, 2, 2, 2, 147, 148, 7, 42, 2, 2, 148, 4, 3, 2, 2, 2, 149, 150, 7, 43, 2, 2, 150, 6, 3, 2, 2, 2, 151, 152, 7, 125, 2, 2, 152, 8, 3, 2, 2, 2, 153, 154, 7, 127, 2, 2, 154, 10, 3, 2, 2, 2, 155, 156, 7, 93, 2, 2, 156, 12, 3, 2, 2, 2, 157, 158, 7, 95, 2, 2, 158, 14, 3, 2, 2, 2, 159, 160, 7, 60, 2, 2, 160, 16, 3, 2, 2, 2, 161, 162, 7, 46, 2, 2, 162, 18, 3, 2, 2, 2, 163, 165, 9, 2, 2, 2, 164, 163, 3, 2, 2, 2, 165, 168, 3, 2, 2, 2, 166, 164, 3, 2, 2, 2, 166, 167, 3, 2, 2, 2, 167, 170, 3, 2, 2, 2, 168, 166, 3, 2, 2, 2, 169, 171, 7, 48, 2, 2, 170, 169, 3, 2, 2, 2, 170, 171, 3, 2, 2, 2, 171, 173, 3, 2, 2, 2, 172, 174, 9, 2, 2, 2, 173, 172, 3, 2, 2, 2, 174, 175, 3, 2, 2, 2, 175, 173, 3, 2, 2, 2, 175, 176, 3, 2, 2, 2, 176, 186, 3, 2, 2, 2, 177, 179, 9, 3, 2, 2, 178, 180, 9, 4, 2, 2, 179, 178, 3, 2, 2, 2, 179, 180, 3, 2, 2, 2, 180, 182, 3, 2, 2, 2, 181, 183, 9, 2, 2, 2, 182, 181, 3, 2, 2, 2, 183, 184, 3, 2, 2, 2, 184, 182, 3, 2, 2, 2, 184, 185, 3, 2, 2, 2, 185, 187, 3, 2, 2, 2, 186, 177, 3, 2, 2, 2, 186, 187, 3, 2, 2, 2, 187, 202, 3, 2, 2, 2, 188, 190, 9, 2, 2, 2, 189, 188, 3, 2, 2, 2, 190, 191, 3, 2, 2, 2, 191, 189, 3, 2, 2, 2, 191, 192, 3, 2, 2, 2, 192, 193, 3, 2, 2, 2, 193, 202, 7, 48, 2, 2, 194, 195, 7, 50, 2, 2, 195, 197, 9, 5, 2, 2, 196, 198, 9, 6, 2, 2, 197, 196, 3, 2, 2, 2, 198, 199, 3, 2, 2, 2, 199, 197, 3, 2, 2, 2, 199, 200, 3, 2, 2, 2, 200, 202, 3, 2, 2, 2, 201, 166, 3, 2, 2, 2, 201, 189, 3, 2, 2, 2, 201, 194, 3, 2, 2, 2, 202, 20, 3, 2, 2, 2, 203, 209, 7, 41, 2, 2, 204, 208, 10, 7, 2, 2, 205, 206, 7, 94, 2, 2, 206, 208, 11, 2, 2, 2, 207, 204, 3, 2, 2, 2, 207, 205, 3, 2, 2, 2, 208, 211, 3, 2, 2, 2, 209, 207, 3, 2, 2, 2, 209, 210, 3, 2, 2, 2, 210, 212, 3, 2, 2, 2, 211, 209, 3, 2, 2, 2, 212, 224, 7, 41, 2, 2, 213, 219, 7, 36, 2, 2, 214, 218, 10, 8, 2, 2, 215, 216, 7, 94, 2, 2, 216, 218, 11, 2, 2, 2, 217, 214, 3, 2, 2, 2, 217, 215, 3, 2, 2, 2, 218, 221, 3, 2, 2, 2, 219, 217, 3, 2, 2, 2, 219, 220, 3, 2, 2, 2, 220, 222, 3, 2, 2, 2, 221, 219, 3, 2, 2, 2, 222, 224, 7, 36, 2, 2, 223, 203, 3, 2, 2, 2, 223, 213, 3, 2, 2, 2, 224, 22, 3, 2, 2, 2, 225, 226, 7, 45, 2, 2, 226, 24, 3, 2, 2, 2, 227, 228, 7, 47, 2, 2, 228, 26, 3, 2, 2, 2, 229, 230, 7, 44, 2, 2, 230, 28, 3, 2, 2, 2, 231, 232, 7, 49, 2, 2, 232, 30, 3, 2, 2, 2, 233, 234, 7, 39, 2, 2, 234, 32, 3, 2, 2, 2, 235, 236, 7, 96, 2, 2, 236, 34, 3, 2, 2, 2, 237, 238, 7, 63, 2, 2, 238, 36, 3, 2, 2, 2, 239, 240, 7, 63, 2, 2, 240, 241, 7, 63, 2, 2, 241, 38, 3, 2, 2, 2, 242, 243, 7, 35, 2, 2, 243, 244, 7, 63, 2, 2, 244, 40, 3, 2, 2, 2, 245, 246, 7, 64, 2, 2, 246, 42, 3, 2, 2, 2, 247, 248, 7, 62, 2, 2, 248, 44, 3, 2, 2, 2, 249, 250, 7, 64, 2, 2, 250, 251, 7, 63, 2, 2, 251, 46, 3, 2, 2, 2, 252, 253, 7, 62, 2, 2, 253, 254, 7, 63, 2, 2, 254, 48, 3, 2, 2, 2, 255, 256, 7, 63, 2, 2, 256, 257, 7, 128, 2, 2, 257, 50, 3, 2, 2, 2, 258, 259, 7, 35, 2, 2, 259, 260, 7, 128, 2, 2, 260, 52, 3, 2, 2, 2, 261, 262, 7, 66, 2, 2, 262, 54, 3, 2, 2, 2, 263, 264, 5, 91, 46, 2, 264, 265, 5, 117, 59, 2, 265, 266, 5, 97, 49, 2, 266, 56, 3, 2, 2, 2, 267, 268, 5, 119, 60, 2, 268, 269, 5, 125, 63, 2, 269, 58, 3, 2, 2, 2, 270, 271, 5, 131, 66, 2, 271, 272, 5, 117, 59, 2, 272, 273, 5, 113, 57, 2, 273, 274, 5, 99, 50, 2, 274, 275, 5, 127, 64, 2, 275, 276, 5, 127, 64, 2, 276, 60, 3, 2, 2, 2, 277, 278, 5, 93, 47, 2, 278, 279, 5, 139, 70, 2, 279, 62, 3, 2, 2, 2, 280, 281, 5, 135, 68, 2, 281, 282, 5, 107, 54, 2, 282, 283, 5, 129, 65, 2, 283, 284, 5, 105, 53, 2, 284, 285, 5, 119, 60, 2, 285, 286, 5, 131, 66, 2, 286, 287, 5, 129, 65, 2, 287, 64, 3, 2, 2, 2, 288, 289, 5, 119, 60, 2, 289, 290, 5, 117, 59, 2, 290, 66, 3, 2, 2, 2, 291, 292, 5, 107, 54, 2, 292, 293, 5, 103, 52, 2, 293, 294, 5, 117, 59, 2, 294, 295, 5, 119, 60, 2, 295, 296, 5, 125, 63, 2, 296, 297, 5, 107, 54, 2, 297, 298, 5, 117, 59, 2, 298, 299, 5, 103, 52, 2, 299, 68, 3, 2, 2, 2, 300, 301, 5, 103, 52, 2, 301, 302, 5, 125, 63, 2, 302, 303, 5, 119, 60, 2, 303, 304, 5, 131, 66, 2, 304, 305, 5, 121, 61, 2, 305, 306, 7, 97, 2, 2, 306, 307, 5, 113, 57, 2, 307, 308, 5, 99, 50, 2, 308, 309, 5, 101, 51, 2, 309, 310, 5, 129, 65, 2, 310, 70, 3, 2, 2, 2, 311, 312, 5, 103, 52, 2, 312, 313, 5, 125, 63, 2, 313, 314, 5, 119, 60, 2, 314, 315, 5, 131, 66, 2, 315, 316, 5, 121, 61, 2, 316, 317, 7, 97, 2, 2, 317, 318, 5, 125, 63, 2, 318, 319, 5, 107, 54, 2, 319, 320, 5, 103, 52, 2, 320, 321, 5, 105, 53, 2, 321, 322, 5, 129, 65, 2, 322, 72, 3, 2, 2, 2, 323, 324, 5, 119, 60, 2, 324, 325, 5, 101, 51, 2, 325, 326, 5, 101, 51, 2, 326, 327, 5, 127, 64, 2, 327, 328, 5, 99, 50, 2, 328, 329, 5, 129, 65, 2, 329, 74, 3, 2, 2, 2, 330, 331, 5, 113, 57, 2, 331, 332, 5, 107, 54, 2, 332, 333, 5, 115, 58, 2, 333, 334, 5, 107, 54, 2, 334, 335, 5, 129, 65, 2, 335, 76, 3, 2, 2, 2, 336, 337, 5, 93, 47, 2, 337, 338, 5, 119, 60, 2, 338, 339, 5, 119, 60, 2, 339, 340, 5, 113, 57, 2, 340, 78, 3, 2, 2, 2, 341, 342, 5, 127, 64, 2, 342, 343, 5, 129, 65, 2, 343, 344, 5, 91, 46, 2, 344, 345, 5, 125, 63, 2, 345, 346, 5, 129, 65, 2, 346, 347, 7, 42, 2, 2, 347, 348, 7, 43, 2, 2, 348, 80, 3, 2, 2, 2, 349, 350, 5, 99, 50, 2, 350, 351, 5, 117, 59, 2, 351, 352, 5, 97, 49, 2, 352, 353, 7, 42, 2, 2, 353, 354, 7, 43, 2, 2, 354, 82, 3, 2, 2, 2, 355, 356, 5, 127, 64, 2, 356, 357, 5, 131, 66, 2, 357, 358, 5, 115, 58, 2, 358, 439, 3, 2, 2, 2, 359, 360, 5, 115, 58, 2, 360, 361, 5, 107, 54, 2, 361, 362, 5, 117, 59, 2, 362, 439, 3, 2, 2, 2, 363, 364, 5, 115, 58, 2, 364, 365, 5, 91, 46, 2, 365, 366, 5, 137, 69, 2, 366, 439, 3, 2, 2, 2, 367, 368, 5, 91, 46, 2, 368, 369, 5, 133, 67, 2, 369, 370, 5, 103, 52, 2, 370, 439, 3, 2, 2, 2, 371, 372, 5, 103, 52, 2, 372, 373, 5, 125, 63, 2, 373, 374, 5, 119, 60, 2, 374, 375, 5, 131, 66, 2, 375, 376, 5, 121, 61, 2, 376, 439, 3, 2, 2, 2, 377, 378, 5, 127, 64, 2, 378, 379, 5, 129, 65, 2, 379, 380, 5, 97, 49, 2, 380, 381, 5, 97, 49, 2, 381, 382, 5, 99, 50, 2, 382, 383, 5, 133, 67, 2, 383, 439, 3, 2, 2, 2, 384, 385, 5, 127, 64, 2, 385, 386, 5, 129, 65, 2, 386, 387, 5, 97, 49, 2, 387, 388, 5, 133, 67, 2, 388, 389, 5, 91, 46, 2, 389, 390, 5, 125, 63, 2, 390, 439, 3, 2, 2, 2, 391, 392, 5, 95, 48, 2, 392, 393, 5, 119, 60, 2, 393, 394, 5, 131, 66, 2, 394, 395, 5, 117, 59, 2, 395, 396, 5, 129, 65, 2, 396, 439, 3, 2, 2, 2, 397, 398, 5, 95, 48, 2, 398, 399, 5, 119, 60, 2, 399, 400, 5, 131, 66, 2, 400, 401, 5, 117, 59, 2, 401, 402, 5, 129, 65, 2, 402, 403, 7, 97, 2, 2, 403, 404, 5, 133, 67, 2, 404, 405, 5, 91, 46, 2, 405, 406, 5, 113, 57, 2, 406, 407, 5, 131, 66, 2, 407, 408, 5, 99, 50, 2, 408, 409, 5, 127, 64, 2, 409, 439, 3, 2, 2, 2, 410, 411, 5, 93, 47, 2, 411, 412, 5, 119, 60, 2, 412, 413, 5, 129, 65, 2, 413, 414, 5, 129, 65, 2, 414, 415, 5, 119, 60, 2, 415, 416, 5, 115, 58, 2, 416, 417, 5, 111, 56, 2, 417, 439, 3, 2, 2, 2, 418, 419, 5, 129, 65, 2, 419, 420, 5, 119, 60, 2, 420, 421, 5, 121, 61, 2, 421, 422, 5, 111, 56, 2, 422, 439, 3, 2, 2, 2, 423, 424, 5, 123, 62, 2, 424, 425, 5, 131, 66, 2, 425, 426, 5, 91, 46, 2, 426, 427, 5, 117, 59, 2, 427, 428, 5, 129, 65, 2, 428, 429, 5, 107, 54, 2, 429, 430, 5, 113, 57, 2, 430, 431, 5, 99, 50, 2, 431, 439, 3, 2, 2, 2, 432, 433, 5, 103, 52, 2, 433, 434, 5, 125, 63, 2, 434, 435, 5, 119, 60, 2, 435, 436, 5, 131, 66, 2, 436, 437, 5, 121, 61, 2, 437, 439, 3, 2, 2, 2, 438, 355, 3, 2, 2, 2, 438, 359, 3, 2, 2, 2, 438, 363, 3, 2, 2, 2, 438, 367, 3, 2, 2, 2, 438, 371, 3, 2, 2, 2, 438, 377, 3, 2, 2, 2, 438, 384, 3, 2, 2, 2, 438, 391, 3, 2, 2, 2, 438, 397, 3, 2, 2, 2, 438, 410, 3, 2, 2, 2, 438, 418, 3, 2, 2, 2, 438, 423, 3, 2, 2, 2, 438, 432, 3, 2, 2, 2, 439, 84, 3, 2, 2, 2, 440, 441, 5, 19, 10, 2, 441, 442, 9, 9, 2, 2, 442, 444, 3, 2, 2, 2, 443, 440, 3, 2, 2, 2, 444, 445, 3, 2, 2, 2, 445, 443, 3, 2, 2, 2, 445, 446, 3, 2, 2, 2, 446, 451, 3, 2, 2, 2, 447, 448, 5, 19, 10, 2, 448, 449, 7, 107, 2, 2, 449, 451, 3, 2, 2, 2, 450, 443, 3, 2, 2, 2, 450, 447, 3, 2, 2, 2, 451, 86, 3, 2, 2, 2, 452, 456, 9, 10, 2, 2, 453, 455, 9, 11, 2, 2, 454, 453, 3, 2, 2, 2, 455, 458, 3, 2, 2, 2, 456, 454, 3, 2, 2, 2, 456, 457, 3, 2, 2, 2, 457, 88, 3, 2, 2, 2, 458, 456, 3, 2, 2, 2, 459, 461, 9, 12, 2, 2, 460, 459, 3, 2, 2, 2, 461, 464, 3, 2, 2, 2, 462, 460, 3, 2, 2, 2, 462, 463, 3, 2, 2, 2, 463, 465, 3, 2, 2, 2, 464, 462, 3, 2, 2, 2, 465, 469, 9, 13, 2, 2, 466, 468, 9, 14, 2, 2, 467, 466, 3, 2, 2, 2, 468, 471, 3, 2, 2, 2, 469, 467, 3, 2, 2, 2, 469, 470, 3, 2, 2, 2, 470, 90, 3, 2, 2, 2, 471, 469, 3, 2, 2, 2, 472, 473, 9, 15, 2, 2, 473, 92, 3, 2, 2, 2, 474, 475, 9, 16, 2, 2, 475, 94, 3, 2, 2, 2, 476, 477, 9, 17, 2, 2, 477, 96, 3, 2, 2, 2, 478, 479, 9, 18, 2, 2, 479, 98, 3, 2, 2, 2, 480, 481, 9, 3, 2, 2, 481, 100, 3, 2, 2, 2, 482, 483, 9, 19, 2, 2, 483, 102, 3, 2, 2, 2, 484, 485, 9, 20, 2, 2, 485, 104, 3, 2, 2, 2, 486, 487, 9, 21, 2, 2, 487, 106, 3, 2, 2, 2, 488, 489, 9, 22, 2, 2, 489, 108, 3, 2, 2, 2, 490, 491, 9, 23, 2, 2, 491, 110, 3, 2, 2, 2, 492, 493, 9, 24, 2, 2, 493, 112, 3, 2, 2, 2, 494, 495, 9, 25, 2, 2, 495, 114, 3, 2, 2, 2, 496, 497, 9, 26, 2, 2, 497, 116, 3, 2, 2, 2, 498, 499, 9, 27, 2, 2, 499, 118, 3, 2, 2, 2, 500, 501, 9, 28, 2, 2, 501, 120, 3, 2, 2, 2, 502, 503, 9, 29, 2, 2, 503, 122, 3, 2, 2, 2, 504, 505, 9, 30, 2, 2, 505, 124, 3, 2, 2, 2, 506, 507, 9, 31, 2, 2, 507, 126, 3, 2, 2, 2, 508, 509, 9, 32, 2, 2, 509, 128, 3, 2, 2, 2, 510, 511, 9, 33, 2, 2, 511, 130, 3, 2, 2, 2, 512, 513, 9, 34, 2, 2, 513, 132, 3, 2, 2, 2, 514, 515, 9, 35, 2, 2, 515, 134, 3, 2, 2, 2, 516, 517, 9, 36, 2, 2, 517, 136, 3, 2, 2, 2, 518, 519, 9, 5, 2, 2, 519, 138, 3, 2, 2, 2, 520, 521, 9, 37, 2, 2, 521, 140, 3, 2, 2, 2, 522, 523, 9, 38, 2, 2, 523, 142, 3, 2, 2, 2, 524, 526, 9, 39, 2, 2, 525, 524, 3, 2, 2, 2, 526, 527, 3, 2, 2, 2, 527, 525, 3, 2, 2, 2, 527, 528, 3, 2, 2, 2, 528, 529, 3, 2, 2, 2, 529, 530, 8, 72, 2, 2, 530, 144, 3, 2, 2, 2, 531, 535, 7, 37, 2, 2, 532, 534, 10, 40, 2, 2, 533, 532, 3, 2, 2, 2, 534, 537, 3, 2, 2, 2, 535, 533, 3, 2, 2, 2, 535, 536, 3, 2, 2, 2, 536, 538, 3, 2, 2, 2, 537, 535, 3, 2, 2, 2, 538, 539, 8, 73, 2, 2, 539, 146, 3, 2, 2, 2, 25, 2, 166, 170, 175, 179, 184, 186, 191, 199, 201, 207, 209, 217, 219, 223, 438, 445, 450, 456, 462, 469, 527, 535, 3, 8, 2, 2] \ No newline at end of file diff --git a/prometheus/src/main/java/filodb/prometheus/antlr/PromQLLexer.java b/prometheus/src/main/java/filodb/prometheus/antlr/PromQLLexer.java index 7f34c782fc..e4492429ec 100644 --- a/prometheus/src/main/java/filodb/prometheus/antlr/PromQLLexer.java +++ b/prometheus/src/main/java/filodb/prometheus/antlr/PromQLLexer.java @@ -1,4 +1,4 @@ -// Generated from PromQL.g4 by ANTLR 4.9.3 +// Generated from PromQL.g4 by ANTLR 4.9.1 package filodb.prometheus.antlr; import org.antlr.v4.runtime.Lexer; import org.antlr.v4.runtime.CharStream; @@ -11,7 +11,7 @@ @SuppressWarnings({"all", "warnings", "unchecked", "unused", "cast"}) public class PromQLLexer extends Lexer { - static { RuntimeMetaData.checkVersion("4.9.3", RuntimeMetaData.VERSION); } + static { RuntimeMetaData.checkVersion("4.9.1", RuntimeMetaData.VERSION); } protected static final DFA[] _decisionToDFA; protected static final PredictionContextCache _sharedContextCache = @@ -19,10 +19,10 @@ public class PromQLLexer extends Lexer { public static final int T__0=1, T__1=2, T__2=3, T__3=4, T__4=5, T__5=6, T__6=7, T__7=8, NUMBER=9, STRING=10, ADD=11, SUB=12, MUL=13, DIV=14, MOD=15, POW=16, EQ=17, DEQ=18, - NE=19, GT=20, LT=21, GE=22, LE=23, RE=24, NRE=25, AND=26, OR=27, UNLESS=28, - BY=29, WITHOUT=30, ON=31, IGNORING=32, GROUP_LEFT=33, GROUP_RIGHT=34, - OFFSET=35, LIMIT=36, BOOL=37, AGGREGATION_OP=38, DURATION=39, IDENTIFIER=40, - IDENTIFIER_EXTENDED=41, WS=42, COMMENT=43; + NE=19, GT=20, LT=21, GE=22, LE=23, RE=24, NRE=25, AT=26, AND=27, OR=28, + UNLESS=29, BY=30, WITHOUT=31, ON=32, IGNORING=33, GROUP_LEFT=34, GROUP_RIGHT=35, + OFFSET=36, LIMIT=37, BOOL=38, START=39, END=40, AGGREGATION_OP=41, DURATION=42, + IDENTIFIER=43, IDENTIFIER_EXTENDED=44, WS=45, COMMENT=46; public static String[] channelNames = { "DEFAULT_TOKEN_CHANNEL", "HIDDEN" }; @@ -35,11 +35,11 @@ private static String[] makeRuleNames() { return new String[] { "T__0", "T__1", "T__2", "T__3", "T__4", "T__5", "T__6", "T__7", "NUMBER", "STRING", "ADD", "SUB", "MUL", "DIV", "MOD", "POW", "EQ", "DEQ", "NE", - "GT", "LT", "GE", "LE", "RE", "NRE", "AND", "OR", "UNLESS", "BY", "WITHOUT", - "ON", "IGNORING", "GROUP_LEFT", "GROUP_RIGHT", "OFFSET", "LIMIT", "BOOL", - "AGGREGATION_OP", "DURATION", "IDENTIFIER", "IDENTIFIER_EXTENDED", "A", - "B", "C", "D", "E", "F", "G", "H", "I", "J", "K", "L", "M", "N", "O", - "P", "Q", "R", "S", "T", "U", "V", "W", "X", "Y", "Z", "WS", "COMMENT" + "GT", "LT", "GE", "LE", "RE", "NRE", "AT", "AND", "OR", "UNLESS", "BY", + "WITHOUT", "ON", "IGNORING", "GROUP_LEFT", "GROUP_RIGHT", "OFFSET", "LIMIT", + "BOOL", "START", "END", "AGGREGATION_OP", "DURATION", "IDENTIFIER", "IDENTIFIER_EXTENDED", + "A", "B", "C", "D", "E", "F", "G", "H", "I", "J", "K", "L", "M", "N", + "O", "P", "Q", "R", "S", "T", "U", "V", "W", "X", "Y", "Z", "WS", "COMMENT" }; } public static final String[] ruleNames = makeRuleNames(); @@ -48,7 +48,7 @@ private static String[] makeLiteralNames() { return new String[] { null, "'('", "')'", "'{'", "'}'", "'['", "']'", "':'", "','", null, null, "'+'", "'-'", "'*'", "'/'", "'%'", "'^'", "'='", "'=='", "'!='", "'>'", - "'<'", "'>='", "'<='", "'=~'", "'!~'" + "'<'", "'>='", "'<='", "'=~'", "'!~'", "'@'" }; } private static final String[] _LITERAL_NAMES = makeLiteralNames(); @@ -56,9 +56,10 @@ private static String[] makeSymbolicNames() { return new String[] { null, null, null, null, null, null, null, null, null, "NUMBER", "STRING", "ADD", "SUB", "MUL", "DIV", "MOD", "POW", "EQ", "DEQ", "NE", "GT", "LT", - "GE", "LE", "RE", "NRE", "AND", "OR", "UNLESS", "BY", "WITHOUT", "ON", - "IGNORING", "GROUP_LEFT", "GROUP_RIGHT", "OFFSET", "LIMIT", "BOOL", "AGGREGATION_OP", - "DURATION", "IDENTIFIER", "IDENTIFIER_EXTENDED", "WS", "COMMENT" + "GE", "LE", "RE", "NRE", "AT", "AND", "OR", "UNLESS", "BY", "WITHOUT", + "ON", "IGNORING", "GROUP_LEFT", "GROUP_RIGHT", "OFFSET", "LIMIT", "BOOL", + "START", "END", "AGGREGATION_OP", "DURATION", "IDENTIFIER", "IDENTIFIER_EXTENDED", + "WS", "COMMENT" }; } private static final String[] _SYMBOLIC_NAMES = makeSymbolicNames(); @@ -120,184 +121,192 @@ public PromQLLexer(CharStream input) { public ATN getATN() { return _ATN; } public static final String _serializedATN = - "\3\u608b\ua72a\u8133\ub9ed\u417c\u3be7\u7786\u5964\2-\u0206\b\1\4\2\t"+ - "\2\4\3\t\3\4\4\t\4\4\5\t\5\4\6\t\6\4\7\t\7\4\b\t\b\4\t\t\t\4\n\t\n\4\13"+ - "\t\13\4\f\t\f\4\r\t\r\4\16\t\16\4\17\t\17\4\20\t\20\4\21\t\21\4\22\t\22"+ - "\4\23\t\23\4\24\t\24\4\25\t\25\4\26\t\26\4\27\t\27\4\30\t\30\4\31\t\31"+ - "\4\32\t\32\4\33\t\33\4\34\t\34\4\35\t\35\4\36\t\36\4\37\t\37\4 \t \4!"+ - "\t!\4\"\t\"\4#\t#\4$\t$\4%\t%\4&\t&\4\'\t\'\4(\t(\4)\t)\4*\t*\4+\t+\4"+ - ",\t,\4-\t-\4.\t.\4/\t/\4\60\t\60\4\61\t\61\4\62\t\62\4\63\t\63\4\64\t"+ - "\64\4\65\t\65\4\66\t\66\4\67\t\67\48\t8\49\t9\4:\t:\4;\t;\4<\t<\4=\t="+ - "\4>\t>\4?\t?\4@\t@\4A\tA\4B\tB\4C\tC\4D\tD\4E\tE\4F\tF\3\2\3\2\3\3\3\3"+ - "\3\4\3\4\3\5\3\5\3\6\3\6\3\7\3\7\3\b\3\b\3\t\3\t\3\n\7\n\u009f\n\n\f\n"+ - "\16\n\u00a2\13\n\3\n\5\n\u00a5\n\n\3\n\6\n\u00a8\n\n\r\n\16\n\u00a9\3"+ - "\n\3\n\5\n\u00ae\n\n\3\n\6\n\u00b1\n\n\r\n\16\n\u00b2\5\n\u00b5\n\n\3"+ - "\n\6\n\u00b8\n\n\r\n\16\n\u00b9\3\n\3\n\3\n\3\n\6\n\u00c0\n\n\r\n\16\n"+ - "\u00c1\5\n\u00c4\n\n\3\13\3\13\3\13\3\13\7\13\u00ca\n\13\f\13\16\13\u00cd"+ - "\13\13\3\13\3\13\3\13\3\13\3\13\7\13\u00d4\n\13\f\13\16\13\u00d7\13\13"+ - "\3\13\5\13\u00da\n\13\3\f\3\f\3\r\3\r\3\16\3\16\3\17\3\17\3\20\3\20\3"+ - "\21\3\21\3\22\3\22\3\23\3\23\3\23\3\24\3\24\3\24\3\25\3\25\3\26\3\26\3"+ - "\27\3\27\3\27\3\30\3\30\3\30\3\31\3\31\3\31\3\32\3\32\3\32\3\33\3\33\3"+ - "\33\3\33\3\34\3\34\3\34\3\35\3\35\3\35\3\35\3\35\3\35\3\35\3\36\3\36\3"+ - "\36\3\37\3\37\3\37\3\37\3\37\3\37\3\37\3\37\3 \3 \3 \3!\3!\3!\3!\3!\3"+ - "!\3!\3!\3!\3\"\3\"\3\"\3\"\3\"\3\"\3\"\3\"\3\"\3\"\3\"\3#\3#\3#\3#\3#"+ - "\3#\3#\3#\3#\3#\3#\3#\3$\3$\3$\3$\3$\3$\3$\3%\3%\3%\3%\3%\3%\3&\3&\3&"+ - "\3&\3&\3\'\3\'\3\'\3\'\3\'\3\'\3\'\3\'\3\'\3\'\3\'\3\'\3\'\3\'\3\'\3\'"+ - "\3\'\3\'\3\'\3\'\3\'\3\'\3\'\3\'\3\'\3\'\3\'\3\'\3\'\3\'\3\'\3\'\3\'\3"+ - "\'\3\'\3\'\3\'\3\'\3\'\3\'\3\'\3\'\3\'\3\'\3\'\3\'\3\'\3\'\3\'\3\'\3\'"+ - "\3\'\3\'\3\'\3\'\3\'\3\'\3\'\3\'\3\'\3\'\3\'\3\'\3\'\3\'\3\'\3\'\3\'\3"+ - "\'\3\'\3\'\3\'\3\'\3\'\3\'\3\'\3\'\3\'\3\'\3\'\3\'\3\'\3\'\5\'\u01a1\n"+ - "\'\3(\3(\3(\6(\u01a6\n(\r(\16(\u01a7\3(\3(\3(\5(\u01ad\n(\3)\3)\7)\u01b1"+ - "\n)\f)\16)\u01b4\13)\3*\7*\u01b7\n*\f*\16*\u01ba\13*\3*\3*\7*\u01be\n"+ - "*\f*\16*\u01c1\13*\3+\3+\3,\3,\3-\3-\3.\3.\3/\3/\3\60\3\60\3\61\3\61\3"+ - "\62\3\62\3\63\3\63\3\64\3\64\3\65\3\65\3\66\3\66\3\67\3\67\38\38\39\3"+ - "9\3:\3:\3;\3;\3<\3<\3=\3=\3>\3>\3?\3?\3@\3@\3A\3A\3B\3B\3C\3C\3D\3D\3"+ - "E\6E\u01f8\nE\rE\16E\u01f9\3E\3E\3F\3F\7F\u0200\nF\fF\16F\u0203\13F\3"+ - "F\3F\2\2G\3\3\5\4\7\5\t\6\13\7\r\b\17\t\21\n\23\13\25\f\27\r\31\16\33"+ - "\17\35\20\37\21!\22#\23%\24\'\25)\26+\27-\30/\31\61\32\63\33\65\34\67"+ - "\359\36;\37= ?!A\"C#E$G%I&K\'M(O)Q*S+U\2W\2Y\2[\2]\2_\2a\2c\2e\2g\2i\2"+ - "k\2m\2o\2q\2s\2u\2w\2y\2{\2}\2\177\2\u0081\2\u0083\2\u0085\2\u0087\2\u0089"+ - ",\u008b-\3\2)\3\2\62;\4\2GGgg\4\2--//\4\2ZZzz\5\2\62;CHch\4\2))^^\4\2"+ - "$$^^\b\2ffjjoouuyy{{\5\2C\\aac|\6\2\62;C\\aac|\4\2<