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 extends AbstractParseTreeVisitor implements * {@link #visitChildren} on {@code ctx}.

*/ @Override public T visitOffset(PromQLParser.OffsetContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

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<\2\2\u00f2,\3\2\2\2\u00f3\u00f4\7@\2\2\u00f4\u00f5\7?\2\2\u00f5.\3"+ - "\2\2\2\u00f6\u00f7\7>\2\2\u00f7\u00f8\7?\2\2\u00f8\60\3\2\2\2\u00f9\u00fa"+ - "\7?\2\2\u00fa\u00fb\7\u0080\2\2\u00fb\62\3\2\2\2\u00fc\u00fd\7#\2\2\u00fd"+ - "\u00fe\7\u0080\2\2\u00fe\64\3\2\2\2\u00ff\u0100\5U+\2\u0100\u0101\5o8"+ - "\2\u0101\u0102\5[.\2\u0102\66\3\2\2\2\u0103\u0104\5q9\2\u0104\u0105\5"+ - "w<\2\u01058\3\2\2\2\u0106\u0107\5}?\2\u0107\u0108\5o8\2\u0108\u0109\5"+ - "k\66\2\u0109\u010a\5]/\2\u010a\u010b\5y=\2\u010b\u010c\5y=\2\u010c:\3"+ - "\2\2\2\u010d\u010e\5W,\2\u010e\u010f\5\u0085C\2\u010f<\3\2\2\2\u0110\u0111"+ - "\5\u0081A\2\u0111\u0112\5e\63\2\u0112\u0113\5{>\2\u0113\u0114\5c\62\2"+ - "\u0114\u0115\5q9\2\u0115\u0116\5}?\2\u0116\u0117\5{>\2\u0117>\3\2\2\2"+ - "\u0118\u0119\5q9\2\u0119\u011a\5o8\2\u011a@\3\2\2\2\u011b\u011c\5e\63"+ - "\2\u011c\u011d\5a\61\2\u011d\u011e\5o8\2\u011e\u011f\5q9\2\u011f\u0120"+ - "\5w<\2\u0120\u0121\5e\63\2\u0121\u0122\5o8\2\u0122\u0123\5a\61\2\u0123"+ - "B\3\2\2\2\u0124\u0125\5a\61\2\u0125\u0126\5w<\2\u0126\u0127\5q9\2\u0127"+ - "\u0128\5}?\2\u0128\u0129\5s:\2\u0129\u012a\7a\2\2\u012a\u012b\5k\66\2"+ - "\u012b\u012c\5]/\2\u012c\u012d\5_\60\2\u012d\u012e\5{>\2\u012eD\3\2\2"+ - "\2\u012f\u0130\5a\61\2\u0130\u0131\5w<\2\u0131\u0132\5q9\2\u0132\u0133"+ - "\5}?\2\u0133\u0134\5s:\2\u0134\u0135\7a\2\2\u0135\u0136\5w<\2\u0136\u0137"+ - "\5e\63\2\u0137\u0138\5a\61\2\u0138\u0139\5c\62\2\u0139\u013a\5{>\2\u013a"+ - "F\3\2\2\2\u013b\u013c\5q9\2\u013c\u013d\5_\60\2\u013d\u013e\5_\60\2\u013e"+ - "\u013f\5y=\2\u013f\u0140\5]/\2\u0140\u0141\5{>\2\u0141H\3\2\2\2\u0142"+ - "\u0143\5k\66\2\u0143\u0144\5e\63\2\u0144\u0145\5m\67\2\u0145\u0146\5e"+ - "\63\2\u0146\u0147\5{>\2\u0147J\3\2\2\2\u0148\u0149\5W,\2\u0149\u014a\5"+ - "q9\2\u014a\u014b\5q9\2\u014b\u014c\5k\66\2\u014cL\3\2\2\2\u014d\u014e"+ - "\5y=\2\u014e\u014f\5}?\2\u014f\u0150\5m\67\2\u0150\u01a1\3\2\2\2\u0151"+ - "\u0152\5m\67\2\u0152\u0153\5e\63\2\u0153\u0154\5o8\2\u0154\u01a1\3\2\2"+ - "\2\u0155\u0156\5m\67\2\u0156\u0157\5U+\2\u0157\u0158\5\u0083B\2\u0158"+ - "\u01a1\3\2\2\2\u0159\u015a\5U+\2\u015a\u015b\5\177@\2\u015b\u015c\5a\61"+ - "\2\u015c\u01a1\3\2\2\2\u015d\u015e\5a\61\2\u015e\u015f\5w<\2\u015f\u0160"+ - "\5q9\2\u0160\u0161\5}?\2\u0161\u0162\5s:\2\u0162\u01a1\3\2\2\2\u0163\u0164"+ - "\5y=\2\u0164\u0165\5{>\2\u0165\u0166\5[.\2\u0166\u0167\5[.\2\u0167\u0168"+ - "\5]/\2\u0168\u0169\5\177@\2\u0169\u01a1\3\2\2\2\u016a\u016b\5y=\2\u016b"+ - "\u016c\5{>\2\u016c\u016d\5[.\2\u016d\u016e\5\177@\2\u016e\u016f\5U+\2"+ - "\u016f\u0170\5w<\2\u0170\u01a1\3\2\2\2\u0171\u0172\5Y-\2\u0172\u0173\5"+ - "q9\2\u0173\u0174\5}?\2\u0174\u0175\5o8\2\u0175\u0176\5{>\2\u0176\u01a1"+ - "\3\2\2\2\u0177\u0178\5Y-\2\u0178\u0179\5q9\2\u0179\u017a\5}?\2\u017a\u017b"+ - "\5o8\2\u017b\u017c\5{>\2\u017c\u017d\7a\2\2\u017d\u017e\5\177@\2\u017e"+ - "\u017f\5U+\2\u017f\u0180\5k\66\2\u0180\u0181\5}?\2\u0181\u0182\5]/\2\u0182"+ - "\u0183\5y=\2\u0183\u01a1\3\2\2\2\u0184\u0185\5W,\2\u0185\u0186\5q9\2\u0186"+ - "\u0187\5{>\2\u0187\u0188\5{>\2\u0188\u0189\5q9\2\u0189\u018a\5m\67\2\u018a"+ - "\u018b\5i\65\2\u018b\u01a1\3\2\2\2\u018c\u018d\5{>\2\u018d\u018e\5q9\2"+ - "\u018e\u018f\5s:\2\u018f\u0190\5i\65\2\u0190\u01a1\3\2\2\2\u0191\u0192"+ - "\5u;\2\u0192\u0193\5}?\2\u0193\u0194\5U+\2\u0194\u0195\5o8\2\u0195\u0196"+ - "\5{>\2\u0196\u0197\5e\63\2\u0197\u0198\5k\66\2\u0198\u0199\5]/\2\u0199"+ - "\u01a1\3\2\2\2\u019a\u019b\5a\61\2\u019b\u019c\5w<\2\u019c\u019d\5q9\2"+ - "\u019d\u019e\5}?\2\u019e\u019f\5s:\2\u019f\u01a1\3\2\2\2\u01a0\u014d\3"+ - "\2\2\2\u01a0\u0151\3\2\2\2\u01a0\u0155\3\2\2\2\u01a0\u0159\3\2\2\2\u01a0"+ - "\u015d\3\2\2\2\u01a0\u0163\3\2\2\2\u01a0\u016a\3\2\2\2\u01a0\u0171\3\2"+ - "\2\2\u01a0\u0177\3\2\2\2\u01a0\u0184\3\2\2\2\u01a0\u018c\3\2\2\2\u01a0"+ - "\u0191\3\2\2\2\u01a0\u019a\3\2\2\2\u01a1N\3\2\2\2\u01a2\u01a3\5\23\n\2"+ - "\u01a3\u01a4\t\t\2\2\u01a4\u01a6\3\2\2\2\u01a5\u01a2\3\2\2\2\u01a6\u01a7"+ - "\3\2\2\2\u01a7\u01a5\3\2\2\2\u01a7\u01a8\3\2\2\2\u01a8\u01ad\3\2\2\2\u01a9"+ - "\u01aa\5\23\n\2\u01aa\u01ab\7k\2\2\u01ab\u01ad\3\2\2\2\u01ac\u01a5\3\2"+ - "\2\2\u01ac\u01a9\3\2\2\2\u01adP\3\2\2\2\u01ae\u01b2\t\n\2\2\u01af\u01b1"+ - "\t\13\2\2\u01b0\u01af\3\2\2\2\u01b1\u01b4\3\2\2\2\u01b2\u01b0\3\2\2\2"+ - "\u01b2\u01b3\3\2\2\2\u01b3R\3\2\2\2\u01b4\u01b2\3\2\2\2\u01b5\u01b7\t"+ - "\f\2\2\u01b6\u01b5\3\2\2\2\u01b7\u01ba\3\2\2\2\u01b8\u01b6\3\2\2\2\u01b8"+ - "\u01b9\3\2\2\2\u01b9\u01bb\3\2\2\2\u01ba\u01b8\3\2\2\2\u01bb\u01bf\t\r"+ - "\2\2\u01bc\u01be\t\16\2\2\u01bd\u01bc\3\2\2\2\u01be\u01c1\3\2\2\2\u01bf"+ - "\u01bd\3\2\2\2\u01bf\u01c0\3\2\2\2\u01c0T\3\2\2\2\u01c1\u01bf\3\2\2\2"+ - "\u01c2\u01c3\t\17\2\2\u01c3V\3\2\2\2\u01c4\u01c5\t\20\2\2\u01c5X\3\2\2"+ - "\2\u01c6\u01c7\t\21\2\2\u01c7Z\3\2\2\2\u01c8\u01c9\t\22\2\2\u01c9\\\3"+ - "\2\2\2\u01ca\u01cb\t\3\2\2\u01cb^\3\2\2\2\u01cc\u01cd\t\23\2\2\u01cd`"+ - "\3\2\2\2\u01ce\u01cf\t\24\2\2\u01cfb\3\2\2\2\u01d0\u01d1\t\25\2\2\u01d1"+ - "d\3\2\2\2\u01d2\u01d3\t\26\2\2\u01d3f\3\2\2\2\u01d4\u01d5\t\27\2\2\u01d5"+ - "h\3\2\2\2\u01d6\u01d7\t\30\2\2\u01d7j\3\2\2\2\u01d8\u01d9\t\31\2\2\u01d9"+ - "l\3\2\2\2\u01da\u01db\t\32\2\2\u01dbn\3\2\2\2\u01dc\u01dd\t\33\2\2\u01dd"+ - "p\3\2\2\2\u01de\u01df\t\34\2\2\u01dfr\3\2\2\2\u01e0\u01e1\t\35\2\2\u01e1"+ - "t\3\2\2\2\u01e2\u01e3\t\36\2\2\u01e3v\3\2\2\2\u01e4\u01e5\t\37\2\2\u01e5"+ - "x\3\2\2\2\u01e6\u01e7\t \2\2\u01e7z\3\2\2\2\u01e8\u01e9\t!\2\2\u01e9|"+ - "\3\2\2\2\u01ea\u01eb\t\"\2\2\u01eb~\3\2\2\2\u01ec\u01ed\t#\2\2\u01ed\u0080"+ - "\3\2\2\2\u01ee\u01ef\t$\2\2\u01ef\u0082\3\2\2\2\u01f0\u01f1\t\5\2\2\u01f1"+ - "\u0084\3\2\2\2\u01f2\u01f3\t%\2\2\u01f3\u0086\3\2\2\2\u01f4\u01f5\t&\2"+ - "\2\u01f5\u0088\3\2\2\2\u01f6\u01f8\t\'\2\2\u01f7\u01f6\3\2\2\2\u01f8\u01f9"+ - "\3\2\2\2\u01f9\u01f7\3\2\2\2\u01f9\u01fa\3\2\2\2\u01fa\u01fb\3\2\2\2\u01fb"+ - "\u01fc\bE\2\2\u01fc\u008a\3\2\2\2\u01fd\u0201\7%\2\2\u01fe\u0200\n(\2"+ - "\2\u01ff\u01fe\3\2\2\2\u0200\u0203\3\2\2\2\u0201\u01ff\3\2\2\2\u0201\u0202"+ - "\3\2\2\2\u0202\u0204\3\2\2\2\u0203\u0201\3\2\2\2\u0204\u0205\bF\2\2\u0205"+ - "\u008c\3\2\2\2\31\2\u00a0\u00a4\u00a9\u00ad\u00b2\u00b4\u00b9\u00c1\u00c3"+ - "\u00c9\u00cb\u00d3\u00d5\u00d9\u01a0\u01a7\u01ac\u01b2\u01b8\u01bf\u01f9"+ - "\u0201\3\b\2\2"; + "\3\u608b\ua72a\u8133\ub9ed\u417c\u3be7\u7786\u5964\2\60\u021c\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\4G\tG\4H\tH\4"+ + "I\tI\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\u00a5\n\n\f\n\16\n\u00a8\13\n\3\n\5\n\u00ab\n\n\3\n\6\n\u00ae\n"+ + "\n\r\n\16\n\u00af\3\n\3\n\5\n\u00b4\n\n\3\n\6\n\u00b7\n\n\r\n\16\n\u00b8"+ + "\5\n\u00bb\n\n\3\n\6\n\u00be\n\n\r\n\16\n\u00bf\3\n\3\n\3\n\3\n\6\n\u00c6"+ + "\n\n\r\n\16\n\u00c7\5\n\u00ca\n\n\3\13\3\13\3\13\3\13\7\13\u00d0\n\13"+ + "\f\13\16\13\u00d3\13\13\3\13\3\13\3\13\3\13\3\13\7\13\u00da\n\13\f\13"+ + "\16\13\u00dd\13\13\3\13\5\13\u00e0\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\34\3\34\3\34\3\34\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 \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*\3*\3*\3*\3*\3*\3*\3*\3*\3*\3"+ + "*\3*\3*\3*\3*\3*\3*\3*\3*\3*\3*\3*\5*\u01b7\n*\3+\3+\3+\6+\u01bc\n+\r"+ + "+\16+\u01bd\3+\3+\3+\5+\u01c3\n+\3,\3,\7,\u01c7\n,\f,\16,\u01ca\13,\3"+ + "-\7-\u01cd\n-\f-\16-\u01d0\13-\3-\3-\7-\u01d4\n-\f-\16-\u01d7\13-\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\39\3:\3:\3;\3;\3<\3<\3=\3=\3>\3>\3?\3?\3@"+ + "\3@\3A\3A\3B\3B\3C\3C\3D\3D\3E\3E\3F\3F\3G\3G\3H\6H\u020e\nH\rH\16H\u020f"+ + "\3H\3H\3I\3I\7I\u0216\nI\fI\16I\u0219\13I\3I\3I\2\2J\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,W-Y.[\2]\2_\2a\2c\2e\2g\2i\2k\2m\2o\2q\2s\2u\2w\2y\2{\2}\2\177"+ + "\2\u0081\2\u0083\2\u0085\2\u0087\2\u0089\2\u008b\2\u008d\2\u008f/\u0091"+ + "\60\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<\2\2\u00f8,\3\2\2\2\u00f9\u00fa\7@\2\2\u00fa\u00fb"+ + "\7?\2\2\u00fb.\3\2\2\2\u00fc\u00fd\7>\2\2\u00fd\u00fe\7?\2\2\u00fe\60"+ + "\3\2\2\2\u00ff\u0100\7?\2\2\u0100\u0101\7\u0080\2\2\u0101\62\3\2\2\2\u0102"+ + "\u0103\7#\2\2\u0103\u0104\7\u0080\2\2\u0104\64\3\2\2\2\u0105\u0106\7B"+ + "\2\2\u0106\66\3\2\2\2\u0107\u0108\5[.\2\u0108\u0109\5u;\2\u0109\u010a"+ + "\5a\61\2\u010a8\3\2\2\2\u010b\u010c\5w<\2\u010c\u010d\5}?\2\u010d:\3\2"+ + "\2\2\u010e\u010f\5\u0083B\2\u010f\u0110\5u;\2\u0110\u0111\5q9\2\u0111"+ + "\u0112\5c\62\2\u0112\u0113\5\177@\2\u0113\u0114\5\177@\2\u0114<\3\2\2"+ + "\2\u0115\u0116\5]/\2\u0116\u0117\5\u008bF\2\u0117>\3\2\2\2\u0118\u0119"+ + "\5\u0087D\2\u0119\u011a\5k\66\2\u011a\u011b\5\u0081A\2\u011b\u011c\5i"+ + "\65\2\u011c\u011d\5w<\2\u011d\u011e\5\u0083B\2\u011e\u011f\5\u0081A\2"+ + "\u011f@\3\2\2\2\u0120\u0121\5w<\2\u0121\u0122\5u;\2\u0122B\3\2\2\2\u0123"+ + "\u0124\5k\66\2\u0124\u0125\5g\64\2\u0125\u0126\5u;\2\u0126\u0127\5w<\2"+ + "\u0127\u0128\5}?\2\u0128\u0129\5k\66\2\u0129\u012a\5u;\2\u012a\u012b\5"+ + "g\64\2\u012bD\3\2\2\2\u012c\u012d\5g\64\2\u012d\u012e\5}?\2\u012e\u012f"+ + "\5w<\2\u012f\u0130\5\u0083B\2\u0130\u0131\5y=\2\u0131\u0132\7a\2\2\u0132"+ + "\u0133\5q9\2\u0133\u0134\5c\62\2\u0134\u0135\5e\63\2\u0135\u0136\5\u0081"+ + "A\2\u0136F\3\2\2\2\u0137\u0138\5g\64\2\u0138\u0139\5}?\2\u0139\u013a\5"+ + "w<\2\u013a\u013b\5\u0083B\2\u013b\u013c\5y=\2\u013c\u013d\7a\2\2\u013d"+ + "\u013e\5}?\2\u013e\u013f\5k\66\2\u013f\u0140\5g\64\2\u0140\u0141\5i\65"+ + "\2\u0141\u0142\5\u0081A\2\u0142H\3\2\2\2\u0143\u0144\5w<\2\u0144\u0145"+ + "\5e\63\2\u0145\u0146\5e\63\2\u0146\u0147\5\177@\2\u0147\u0148\5c\62\2"+ + "\u0148\u0149\5\u0081A\2\u0149J\3\2\2\2\u014a\u014b\5q9\2\u014b\u014c\5"+ + "k\66\2\u014c\u014d\5s:\2\u014d\u014e\5k\66\2\u014e\u014f\5\u0081A\2\u014f"+ + "L\3\2\2\2\u0150\u0151\5]/\2\u0151\u0152\5w<\2\u0152\u0153\5w<\2\u0153"+ + "\u0154\5q9\2\u0154N\3\2\2\2\u0155\u0156\5\177@\2\u0156\u0157\5\u0081A"+ + "\2\u0157\u0158\5[.\2\u0158\u0159\5}?\2\u0159\u015a\5\u0081A\2\u015a\u015b"+ + "\7*\2\2\u015b\u015c\7+\2\2\u015cP\3\2\2\2\u015d\u015e\5c\62\2\u015e\u015f"+ + "\5u;\2\u015f\u0160\5a\61\2\u0160\u0161\7*\2\2\u0161\u0162\7+\2\2\u0162"+ + "R\3\2\2\2\u0163\u0164\5\177@\2\u0164\u0165\5\u0083B\2\u0165\u0166\5s:"+ + "\2\u0166\u01b7\3\2\2\2\u0167\u0168\5s:\2\u0168\u0169\5k\66\2\u0169\u016a"+ + "\5u;\2\u016a\u01b7\3\2\2\2\u016b\u016c\5s:\2\u016c\u016d\5[.\2\u016d\u016e"+ + "\5\u0089E\2\u016e\u01b7\3\2\2\2\u016f\u0170\5[.\2\u0170\u0171\5\u0085"+ + "C\2\u0171\u0172\5g\64\2\u0172\u01b7\3\2\2\2\u0173\u0174\5g\64\2\u0174"+ + "\u0175\5}?\2\u0175\u0176\5w<\2\u0176\u0177\5\u0083B\2\u0177\u0178\5y="+ + "\2\u0178\u01b7\3\2\2\2\u0179\u017a\5\177@\2\u017a\u017b\5\u0081A\2\u017b"+ + "\u017c\5a\61\2\u017c\u017d\5a\61\2\u017d\u017e\5c\62\2\u017e\u017f\5\u0085"+ + "C\2\u017f\u01b7\3\2\2\2\u0180\u0181\5\177@\2\u0181\u0182\5\u0081A\2\u0182"+ + "\u0183\5a\61\2\u0183\u0184\5\u0085C\2\u0184\u0185\5[.\2\u0185\u0186\5"+ + "}?\2\u0186\u01b7\3\2\2\2\u0187\u0188\5_\60\2\u0188\u0189\5w<\2\u0189\u018a"+ + "\5\u0083B\2\u018a\u018b\5u;\2\u018b\u018c\5\u0081A\2\u018c\u01b7\3\2\2"+ + "\2\u018d\u018e\5_\60\2\u018e\u018f\5w<\2\u018f\u0190\5\u0083B\2\u0190"+ + "\u0191\5u;\2\u0191\u0192\5\u0081A\2\u0192\u0193\7a\2\2\u0193\u0194\5\u0085"+ + "C\2\u0194\u0195\5[.\2\u0195\u0196\5q9\2\u0196\u0197\5\u0083B\2\u0197\u0198"+ + "\5c\62\2\u0198\u0199\5\177@\2\u0199\u01b7\3\2\2\2\u019a\u019b\5]/\2\u019b"+ + "\u019c\5w<\2\u019c\u019d\5\u0081A\2\u019d\u019e\5\u0081A\2\u019e\u019f"+ + "\5w<\2\u019f\u01a0\5s:\2\u01a0\u01a1\5o8\2\u01a1\u01b7\3\2\2\2\u01a2\u01a3"+ + "\5\u0081A\2\u01a3\u01a4\5w<\2\u01a4\u01a5\5y=\2\u01a5\u01a6\5o8\2\u01a6"+ + "\u01b7\3\2\2\2\u01a7\u01a8\5{>\2\u01a8\u01a9\5\u0083B\2\u01a9\u01aa\5"+ + "[.\2\u01aa\u01ab\5u;\2\u01ab\u01ac\5\u0081A\2\u01ac\u01ad\5k\66\2\u01ad"+ + "\u01ae\5q9\2\u01ae\u01af\5c\62\2\u01af\u01b7\3\2\2\2\u01b0\u01b1\5g\64"+ + "\2\u01b1\u01b2\5}?\2\u01b2\u01b3\5w<\2\u01b3\u01b4\5\u0083B\2\u01b4\u01b5"+ + "\5y=\2\u01b5\u01b7\3\2\2\2\u01b6\u0163\3\2\2\2\u01b6\u0167\3\2\2\2\u01b6"+ + "\u016b\3\2\2\2\u01b6\u016f\3\2\2\2\u01b6\u0173\3\2\2\2\u01b6\u0179\3\2"+ + "\2\2\u01b6\u0180\3\2\2\2\u01b6\u0187\3\2\2\2\u01b6\u018d\3\2\2\2\u01b6"+ + "\u019a\3\2\2\2\u01b6\u01a2\3\2\2\2\u01b6\u01a7\3\2\2\2\u01b6\u01b0\3\2"+ + "\2\2\u01b7T\3\2\2\2\u01b8\u01b9\5\23\n\2\u01b9\u01ba\t\t\2\2\u01ba\u01bc"+ + "\3\2\2\2\u01bb\u01b8\3\2\2\2\u01bc\u01bd\3\2\2\2\u01bd\u01bb\3\2\2\2\u01bd"+ + "\u01be\3\2\2\2\u01be\u01c3\3\2\2\2\u01bf\u01c0\5\23\n\2\u01c0\u01c1\7"+ + "k\2\2\u01c1\u01c3\3\2\2\2\u01c2\u01bb\3\2\2\2\u01c2\u01bf\3\2\2\2\u01c3"+ + "V\3\2\2\2\u01c4\u01c8\t\n\2\2\u01c5\u01c7\t\13\2\2\u01c6\u01c5\3\2\2\2"+ + "\u01c7\u01ca\3\2\2\2\u01c8\u01c6\3\2\2\2\u01c8\u01c9\3\2\2\2\u01c9X\3"+ + "\2\2\2\u01ca\u01c8\3\2\2\2\u01cb\u01cd\t\f\2\2\u01cc\u01cb\3\2\2\2\u01cd"+ + "\u01d0\3\2\2\2\u01ce\u01cc\3\2\2\2\u01ce\u01cf\3\2\2\2\u01cf\u01d1\3\2"+ + "\2\2\u01d0\u01ce\3\2\2\2\u01d1\u01d5\t\r\2\2\u01d2\u01d4\t\16\2\2\u01d3"+ + "\u01d2\3\2\2\2\u01d4\u01d7\3\2\2\2\u01d5\u01d3\3\2\2\2\u01d5\u01d6\3\2"+ + "\2\2\u01d6Z\3\2\2\2\u01d7\u01d5\3\2\2\2\u01d8\u01d9\t\17\2\2\u01d9\\\3"+ + "\2\2\2\u01da\u01db\t\20\2\2\u01db^\3\2\2\2\u01dc\u01dd\t\21\2\2\u01dd"+ + "`\3\2\2\2\u01de\u01df\t\22\2\2\u01dfb\3\2\2\2\u01e0\u01e1\t\3\2\2\u01e1"+ + "d\3\2\2\2\u01e2\u01e3\t\23\2\2\u01e3f\3\2\2\2\u01e4\u01e5\t\24\2\2\u01e5"+ + "h\3\2\2\2\u01e6\u01e7\t\25\2\2\u01e7j\3\2\2\2\u01e8\u01e9\t\26\2\2\u01e9"+ + "l\3\2\2\2\u01ea\u01eb\t\27\2\2\u01ebn\3\2\2\2\u01ec\u01ed\t\30\2\2\u01ed"+ + "p\3\2\2\2\u01ee\u01ef\t\31\2\2\u01efr\3\2\2\2\u01f0\u01f1\t\32\2\2\u01f1"+ + "t\3\2\2\2\u01f2\u01f3\t\33\2\2\u01f3v\3\2\2\2\u01f4\u01f5\t\34\2\2\u01f5"+ + "x\3\2\2\2\u01f6\u01f7\t\35\2\2\u01f7z\3\2\2\2\u01f8\u01f9\t\36\2\2\u01f9"+ + "|\3\2\2\2\u01fa\u01fb\t\37\2\2\u01fb~\3\2\2\2\u01fc\u01fd\t \2\2\u01fd"+ + "\u0080\3\2\2\2\u01fe\u01ff\t!\2\2\u01ff\u0082\3\2\2\2\u0200\u0201\t\""+ + "\2\2\u0201\u0084\3\2\2\2\u0202\u0203\t#\2\2\u0203\u0086\3\2\2\2\u0204"+ + "\u0205\t$\2\2\u0205\u0088\3\2\2\2\u0206\u0207\t\5\2\2\u0207\u008a\3\2"+ + "\2\2\u0208\u0209\t%\2\2\u0209\u008c\3\2\2\2\u020a\u020b\t&\2\2\u020b\u008e"+ + "\3\2\2\2\u020c\u020e\t\'\2\2\u020d\u020c\3\2\2\2\u020e\u020f\3\2\2\2\u020f"+ + "\u020d\3\2\2\2\u020f\u0210\3\2\2\2\u0210\u0211\3\2\2\2\u0211\u0212\bH"+ + "\2\2\u0212\u0090\3\2\2\2\u0213\u0217\7%\2\2\u0214\u0216\n(\2\2\u0215\u0214"+ + "\3\2\2\2\u0216\u0219\3\2\2\2\u0217\u0215\3\2\2\2\u0217\u0218\3\2\2\2\u0218"+ + "\u021a\3\2\2\2\u0219\u0217\3\2\2\2\u021a\u021b\bI\2\2\u021b\u0092\3\2"+ + "\2\2\31\2\u00a6\u00aa\u00af\u00b3\u00b8\u00ba\u00bf\u00c7\u00c9\u00cf"+ + "\u00d1\u00d9\u00db\u00df\u01b6\u01bd\u01c2\u01c8\u01ce\u01d5\u020f\u0217"+ + "\3\b\2\2"; public static final ATN _ATN = new ATNDeserializer().deserialize(_serializedATN.toCharArray()); static { diff --git a/prometheus/src/main/java/filodb/prometheus/antlr/PromQLLexer.tokens b/prometheus/src/main/java/filodb/prometheus/antlr/PromQLLexer.tokens index 4a4589c50f..993fc6e637 100644 --- a/prometheus/src/main/java/filodb/prometheus/antlr/PromQLLexer.tokens +++ b/prometheus/src/main/java/filodb/prometheus/antlr/PromQLLexer.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/PromQLParser.java b/prometheus/src/main/java/filodb/prometheus/antlr/PromQLParser.java index ad89f2710f..9014deb0a9 100644 --- a/prometheus/src/main/java/filodb/prometheus/antlr/PromQLParser.java +++ b/prometheus/src/main/java/filodb/prometheus/antlr/PromQLParser.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.atn.*; import org.antlr.v4.runtime.dfa.DFA; @@ -11,7 +11,7 @@ @SuppressWarnings({"all", "warnings", "unchecked", "unused", "cast"}) public class PromQLParser extends Parser { - 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,30 +19,31 @@ public class PromQLParser extends Parser { 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 final int RULE_expression = 0, RULE_vectorExpression = 1, RULE_unaryOp = 2, RULE_powOp = 3, RULE_multOp = 4, RULE_addOp = 5, RULE_compareOp = 6, RULE_andUnlessOp = 7, RULE_orOp = 8, RULE_vector = 9, RULE_parens = 10, RULE_instantOrRangeSelector = 11, - RULE_instantSelector = 12, RULE_window = 13, RULE_offset = 14, RULE_limit = 15, - RULE_subquery = 16, RULE_labelMatcher = 17, RULE_labelMatcherOp = 18, - RULE_labelMatcherList = 19, RULE_function = 20, RULE_parameter = 21, RULE_parameterList = 22, - RULE_aggregation = 23, RULE_by = 24, RULE_without = 25, RULE_grouping = 26, - RULE_on = 27, RULE_ignoring = 28, RULE_groupLeft = 29, RULE_groupRight = 30, - RULE_metricName = 31, RULE_metricKeyword = 32, RULE_labelName = 33, RULE_labelNameList = 34, - RULE_labelKeyword = 35, RULE_literal = 36; + RULE_instantSelector = 12, RULE_window = 13, RULE_offset = 14, RULE_atModifier = 15, + RULE_modifier = 16, RULE_limit = 17, RULE_subquery = 18, RULE_labelMatcher = 19, + RULE_labelMatcherOp = 20, RULE_labelMatcherList = 21, RULE_function = 22, + RULE_parameter = 23, RULE_parameterList = 24, RULE_aggregation = 25, RULE_by = 26, + RULE_without = 27, RULE_grouping = 28, RULE_on = 29, RULE_ignoring = 30, + RULE_groupLeft = 31, RULE_groupRight = 32, RULE_metricName = 33, RULE_metricKeyword = 34, + RULE_labelName = 35, RULE_labelNameList = 36, RULE_labelKeyword = 37, + RULE_literal = 38; private static String[] makeRuleNames() { return new String[] { "expression", "vectorExpression", "unaryOp", "powOp", "multOp", "addOp", "compareOp", "andUnlessOp", "orOp", "vector", "parens", "instantOrRangeSelector", - "instantSelector", "window", "offset", "limit", "subquery", "labelMatcher", - "labelMatcherOp", "labelMatcherList", "function", "parameter", "parameterList", - "aggregation", "by", "without", "grouping", "on", "ignoring", "groupLeft", - "groupRight", "metricName", "metricKeyword", "labelName", "labelNameList", - "labelKeyword", "literal" + "instantSelector", "window", "offset", "atModifier", "modifier", "limit", + "subquery", "labelMatcher", "labelMatcherOp", "labelMatcherList", "function", + "parameter", "parameterList", "aggregation", "by", "without", "grouping", + "on", "ignoring", "groupLeft", "groupRight", "metricName", "metricKeyword", + "labelName", "labelNameList", "labelKeyword", "literal" }; } public static final String[] ruleNames = makeRuleNames(); @@ -51,7 +52,7 @@ private static String[] makeLiteralNames() { return new String[] { null, "'('", "')'", "'{'", "'}'", "'['", "']'", "':'", "','", null, null, "'+'", "'-'", "'*'", "'/'", "'%'", "'^'", "'='", "'=='", "'!='", "'>'", - "'<'", "'>='", "'<='", "'=~'", "'!~'" + "'<'", "'>='", "'<='", "'=~'", "'!~'", "'@'" }; } private static final String[] _LITERAL_NAMES = makeLiteralNames(); @@ -59,9 +60,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(); @@ -137,9 +139,9 @@ public final ExpressionContext expression() throws RecognitionException { try { enterOuterAlt(_localctx, 1); { - setState(74); + setState(78); vectorExpression(0); - setState(75); + setState(79); match(EOF); } } @@ -232,8 +234,8 @@ public VectorExpressionContext vectorExpression() { public SubqueryContext subquery() { return getRuleContext(SubqueryContext.class,0); } - public OffsetContext offset() { - return getRuleContext(OffsetContext.class,0); + public ModifierContext modifier() { + return getRuleContext(ModifierContext.class,0); } public SubqueryOperationContext(VectorExpressionContext ctx) { copyFrom(ctx); } @Override @@ -273,7 +275,7 @@ private VectorExpressionContext vectorExpression(int _p) throws RecognitionExcep int _alt; enterOuterAlt(_localctx, 1); { - setState(82); + setState(86); _errHandler.sync(this); switch (_input.LA(1)) { case ADD: @@ -283,9 +285,9 @@ private VectorExpressionContext vectorExpression(int _p) throws RecognitionExcep _ctx = _localctx; _prevctx = _localctx; - setState(78); + setState(82); unaryOp(); - setState(79); + setState(83); vectorExpression(9); } break; @@ -307,7 +309,7 @@ private VectorExpressionContext vectorExpression(int _p) throws RecognitionExcep _localctx = new VectorOperationContext(_localctx); _ctx = _localctx; _prevctx = _localctx; - setState(81); + setState(85); vector(); } break; @@ -315,7 +317,7 @@ private VectorExpressionContext vectorExpression(int _p) throws RecognitionExcep throw new NoViableAltException(this); } _ctx.stop = _input.LT(-1); - setState(135); + setState(139); _errHandler.sync(this); _alt = getInterpreter().adaptivePredict(_input,9,_ctx); while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { @@ -323,28 +325,28 @@ private VectorExpressionContext vectorExpression(int _p) throws RecognitionExcep if ( _parseListeners!=null ) triggerExitRuleEvent(); _prevctx = _localctx; { - setState(133); + setState(137); _errHandler.sync(this); switch ( getInterpreter().adaptivePredict(_input,8,_ctx) ) { case 1: { _localctx = new BinaryOperationContext(new VectorExpressionContext(_parentctx, _parentState)); pushNewRecursionContext(_localctx, _startState, RULE_vectorExpression); - setState(84); + setState(88); if (!(precpred(_ctx, 10))) throw new FailedPredicateException(this, "precpred(_ctx, 10)"); - setState(85); + setState(89); powOp(); - setState(87); + setState(91); _errHandler.sync(this); _la = _input.LA(1); if (_la==ON || _la==IGNORING) { { - setState(86); + setState(90); grouping(); } } - setState(89); + setState(93); vectorExpression(10); } break; @@ -352,21 +354,21 @@ private VectorExpressionContext vectorExpression(int _p) throws RecognitionExcep { _localctx = new BinaryOperationContext(new VectorExpressionContext(_parentctx, _parentState)); pushNewRecursionContext(_localctx, _startState, RULE_vectorExpression); - setState(91); + setState(95); if (!(precpred(_ctx, 8))) throw new FailedPredicateException(this, "precpred(_ctx, 8)"); - setState(92); + setState(96); multOp(); - setState(94); + setState(98); _errHandler.sync(this); _la = _input.LA(1); if (_la==ON || _la==IGNORING) { { - setState(93); + setState(97); grouping(); } } - setState(96); + setState(100); vectorExpression(9); } break; @@ -374,21 +376,21 @@ private VectorExpressionContext vectorExpression(int _p) throws RecognitionExcep { _localctx = new BinaryOperationContext(new VectorExpressionContext(_parentctx, _parentState)); pushNewRecursionContext(_localctx, _startState, RULE_vectorExpression); - setState(98); + setState(102); if (!(precpred(_ctx, 7))) throw new FailedPredicateException(this, "precpred(_ctx, 7)"); - setState(99); + setState(103); addOp(); - setState(101); + setState(105); _errHandler.sync(this); _la = _input.LA(1); if (_la==ON || _la==IGNORING) { { - setState(100); + setState(104); grouping(); } } - setState(103); + setState(107); vectorExpression(8); } break; @@ -396,21 +398,21 @@ private VectorExpressionContext vectorExpression(int _p) throws RecognitionExcep { _localctx = new BinaryOperationContext(new VectorExpressionContext(_parentctx, _parentState)); pushNewRecursionContext(_localctx, _startState, RULE_vectorExpression); - setState(105); + setState(109); if (!(precpred(_ctx, 6))) throw new FailedPredicateException(this, "precpred(_ctx, 6)"); - setState(106); + setState(110); compareOp(); - setState(108); + setState(112); _errHandler.sync(this); _la = _input.LA(1); if (_la==ON || _la==IGNORING) { { - setState(107); + setState(111); grouping(); } } - setState(110); + setState(114); vectorExpression(7); } break; @@ -418,21 +420,21 @@ private VectorExpressionContext vectorExpression(int _p) throws RecognitionExcep { _localctx = new BinaryOperationContext(new VectorExpressionContext(_parentctx, _parentState)); pushNewRecursionContext(_localctx, _startState, RULE_vectorExpression); - setState(112); + setState(116); if (!(precpred(_ctx, 5))) throw new FailedPredicateException(this, "precpred(_ctx, 5)"); - setState(113); + setState(117); andUnlessOp(); - setState(115); + setState(119); _errHandler.sync(this); _la = _input.LA(1); if (_la==ON || _la==IGNORING) { { - setState(114); + setState(118); grouping(); } } - setState(117); + setState(121); vectorExpression(6); } break; @@ -440,21 +442,21 @@ private VectorExpressionContext vectorExpression(int _p) throws RecognitionExcep { _localctx = new BinaryOperationContext(new VectorExpressionContext(_parentctx, _parentState)); pushNewRecursionContext(_localctx, _startState, RULE_vectorExpression); - setState(119); + setState(123); if (!(precpred(_ctx, 4))) throw new FailedPredicateException(this, "precpred(_ctx, 4)"); - setState(120); + setState(124); orOp(); - setState(122); + setState(126); _errHandler.sync(this); _la = _input.LA(1); if (_la==ON || _la==IGNORING) { { - setState(121); + setState(125); grouping(); } } - setState(124); + setState(128); vectorExpression(5); } break; @@ -462,17 +464,17 @@ private VectorExpressionContext vectorExpression(int _p) throws RecognitionExcep { _localctx = new SubqueryOperationContext(new VectorExpressionContext(_parentctx, _parentState)); pushNewRecursionContext(_localctx, _startState, RULE_vectorExpression); - setState(126); + setState(130); if (!(precpred(_ctx, 3))) throw new FailedPredicateException(this, "precpred(_ctx, 3)"); - setState(127); + setState(131); subquery(); - setState(129); + setState(133); _errHandler.sync(this); switch ( getInterpreter().adaptivePredict(_input,7,_ctx) ) { case 1: { - setState(128); - offset(); + setState(132); + modifier(); } break; } @@ -482,16 +484,16 @@ private VectorExpressionContext vectorExpression(int _p) throws RecognitionExcep { _localctx = new LimitOperationContext(new VectorExpressionContext(_parentctx, _parentState)); pushNewRecursionContext(_localctx, _startState, RULE_vectorExpression); - setState(131); + setState(135); if (!(precpred(_ctx, 2))) throw new FailedPredicateException(this, "precpred(_ctx, 2)"); - setState(132); + setState(136); limit(); } break; } } } - setState(137); + setState(141); _errHandler.sync(this); _alt = getInterpreter().adaptivePredict(_input,9,_ctx); } @@ -529,7 +531,7 @@ public final UnaryOpContext unaryOp() throws RecognitionException { try { enterOuterAlt(_localctx, 1); { - setState(138); + setState(142); _la = _input.LA(1); if ( !(_la==ADD || _la==SUB) ) { _errHandler.recoverInline(this); @@ -571,7 +573,7 @@ public final PowOpContext powOp() throws RecognitionException { try { enterOuterAlt(_localctx, 1); { - setState(140); + setState(144); match(POW); } } @@ -608,7 +610,7 @@ public final MultOpContext multOp() throws RecognitionException { try { enterOuterAlt(_localctx, 1); { - setState(142); + setState(146); _la = _input.LA(1); if ( !((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << MUL) | (1L << DIV) | (1L << MOD))) != 0)) ) { _errHandler.recoverInline(this); @@ -652,7 +654,7 @@ public final AddOpContext addOp() throws RecognitionException { try { enterOuterAlt(_localctx, 1); { - setState(144); + setState(148); _la = _input.LA(1); if ( !(_la==ADD || _la==SUB) ) { _errHandler.recoverInline(this); @@ -701,7 +703,7 @@ public final CompareOpContext compareOp() throws RecognitionException { try { enterOuterAlt(_localctx, 1); { - setState(146); + setState(150); _la = _input.LA(1); if ( !((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << DEQ) | (1L << NE) | (1L << GT) | (1L << LT) | (1L << GE) | (1L << LE))) != 0)) ) { _errHandler.recoverInline(this); @@ -711,12 +713,12 @@ public final CompareOpContext compareOp() throws RecognitionException { _errHandler.reportMatch(this); consume(); } - setState(148); + setState(152); _errHandler.sync(this); _la = _input.LA(1); if (_la==BOOL) { { - setState(147); + setState(151); match(BOOL); } } @@ -755,7 +757,7 @@ public final AndUnlessOpContext andUnlessOp() throws RecognitionException { try { enterOuterAlt(_localctx, 1); { - setState(150); + setState(154); _la = _input.LA(1); if ( !(_la==AND || _la==UNLESS) ) { _errHandler.recoverInline(this); @@ -797,7 +799,7 @@ public final OrOpContext orOp() throws RecognitionException { try { enterOuterAlt(_localctx, 1); { - setState(152); + setState(156); match(OR); } } @@ -843,41 +845,41 @@ public final VectorContext vector() throws RecognitionException { VectorContext _localctx = new VectorContext(_ctx, getState()); enterRule(_localctx, 18, RULE_vector); try { - setState(159); + setState(163); _errHandler.sync(this); switch ( getInterpreter().adaptivePredict(_input,11,_ctx) ) { case 1: enterOuterAlt(_localctx, 1); { - setState(154); + setState(158); function(); } break; case 2: enterOuterAlt(_localctx, 2); { - setState(155); + setState(159); aggregation(); } break; case 3: enterOuterAlt(_localctx, 3); { - setState(156); + setState(160); instantOrRangeSelector(); } break; case 4: enterOuterAlt(_localctx, 4); { - setState(157); + setState(161); literal(); } break; case 5: enterOuterAlt(_localctx, 5); { - setState(158); + setState(162); parens(); } break; @@ -915,11 +917,11 @@ public final ParensContext parens() throws RecognitionException { try { enterOuterAlt(_localctx, 1); { - setState(161); + setState(165); match(T__0); - setState(162); + setState(166); vectorExpression(0); - setState(163); + setState(167); match(T__1); } } @@ -941,8 +943,8 @@ public InstantSelectorContext instantSelector() { public WindowContext window() { return getRuleContext(WindowContext.class,0); } - public OffsetContext offset() { - return getRuleContext(OffsetContext.class,0); + public ModifierContext modifier() { + return getRuleContext(ModifierContext.class,0); } public InstantOrRangeSelectorContext(ParserRuleContext parent, int invokingState) { super(parent, invokingState); @@ -961,25 +963,25 @@ public final InstantOrRangeSelectorContext instantOrRangeSelector() throws Recog try { enterOuterAlt(_localctx, 1); { - setState(165); + setState(169); instantSelector(); - setState(167); + setState(171); _errHandler.sync(this); switch ( getInterpreter().adaptivePredict(_input,12,_ctx) ) { case 1: { - setState(166); + setState(170); window(); } break; } - setState(170); + setState(174); _errHandler.sync(this); switch ( getInterpreter().adaptivePredict(_input,13,_ctx) ) { case 1: { - setState(169); - offset(); + setState(173); + modifier(); } break; } @@ -1019,7 +1021,7 @@ public final InstantSelectorContext instantSelector() throws RecognitionExceptio enterRule(_localctx, 24, RULE_instantSelector); int _la; try { - setState(184); + setState(188); _errHandler.sync(this); switch (_input.LA(1)) { case AND: @@ -1034,26 +1036,26 @@ public final InstantSelectorContext instantSelector() throws RecognitionExceptio case IDENTIFIER_EXTENDED: enterOuterAlt(_localctx, 1); { - setState(172); + setState(176); metricName(); - setState(178); + setState(182); _errHandler.sync(this); switch ( getInterpreter().adaptivePredict(_input,15,_ctx) ) { case 1: { - setState(173); + setState(177); match(T__2); - setState(175); + setState(179); _errHandler.sync(this); _la = _input.LA(1); if ((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << AND) | (1L << OR) | (1L << UNLESS) | (1L << BY) | (1L << WITHOUT) | (1L << ON) | (1L << IGNORING) | (1L << GROUP_LEFT) | (1L << GROUP_RIGHT) | (1L << OFFSET) | (1L << LIMIT) | (1L << BOOL) | (1L << AGGREGATION_OP) | (1L << IDENTIFIER))) != 0)) { { - setState(174); + setState(178); labelMatcherList(); } } - setState(177); + setState(181); match(T__3); } break; @@ -1063,11 +1065,11 @@ public final InstantSelectorContext instantSelector() throws RecognitionExceptio case T__2: enterOuterAlt(_localctx, 2); { - setState(180); + setState(184); match(T__2); - setState(181); + setState(185); labelMatcherList(); - setState(182); + setState(186); match(T__3); } break; @@ -1105,11 +1107,11 @@ public final WindowContext window() throws RecognitionException { try { enterOuterAlt(_localctx, 1); { - setState(186); + setState(190); match(T__4); - setState(187); + setState(191); match(DURATION); - setState(188); + setState(192); match(T__5); } } @@ -1144,9 +1146,9 @@ public final OffsetContext offset() throws RecognitionException { try { enterOuterAlt(_localctx, 1); { - setState(190); + setState(194); match(OFFSET); - setState(191); + setState(195); match(DURATION); } } @@ -1161,6 +1163,124 @@ public final OffsetContext offset() throws RecognitionException { return _localctx; } + public static class AtModifierContext extends ParserRuleContext { + public TerminalNode AT() { return getToken(PromQLParser.AT, 0); } + public TerminalNode NUMBER() { return getToken(PromQLParser.NUMBER, 0); } + public TerminalNode START() { return getToken(PromQLParser.START, 0); } + public TerminalNode END() { return getToken(PromQLParser.END, 0); } + public AtModifierContext(ParserRuleContext parent, int invokingState) { + super(parent, invokingState); + } + @Override public int getRuleIndex() { return RULE_atModifier; } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof PromQLVisitor ) return ((PromQLVisitor)visitor).visitAtModifier(this); + else return visitor.visitChildren(this); + } + } + + public final AtModifierContext atModifier() throws RecognitionException { + AtModifierContext _localctx = new AtModifierContext(_ctx, getState()); + enterRule(_localctx, 30, RULE_atModifier); + int _la; + try { + enterOuterAlt(_localctx, 1); + { + setState(197); + match(AT); + setState(198); + _la = _input.LA(1); + if ( !((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << NUMBER) | (1L << START) | (1L << END))) != 0)) ) { + _errHandler.recoverInline(this); + } + else { + if ( _input.LA(1)==Token.EOF ) matchedEOF = true; + _errHandler.reportMatch(this); + consume(); + } + } + } + catch (RecognitionException re) { + _localctx.exception = re; + _errHandler.reportError(this, re); + _errHandler.recover(this, re); + } + finally { + exitRule(); + } + return _localctx; + } + + public static class ModifierContext extends ParserRuleContext { + public OffsetContext offset() { + return getRuleContext(OffsetContext.class,0); + } + public AtModifierContext atModifier() { + return getRuleContext(AtModifierContext.class,0); + } + public ModifierContext(ParserRuleContext parent, int invokingState) { + super(parent, invokingState); + } + @Override public int getRuleIndex() { return RULE_modifier; } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof PromQLVisitor ) return ((PromQLVisitor)visitor).visitModifier(this); + else return visitor.visitChildren(this); + } + } + + public final ModifierContext modifier() throws RecognitionException { + ModifierContext _localctx = new ModifierContext(_ctx, getState()); + enterRule(_localctx, 32, RULE_modifier); + try { + setState(208); + _errHandler.sync(this); + switch ( getInterpreter().adaptivePredict(_input,17,_ctx) ) { + case 1: + enterOuterAlt(_localctx, 1); + { + setState(200); + offset(); + } + break; + case 2: + enterOuterAlt(_localctx, 2); + { + setState(201); + atModifier(); + } + break; + case 3: + enterOuterAlt(_localctx, 3); + { + setState(202); + offset(); + setState(203); + atModifier(); + } + break; + case 4: + enterOuterAlt(_localctx, 4); + { + setState(205); + atModifier(); + setState(206); + offset(); + } + break; + } + } + catch (RecognitionException re) { + _localctx.exception = re; + _errHandler.reportError(this, re); + _errHandler.recover(this, re); + } + finally { + exitRule(); + } + return _localctx; + } + public static class LimitContext extends ParserRuleContext { public TerminalNode LIMIT() { return getToken(PromQLParser.LIMIT, 0); } public TerminalNode NUMBER() { return getToken(PromQLParser.NUMBER, 0); } @@ -1177,13 +1297,13 @@ public T accept(ParseTreeVisitor visitor) { public final LimitContext limit() throws RecognitionException { LimitContext _localctx = new LimitContext(_ctx, getState()); - enterRule(_localctx, 30, RULE_limit); + enterRule(_localctx, 34, RULE_limit); try { enterOuterAlt(_localctx, 1); { - setState(193); + setState(210); match(LIMIT); - setState(194); + setState(211); match(NUMBER); } } @@ -1216,28 +1336,28 @@ public T accept(ParseTreeVisitor visitor) { public final SubqueryContext subquery() throws RecognitionException { SubqueryContext _localctx = new SubqueryContext(_ctx, getState()); - enterRule(_localctx, 32, RULE_subquery); + enterRule(_localctx, 36, RULE_subquery); int _la; try { enterOuterAlt(_localctx, 1); { - setState(196); + setState(213); match(T__4); - setState(197); + setState(214); match(DURATION); - setState(198); + setState(215); match(T__6); - setState(200); + setState(217); _errHandler.sync(this); _la = _input.LA(1); if (_la==DURATION) { { - setState(199); + setState(216); match(DURATION); } } - setState(202); + setState(219); match(T__5); } } @@ -1273,15 +1393,15 @@ public T accept(ParseTreeVisitor visitor) { public final LabelMatcherContext labelMatcher() throws RecognitionException { LabelMatcherContext _localctx = new LabelMatcherContext(_ctx, getState()); - enterRule(_localctx, 34, RULE_labelMatcher); + enterRule(_localctx, 38, RULE_labelMatcher); try { enterOuterAlt(_localctx, 1); { - setState(204); + setState(221); labelName(); - setState(205); + setState(222); labelMatcherOp(); - setState(206); + setState(223); match(STRING); } } @@ -1314,12 +1434,12 @@ public T accept(ParseTreeVisitor visitor) { public final LabelMatcherOpContext labelMatcherOp() throws RecognitionException { LabelMatcherOpContext _localctx = new LabelMatcherOpContext(_ctx, getState()); - enterRule(_localctx, 36, RULE_labelMatcherOp); + enterRule(_localctx, 40, RULE_labelMatcherOp); int _la; try { enterOuterAlt(_localctx, 1); { - setState(208); + setState(225); _la = _input.LA(1); if ( !((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << EQ) | (1L << NE) | (1L << RE) | (1L << NRE))) != 0)) ) { _errHandler.recoverInline(this); @@ -1362,26 +1482,26 @@ public T accept(ParseTreeVisitor visitor) { public final LabelMatcherListContext labelMatcherList() throws RecognitionException { LabelMatcherListContext _localctx = new LabelMatcherListContext(_ctx, getState()); - enterRule(_localctx, 38, RULE_labelMatcherList); + enterRule(_localctx, 42, RULE_labelMatcherList); int _la; try { enterOuterAlt(_localctx, 1); { - setState(210); + setState(227); labelMatcher(); - setState(215); + setState(232); _errHandler.sync(this); _la = _input.LA(1); while (_la==T__7) { { { - setState(211); + setState(228); match(T__7); - setState(212); + setState(229); labelMatcher(); } } - setState(217); + setState(234); _errHandler.sync(this); _la = _input.LA(1); } @@ -1416,13 +1536,13 @@ public T accept(ParseTreeVisitor visitor) { public final FunctionContext function() throws RecognitionException { FunctionContext _localctx = new FunctionContext(_ctx, getState()); - enterRule(_localctx, 40, RULE_function); + enterRule(_localctx, 44, RULE_function); try { enterOuterAlt(_localctx, 1); { - setState(218); + setState(235); match(IDENTIFIER); - setState(219); + setState(236); parameterList(); } } @@ -1457,22 +1577,22 @@ public T accept(ParseTreeVisitor visitor) { public final ParameterContext parameter() throws RecognitionException { ParameterContext _localctx = new ParameterContext(_ctx, getState()); - enterRule(_localctx, 42, RULE_parameter); + enterRule(_localctx, 46, RULE_parameter); try { - setState(223); + setState(240); _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,19,_ctx) ) { + switch ( getInterpreter().adaptivePredict(_input,20,_ctx) ) { case 1: enterOuterAlt(_localctx, 1); { - setState(221); + setState(238); literal(); } break; case 2: enterOuterAlt(_localctx, 2); { - setState(222); + setState(239); vectorExpression(0); } break; @@ -1509,40 +1629,40 @@ public T accept(ParseTreeVisitor visitor) { public final ParameterListContext parameterList() throws RecognitionException { ParameterListContext _localctx = new ParameterListContext(_ctx, getState()); - enterRule(_localctx, 44, RULE_parameterList); + enterRule(_localctx, 48, RULE_parameterList); int _la; try { enterOuterAlt(_localctx, 1); { - setState(225); + setState(242); match(T__0); - setState(234); + setState(251); _errHandler.sync(this); _la = _input.LA(1); if ((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << T__0) | (1L << T__2) | (1L << NUMBER) | (1L << STRING) | (1L << ADD) | (1L << SUB) | (1L << AND) | (1L << OR) | (1L << UNLESS) | (1L << BY) | (1L << WITHOUT) | (1L << OFFSET) | (1L << LIMIT) | (1L << AGGREGATION_OP) | (1L << IDENTIFIER) | (1L << IDENTIFIER_EXTENDED))) != 0)) { { - setState(226); + setState(243); parameter(); - setState(231); + setState(248); _errHandler.sync(this); _la = _input.LA(1); while (_la==T__7) { { { - setState(227); + setState(244); match(T__7); - setState(228); + setState(245); parameter(); } } - setState(233); + setState(250); _errHandler.sync(this); _la = _input.LA(1); } } } - setState(236); + setState(253); match(T__1); } } @@ -1581,66 +1701,66 @@ public T accept(ParseTreeVisitor visitor) { public final AggregationContext aggregation() throws RecognitionException { AggregationContext _localctx = new AggregationContext(_ctx, getState()); - enterRule(_localctx, 46, RULE_aggregation); + enterRule(_localctx, 50, RULE_aggregation); try { - setState(253); + setState(270); _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,24,_ctx) ) { + switch ( getInterpreter().adaptivePredict(_input,25,_ctx) ) { case 1: enterOuterAlt(_localctx, 1); { - setState(238); + setState(255); match(AGGREGATION_OP); - setState(239); + setState(256); parameterList(); } break; case 2: enterOuterAlt(_localctx, 2); { - setState(240); + setState(257); match(AGGREGATION_OP); - setState(243); + setState(260); _errHandler.sync(this); switch (_input.LA(1)) { case BY: { - setState(241); + setState(258); by(); } break; case WITHOUT: { - setState(242); + setState(259); without(); } break; default: throw new NoViableAltException(this); } - setState(245); + setState(262); parameterList(); } break; case 3: enterOuterAlt(_localctx, 3); { - setState(247); + setState(264); match(AGGREGATION_OP); - setState(248); + setState(265); parameterList(); - setState(251); + setState(268); _errHandler.sync(this); switch (_input.LA(1)) { case BY: { - setState(249); + setState(266); by(); } break; case WITHOUT: { - setState(250); + setState(267); without(); } break; @@ -1680,13 +1800,13 @@ public T accept(ParseTreeVisitor visitor) { public final ByContext by() throws RecognitionException { ByContext _localctx = new ByContext(_ctx, getState()); - enterRule(_localctx, 48, RULE_by); + enterRule(_localctx, 52, RULE_by); try { enterOuterAlt(_localctx, 1); { - setState(255); + setState(272); match(BY); - setState(256); + setState(273); labelNameList(); } } @@ -1719,13 +1839,13 @@ public T accept(ParseTreeVisitor visitor) { public final WithoutContext without() throws RecognitionException { WithoutContext _localctx = new WithoutContext(_ctx, getState()); - enterRule(_localctx, 50, RULE_without); + enterRule(_localctx, 54, RULE_without); try { enterOuterAlt(_localctx, 1); { - setState(258); + setState(275); match(WITHOUT); - setState(259); + setState(276); labelNameList(); } } @@ -1766,40 +1886,40 @@ public T accept(ParseTreeVisitor visitor) { public final GroupingContext grouping() throws RecognitionException { GroupingContext _localctx = new GroupingContext(_ctx, getState()); - enterRule(_localctx, 52, RULE_grouping); + enterRule(_localctx, 56, RULE_grouping); try { enterOuterAlt(_localctx, 1); { - setState(263); + setState(280); _errHandler.sync(this); switch (_input.LA(1)) { case ON: { - setState(261); + setState(278); on(); } break; case IGNORING: { - setState(262); + setState(279); ignoring(); } break; default: throw new NoViableAltException(this); } - setState(267); + setState(284); _errHandler.sync(this); switch (_input.LA(1)) { case GROUP_LEFT: { - setState(265); + setState(282); groupLeft(); } break; case GROUP_RIGHT: { - setState(266); + setState(283); groupRight(); } break; @@ -1854,13 +1974,13 @@ public T accept(ParseTreeVisitor visitor) { public final OnContext on() throws RecognitionException { OnContext _localctx = new OnContext(_ctx, getState()); - enterRule(_localctx, 54, RULE_on); + enterRule(_localctx, 58, RULE_on); try { enterOuterAlt(_localctx, 1); { - setState(269); + setState(286); match(ON); - setState(270); + setState(287); labelNameList(); } } @@ -1893,13 +2013,13 @@ public T accept(ParseTreeVisitor visitor) { public final IgnoringContext ignoring() throws RecognitionException { IgnoringContext _localctx = new IgnoringContext(_ctx, getState()); - enterRule(_localctx, 56, RULE_ignoring); + enterRule(_localctx, 60, RULE_ignoring); try { enterOuterAlt(_localctx, 1); { - setState(272); + setState(289); match(IGNORING); - setState(273); + setState(290); labelNameList(); } } @@ -1932,18 +2052,18 @@ public T accept(ParseTreeVisitor visitor) { public final GroupLeftContext groupLeft() throws RecognitionException { GroupLeftContext _localctx = new GroupLeftContext(_ctx, getState()); - enterRule(_localctx, 58, RULE_groupLeft); + enterRule(_localctx, 62, RULE_groupLeft); try { enterOuterAlt(_localctx, 1); { - setState(275); + setState(292); match(GROUP_LEFT); - setState(277); + setState(294); _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,27,_ctx) ) { + switch ( getInterpreter().adaptivePredict(_input,28,_ctx) ) { case 1: { - setState(276); + setState(293); labelNameList(); } break; @@ -1979,18 +2099,18 @@ public T accept(ParseTreeVisitor visitor) { public final GroupRightContext groupRight() throws RecognitionException { GroupRightContext _localctx = new GroupRightContext(_ctx, getState()); - enterRule(_localctx, 60, RULE_groupRight); + enterRule(_localctx, 64, RULE_groupRight); try { enterOuterAlt(_localctx, 1); { - setState(279); + setState(296); match(GROUP_RIGHT); - setState(281); + setState(298); _errHandler.sync(this); - switch ( getInterpreter().adaptivePredict(_input,28,_ctx) ) { + switch ( getInterpreter().adaptivePredict(_input,29,_ctx) ) { case 1: { - setState(280); + setState(297); labelNameList(); } break; @@ -2027,9 +2147,9 @@ public T accept(ParseTreeVisitor visitor) { public final MetricNameContext metricName() throws RecognitionException { MetricNameContext _localctx = new MetricNameContext(_ctx, getState()); - enterRule(_localctx, 62, RULE_metricName); + enterRule(_localctx, 66, RULE_metricName); try { - setState(286); + setState(303); _errHandler.sync(this); switch (_input.LA(1)) { case AND: @@ -2042,21 +2162,21 @@ public final MetricNameContext metricName() throws RecognitionException { case AGGREGATION_OP: enterOuterAlt(_localctx, 1); { - setState(283); + setState(300); metricKeyword(); } break; case IDENTIFIER: enterOuterAlt(_localctx, 2); { - setState(284); + setState(301); match(IDENTIFIER); } break; case IDENTIFIER_EXTENDED: enterOuterAlt(_localctx, 3); { - setState(285); + setState(302); match(IDENTIFIER_EXTENDED); } break; @@ -2097,12 +2217,12 @@ public T accept(ParseTreeVisitor visitor) { public final MetricKeywordContext metricKeyword() throws RecognitionException { MetricKeywordContext _localctx = new MetricKeywordContext(_ctx, getState()); - enterRule(_localctx, 64, RULE_metricKeyword); + enterRule(_localctx, 68, RULE_metricKeyword); int _la; try { enterOuterAlt(_localctx, 1); { - setState(288); + setState(305); _la = _input.LA(1); if ( !((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << AND) | (1L << OR) | (1L << UNLESS) | (1L << BY) | (1L << WITHOUT) | (1L << OFFSET) | (1L << LIMIT) | (1L << AGGREGATION_OP))) != 0)) ) { _errHandler.recoverInline(this); @@ -2143,9 +2263,9 @@ public T accept(ParseTreeVisitor visitor) { public final LabelNameContext labelName() throws RecognitionException { LabelNameContext _localctx = new LabelNameContext(_ctx, getState()); - enterRule(_localctx, 66, RULE_labelName); + enterRule(_localctx, 70, RULE_labelName); try { - setState(292); + setState(309); _errHandler.sync(this); switch (_input.LA(1)) { case AND: @@ -2163,14 +2283,14 @@ public final LabelNameContext labelName() throws RecognitionException { case AGGREGATION_OP: enterOuterAlt(_localctx, 1); { - setState(290); + setState(307); labelKeyword(); } break; case IDENTIFIER: enterOuterAlt(_localctx, 2); { - setState(291); + setState(308); match(IDENTIFIER); } break; @@ -2209,53 +2329,53 @@ public T accept(ParseTreeVisitor visitor) { public final LabelNameListContext labelNameList() throws RecognitionException { LabelNameListContext _localctx = new LabelNameListContext(_ctx, getState()); - enterRule(_localctx, 68, RULE_labelNameList); + enterRule(_localctx, 72, RULE_labelNameList); int _la; try { int _alt; enterOuterAlt(_localctx, 1); { - setState(294); + setState(311); match(T__0); - setState(303); + setState(320); _errHandler.sync(this); _la = _input.LA(1); if ((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << AND) | (1L << OR) | (1L << UNLESS) | (1L << BY) | (1L << WITHOUT) | (1L << ON) | (1L << IGNORING) | (1L << GROUP_LEFT) | (1L << GROUP_RIGHT) | (1L << OFFSET) | (1L << LIMIT) | (1L << BOOL) | (1L << AGGREGATION_OP) | (1L << IDENTIFIER))) != 0)) { { - setState(295); + setState(312); labelName(); - setState(300); + setState(317); _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,31,_ctx); + _alt = getInterpreter().adaptivePredict(_input,32,_ctx); while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { if ( _alt==1 ) { { { - setState(296); + setState(313); match(T__7); - setState(297); + setState(314); labelName(); } } } - setState(302); + setState(319); _errHandler.sync(this); - _alt = getInterpreter().adaptivePredict(_input,31,_ctx); + _alt = getInterpreter().adaptivePredict(_input,32,_ctx); } } } - setState(306); + setState(323); _errHandler.sync(this); _la = _input.LA(1); if (_la==T__7) { { - setState(305); + setState(322); match(T__7); } } - setState(308); + setState(325); match(T__1); } } @@ -2297,12 +2417,12 @@ public T accept(ParseTreeVisitor visitor) { public final LabelKeywordContext labelKeyword() throws RecognitionException { LabelKeywordContext _localctx = new LabelKeywordContext(_ctx, getState()); - enterRule(_localctx, 70, RULE_labelKeyword); + enterRule(_localctx, 74, RULE_labelKeyword); int _la; try { enterOuterAlt(_localctx, 1); { - setState(310); + setState(327); _la = _input.LA(1); if ( !((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << AND) | (1L << OR) | (1L << UNLESS) | (1L << BY) | (1L << WITHOUT) | (1L << ON) | (1L << IGNORING) | (1L << GROUP_LEFT) | (1L << GROUP_RIGHT) | (1L << OFFSET) | (1L << LIMIT) | (1L << BOOL) | (1L << AGGREGATION_OP))) != 0)) ) { _errHandler.recoverInline(this); @@ -2341,12 +2461,12 @@ public T accept(ParseTreeVisitor visitor) { public final LiteralContext literal() throws RecognitionException { LiteralContext _localctx = new LiteralContext(_ctx, getState()); - enterRule(_localctx, 72, RULE_literal); + enterRule(_localctx, 76, RULE_literal); int _la; try { enterOuterAlt(_localctx, 1); { - setState(312); + setState(329); _la = _input.LA(1); if ( !(_la==NUMBER || _la==STRING) ) { _errHandler.recoverInline(this); @@ -2399,113 +2519,121 @@ private boolean vectorExpression_sempred(VectorExpressionContext _localctx, int } public static final String _serializedATN = - "\3\u608b\ua72a\u8133\ub9ed\u417c\u3be7\u7786\u5964\3-\u013d\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"+ + "\3\u608b\ua72a\u8133\ub9ed\u417c\u3be7\u7786\u5964\3\60\u014e\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&\3\2\3\2\3\2\3\3\3\3\3\3\3\3\3\3\5"+ - "\3U\n\3\3\3\3\3\3\3\5\3Z\n\3\3\3\3\3\3\3\3\3\3\3\5\3a\n\3\3\3\3\3\3\3"+ - "\3\3\3\3\5\3h\n\3\3\3\3\3\3\3\3\3\3\3\5\3o\n\3\3\3\3\3\3\3\3\3\3\3\5\3"+ - "v\n\3\3\3\3\3\3\3\3\3\3\3\5\3}\n\3\3\3\3\3\3\3\3\3\3\3\5\3\u0084\n\3\3"+ - "\3\3\3\7\3\u0088\n\3\f\3\16\3\u008b\13\3\3\4\3\4\3\5\3\5\3\6\3\6\3\7\3"+ - "\7\3\b\3\b\5\b\u0097\n\b\3\t\3\t\3\n\3\n\3\13\3\13\3\13\3\13\3\13\5\13"+ - "\u00a2\n\13\3\f\3\f\3\f\3\f\3\r\3\r\5\r\u00aa\n\r\3\r\5\r\u00ad\n\r\3"+ - "\16\3\16\3\16\5\16\u00b2\n\16\3\16\5\16\u00b5\n\16\3\16\3\16\3\16\3\16"+ - "\5\16\u00bb\n\16\3\17\3\17\3\17\3\17\3\20\3\20\3\20\3\21\3\21\3\21\3\22"+ - "\3\22\3\22\3\22\5\22\u00cb\n\22\3\22\3\22\3\23\3\23\3\23\3\23\3\24\3\24"+ - "\3\25\3\25\3\25\7\25\u00d8\n\25\f\25\16\25\u00db\13\25\3\26\3\26\3\26"+ - "\3\27\3\27\5\27\u00e2\n\27\3\30\3\30\3\30\3\30\7\30\u00e8\n\30\f\30\16"+ - "\30\u00eb\13\30\5\30\u00ed\n\30\3\30\3\30\3\31\3\31\3\31\3\31\3\31\5\31"+ - "\u00f6\n\31\3\31\3\31\3\31\3\31\3\31\3\31\5\31\u00fe\n\31\5\31\u0100\n"+ - "\31\3\32\3\32\3\32\3\33\3\33\3\33\3\34\3\34\5\34\u010a\n\34\3\34\3\34"+ - "\5\34\u010e\n\34\3\35\3\35\3\35\3\36\3\36\3\36\3\37\3\37\5\37\u0118\n"+ - "\37\3 \3 \5 \u011c\n \3!\3!\3!\5!\u0121\n!\3\"\3\"\3#\3#\5#\u0127\n#\3"+ - "$\3$\3$\3$\7$\u012d\n$\f$\16$\u0130\13$\5$\u0132\n$\3$\5$\u0135\n$\3$"+ - "\3$\3%\3%\3&\3&\3&\2\3\4\'\2\4\6\b\n\f\16\20\22\24\26\30\32\34\36 \"$"+ - "&(*,.\60\62\64\668:<>@BDFHJ\2\n\3\2\r\16\3\2\17\21\3\2\24\31\4\2\34\34"+ - "\36\36\5\2\23\23\25\25\32\33\5\2\34 %&((\3\2\34(\3\2\13\f\2\u0145\2L\3"+ - "\2\2\2\4T\3\2\2\2\6\u008c\3\2\2\2\b\u008e\3\2\2\2\n\u0090\3\2\2\2\f\u0092"+ - "\3\2\2\2\16\u0094\3\2\2\2\20\u0098\3\2\2\2\22\u009a\3\2\2\2\24\u00a1\3"+ - "\2\2\2\26\u00a3\3\2\2\2\30\u00a7\3\2\2\2\32\u00ba\3\2\2\2\34\u00bc\3\2"+ - "\2\2\36\u00c0\3\2\2\2 \u00c3\3\2\2\2\"\u00c6\3\2\2\2$\u00ce\3\2\2\2&\u00d2"+ - "\3\2\2\2(\u00d4\3\2\2\2*\u00dc\3\2\2\2,\u00e1\3\2\2\2.\u00e3\3\2\2\2\60"+ - "\u00ff\3\2\2\2\62\u0101\3\2\2\2\64\u0104\3\2\2\2\66\u0109\3\2\2\28\u010f"+ - "\3\2\2\2:\u0112\3\2\2\2<\u0115\3\2\2\2>\u0119\3\2\2\2@\u0120\3\2\2\2B"+ - "\u0122\3\2\2\2D\u0126\3\2\2\2F\u0128\3\2\2\2H\u0138\3\2\2\2J\u013a\3\2"+ - "\2\2LM\5\4\3\2MN\7\2\2\3N\3\3\2\2\2OP\b\3\1\2PQ\5\6\4\2QR\5\4\3\13RU\3"+ - "\2\2\2SU\5\24\13\2TO\3\2\2\2TS\3\2\2\2U\u0089\3\2\2\2VW\f\f\2\2WY\5\b"+ - "\5\2XZ\5\66\34\2YX\3\2\2\2YZ\3\2\2\2Z[\3\2\2\2[\\\5\4\3\f\\\u0088\3\2"+ - "\2\2]^\f\n\2\2^`\5\n\6\2_a\5\66\34\2`_\3\2\2\2`a\3\2\2\2ab\3\2\2\2bc\5"+ - "\4\3\13c\u0088\3\2\2\2de\f\t\2\2eg\5\f\7\2fh\5\66\34\2gf\3\2\2\2gh\3\2"+ - "\2\2hi\3\2\2\2ij\5\4\3\nj\u0088\3\2\2\2kl\f\b\2\2ln\5\16\b\2mo\5\66\34"+ - "\2nm\3\2\2\2no\3\2\2\2op\3\2\2\2pq\5\4\3\tq\u0088\3\2\2\2rs\f\7\2\2su"+ - "\5\20\t\2tv\5\66\34\2ut\3\2\2\2uv\3\2\2\2vw\3\2\2\2wx\5\4\3\bx\u0088\3"+ - "\2\2\2yz\f\6\2\2z|\5\22\n\2{}\5\66\34\2|{\3\2\2\2|}\3\2\2\2}~\3\2\2\2"+ - "~\177\5\4\3\7\177\u0088\3\2\2\2\u0080\u0081\f\5\2\2\u0081\u0083\5\"\22"+ - "\2\u0082\u0084\5\36\20\2\u0083\u0082\3\2\2\2\u0083\u0084\3\2\2\2\u0084"+ - "\u0088\3\2\2\2\u0085\u0086\f\4\2\2\u0086\u0088\5 \21\2\u0087V\3\2\2\2"+ - "\u0087]\3\2\2\2\u0087d\3\2\2\2\u0087k\3\2\2\2\u0087r\3\2\2\2\u0087y\3"+ - "\2\2\2\u0087\u0080\3\2\2\2\u0087\u0085\3\2\2\2\u0088\u008b\3\2\2\2\u0089"+ - "\u0087\3\2\2\2\u0089\u008a\3\2\2\2\u008a\5\3\2\2\2\u008b\u0089\3\2\2\2"+ - "\u008c\u008d\t\2\2\2\u008d\7\3\2\2\2\u008e\u008f\7\22\2\2\u008f\t\3\2"+ - "\2\2\u0090\u0091\t\3\2\2\u0091\13\3\2\2\2\u0092\u0093\t\2\2\2\u0093\r"+ - "\3\2\2\2\u0094\u0096\t\4\2\2\u0095\u0097\7\'\2\2\u0096\u0095\3\2\2\2\u0096"+ - "\u0097\3\2\2\2\u0097\17\3\2\2\2\u0098\u0099\t\5\2\2\u0099\21\3\2\2\2\u009a"+ - "\u009b\7\35\2\2\u009b\23\3\2\2\2\u009c\u00a2\5*\26\2\u009d\u00a2\5\60"+ - "\31\2\u009e\u00a2\5\30\r\2\u009f\u00a2\5J&\2\u00a0\u00a2\5\26\f\2\u00a1"+ - "\u009c\3\2\2\2\u00a1\u009d\3\2\2\2\u00a1\u009e\3\2\2\2\u00a1\u009f\3\2"+ - "\2\2\u00a1\u00a0\3\2\2\2\u00a2\25\3\2\2\2\u00a3\u00a4\7\3\2\2\u00a4\u00a5"+ - "\5\4\3\2\u00a5\u00a6\7\4\2\2\u00a6\27\3\2\2\2\u00a7\u00a9\5\32\16\2\u00a8"+ - "\u00aa\5\34\17\2\u00a9\u00a8\3\2\2\2\u00a9\u00aa\3\2\2\2\u00aa\u00ac\3"+ - "\2\2\2\u00ab\u00ad\5\36\20\2\u00ac\u00ab\3\2\2\2\u00ac\u00ad\3\2\2\2\u00ad"+ - "\31\3\2\2\2\u00ae\u00b4\5@!\2\u00af\u00b1\7\5\2\2\u00b0\u00b2\5(\25\2"+ - "\u00b1\u00b0\3\2\2\2\u00b1\u00b2\3\2\2\2\u00b2\u00b3\3\2\2\2\u00b3\u00b5"+ - "\7\6\2\2\u00b4\u00af\3\2\2\2\u00b4\u00b5\3\2\2\2\u00b5\u00bb\3\2\2\2\u00b6"+ - "\u00b7\7\5\2\2\u00b7\u00b8\5(\25\2\u00b8\u00b9\7\6\2\2\u00b9\u00bb\3\2"+ - "\2\2\u00ba\u00ae\3\2\2\2\u00ba\u00b6\3\2\2\2\u00bb\33\3\2\2\2\u00bc\u00bd"+ - "\7\7\2\2\u00bd\u00be\7)\2\2\u00be\u00bf\7\b\2\2\u00bf\35\3\2\2\2\u00c0"+ - "\u00c1\7%\2\2\u00c1\u00c2\7)\2\2\u00c2\37\3\2\2\2\u00c3\u00c4\7&\2\2\u00c4"+ - "\u00c5\7\13\2\2\u00c5!\3\2\2\2\u00c6\u00c7\7\7\2\2\u00c7\u00c8\7)\2\2"+ - "\u00c8\u00ca\7\t\2\2\u00c9\u00cb\7)\2\2\u00ca\u00c9\3\2\2\2\u00ca\u00cb"+ - "\3\2\2\2\u00cb\u00cc\3\2\2\2\u00cc\u00cd\7\b\2\2\u00cd#\3\2\2\2\u00ce"+ - "\u00cf\5D#\2\u00cf\u00d0\5&\24\2\u00d0\u00d1\7\f\2\2\u00d1%\3\2\2\2\u00d2"+ - "\u00d3\t\6\2\2\u00d3\'\3\2\2\2\u00d4\u00d9\5$\23\2\u00d5\u00d6\7\n\2\2"+ - "\u00d6\u00d8\5$\23\2\u00d7\u00d5\3\2\2\2\u00d8\u00db\3\2\2\2\u00d9\u00d7"+ - "\3\2\2\2\u00d9\u00da\3\2\2\2\u00da)\3\2\2\2\u00db\u00d9\3\2\2\2\u00dc"+ - "\u00dd\7*\2\2\u00dd\u00de\5.\30\2\u00de+\3\2\2\2\u00df\u00e2\5J&\2\u00e0"+ - "\u00e2\5\4\3\2\u00e1\u00df\3\2\2\2\u00e1\u00e0\3\2\2\2\u00e2-\3\2\2\2"+ - "\u00e3\u00ec\7\3\2\2\u00e4\u00e9\5,\27\2\u00e5\u00e6\7\n\2\2\u00e6\u00e8"+ - "\5,\27\2\u00e7\u00e5\3\2\2\2\u00e8\u00eb\3\2\2\2\u00e9\u00e7\3\2\2\2\u00e9"+ - "\u00ea\3\2\2\2\u00ea\u00ed\3\2\2\2\u00eb\u00e9\3\2\2\2\u00ec\u00e4\3\2"+ - "\2\2\u00ec\u00ed\3\2\2\2\u00ed\u00ee\3\2\2\2\u00ee\u00ef\7\4\2\2\u00ef"+ - "/\3\2\2\2\u00f0\u00f1\7(\2\2\u00f1\u0100\5.\30\2\u00f2\u00f5\7(\2\2\u00f3"+ - "\u00f6\5\62\32\2\u00f4\u00f6\5\64\33\2\u00f5\u00f3\3\2\2\2\u00f5\u00f4"+ - "\3\2\2\2\u00f6\u00f7\3\2\2\2\u00f7\u00f8\5.\30\2\u00f8\u0100\3\2\2\2\u00f9"+ - "\u00fa\7(\2\2\u00fa\u00fd\5.\30\2\u00fb\u00fe\5\62\32\2\u00fc\u00fe\5"+ - "\64\33\2\u00fd\u00fb\3\2\2\2\u00fd\u00fc\3\2\2\2\u00fe\u0100\3\2\2\2\u00ff"+ - "\u00f0\3\2\2\2\u00ff\u00f2\3\2\2\2\u00ff\u00f9\3\2\2\2\u0100\61\3\2\2"+ - "\2\u0101\u0102\7\37\2\2\u0102\u0103\5F$\2\u0103\63\3\2\2\2\u0104\u0105"+ - "\7 \2\2\u0105\u0106\5F$\2\u0106\65\3\2\2\2\u0107\u010a\58\35\2\u0108\u010a"+ - "\5:\36\2\u0109\u0107\3\2\2\2\u0109\u0108\3\2\2\2\u010a\u010d\3\2\2\2\u010b"+ - "\u010e\5<\37\2\u010c\u010e\5> \2\u010d\u010b\3\2\2\2\u010d\u010c\3\2\2"+ - "\2\u010d\u010e\3\2\2\2\u010e\67\3\2\2\2\u010f\u0110\7!\2\2\u0110\u0111"+ - "\5F$\2\u01119\3\2\2\2\u0112\u0113\7\"\2\2\u0113\u0114\5F$\2\u0114;\3\2"+ - "\2\2\u0115\u0117\7#\2\2\u0116\u0118\5F$\2\u0117\u0116\3\2\2\2\u0117\u0118"+ - "\3\2\2\2\u0118=\3\2\2\2\u0119\u011b\7$\2\2\u011a\u011c\5F$\2\u011b\u011a"+ - "\3\2\2\2\u011b\u011c\3\2\2\2\u011c?\3\2\2\2\u011d\u0121\5B\"\2\u011e\u0121"+ - "\7*\2\2\u011f\u0121\7+\2\2\u0120\u011d\3\2\2\2\u0120\u011e\3\2\2\2\u0120"+ - "\u011f\3\2\2\2\u0121A\3\2\2\2\u0122\u0123\t\7\2\2\u0123C\3\2\2\2\u0124"+ - "\u0127\5H%\2\u0125\u0127\7*\2\2\u0126\u0124\3\2\2\2\u0126\u0125\3\2\2"+ - "\2\u0127E\3\2\2\2\u0128\u0131\7\3\2\2\u0129\u012e\5D#\2\u012a\u012b\7"+ - "\n\2\2\u012b\u012d\5D#\2\u012c\u012a\3\2\2\2\u012d\u0130\3\2\2\2\u012e"+ - "\u012c\3\2\2\2\u012e\u012f\3\2\2\2\u012f\u0132\3\2\2\2\u0130\u012e\3\2"+ - "\2\2\u0131\u0129\3\2\2\2\u0131\u0132\3\2\2\2\u0132\u0134\3\2\2\2\u0133"+ - "\u0135\7\n\2\2\u0134\u0133\3\2\2\2\u0134\u0135\3\2\2\2\u0135\u0136\3\2"+ - "\2\2\u0136\u0137\7\4\2\2\u0137G\3\2\2\2\u0138\u0139\t\b\2\2\u0139I\3\2"+ - "\2\2\u013a\u013b\t\t\2\2\u013bK\3\2\2\2$TY`gnu|\u0083\u0087\u0089\u0096"+ - "\u00a1\u00a9\u00ac\u00b1\u00b4\u00ba\u00ca\u00d9\u00e1\u00e9\u00ec\u00f5"+ - "\u00fd\u00ff\u0109\u010d\u0117\u011b\u0120\u0126\u012e\u0131\u0134"; + "\t!\4\"\t\"\4#\t#\4$\t$\4%\t%\4&\t&\4\'\t\'\4(\t(\3\2\3\2\3\2\3\3\3\3"+ + "\3\3\3\3\3\3\5\3Y\n\3\3\3\3\3\3\3\5\3^\n\3\3\3\3\3\3\3\3\3\3\3\5\3e\n"+ + "\3\3\3\3\3\3\3\3\3\3\3\5\3l\n\3\3\3\3\3\3\3\3\3\3\3\5\3s\n\3\3\3\3\3\3"+ + "\3\3\3\3\3\5\3z\n\3\3\3\3\3\3\3\3\3\3\3\5\3\u0081\n\3\3\3\3\3\3\3\3\3"+ + "\3\3\5\3\u0088\n\3\3\3\3\3\7\3\u008c\n\3\f\3\16\3\u008f\13\3\3\4\3\4\3"+ + "\5\3\5\3\6\3\6\3\7\3\7\3\b\3\b\5\b\u009b\n\b\3\t\3\t\3\n\3\n\3\13\3\13"+ + "\3\13\3\13\3\13\5\13\u00a6\n\13\3\f\3\f\3\f\3\f\3\r\3\r\5\r\u00ae\n\r"+ + "\3\r\5\r\u00b1\n\r\3\16\3\16\3\16\5\16\u00b6\n\16\3\16\5\16\u00b9\n\16"+ + "\3\16\3\16\3\16\3\16\5\16\u00bf\n\16\3\17\3\17\3\17\3\17\3\20\3\20\3\20"+ + "\3\21\3\21\3\21\3\22\3\22\3\22\3\22\3\22\3\22\3\22\3\22\5\22\u00d3\n\22"+ + "\3\23\3\23\3\23\3\24\3\24\3\24\3\24\5\24\u00dc\n\24\3\24\3\24\3\25\3\25"+ + "\3\25\3\25\3\26\3\26\3\27\3\27\3\27\7\27\u00e9\n\27\f\27\16\27\u00ec\13"+ + "\27\3\30\3\30\3\30\3\31\3\31\5\31\u00f3\n\31\3\32\3\32\3\32\3\32\7\32"+ + "\u00f9\n\32\f\32\16\32\u00fc\13\32\5\32\u00fe\n\32\3\32\3\32\3\33\3\33"+ + "\3\33\3\33\3\33\5\33\u0107\n\33\3\33\3\33\3\33\3\33\3\33\3\33\5\33\u010f"+ + "\n\33\5\33\u0111\n\33\3\34\3\34\3\34\3\35\3\35\3\35\3\36\3\36\5\36\u011b"+ + "\n\36\3\36\3\36\5\36\u011f\n\36\3\37\3\37\3\37\3 \3 \3 \3!\3!\5!\u0129"+ + "\n!\3\"\3\"\5\"\u012d\n\"\3#\3#\3#\5#\u0132\n#\3$\3$\3%\3%\5%\u0138\n"+ + "%\3&\3&\3&\3&\7&\u013e\n&\f&\16&\u0141\13&\5&\u0143\n&\3&\5&\u0146\n&"+ + "\3&\3&\3\'\3\'\3(\3(\3(\2\3\4)\2\4\6\b\n\f\16\20\22\24\26\30\32\34\36"+ + " \"$&(*,.\60\62\64\668:<>@BDFHJLN\2\13\3\2\r\16\3\2\17\21\3\2\24\31\4"+ + "\2\35\35\37\37\4\2\13\13)*\5\2\23\23\25\25\32\33\5\2\35!&\'++\4\2\35("+ + "++\3\2\13\f\2\u0157\2P\3\2\2\2\4X\3\2\2\2\6\u0090\3\2\2\2\b\u0092\3\2"+ + "\2\2\n\u0094\3\2\2\2\f\u0096\3\2\2\2\16\u0098\3\2\2\2\20\u009c\3\2\2\2"+ + "\22\u009e\3\2\2\2\24\u00a5\3\2\2\2\26\u00a7\3\2\2\2\30\u00ab\3\2\2\2\32"+ + "\u00be\3\2\2\2\34\u00c0\3\2\2\2\36\u00c4\3\2\2\2 \u00c7\3\2\2\2\"\u00d2"+ + "\3\2\2\2$\u00d4\3\2\2\2&\u00d7\3\2\2\2(\u00df\3\2\2\2*\u00e3\3\2\2\2,"+ + "\u00e5\3\2\2\2.\u00ed\3\2\2\2\60\u00f2\3\2\2\2\62\u00f4\3\2\2\2\64\u0110"+ + "\3\2\2\2\66\u0112\3\2\2\28\u0115\3\2\2\2:\u011a\3\2\2\2<\u0120\3\2\2\2"+ + ">\u0123\3\2\2\2@\u0126\3\2\2\2B\u012a\3\2\2\2D\u0131\3\2\2\2F\u0133\3"+ + "\2\2\2H\u0137\3\2\2\2J\u0139\3\2\2\2L\u0149\3\2\2\2N\u014b\3\2\2\2PQ\5"+ + "\4\3\2QR\7\2\2\3R\3\3\2\2\2ST\b\3\1\2TU\5\6\4\2UV\5\4\3\13VY\3\2\2\2W"+ + "Y\5\24\13\2XS\3\2\2\2XW\3\2\2\2Y\u008d\3\2\2\2Z[\f\f\2\2[]\5\b\5\2\\^"+ + "\5:\36\2]\\\3\2\2\2]^\3\2\2\2^_\3\2\2\2_`\5\4\3\f`\u008c\3\2\2\2ab\f\n"+ + "\2\2bd\5\n\6\2ce\5:\36\2dc\3\2\2\2de\3\2\2\2ef\3\2\2\2fg\5\4\3\13g\u008c"+ + "\3\2\2\2hi\f\t\2\2ik\5\f\7\2jl\5:\36\2kj\3\2\2\2kl\3\2\2\2lm\3\2\2\2m"+ + "n\5\4\3\nn\u008c\3\2\2\2op\f\b\2\2pr\5\16\b\2qs\5:\36\2rq\3\2\2\2rs\3"+ + "\2\2\2st\3\2\2\2tu\5\4\3\tu\u008c\3\2\2\2vw\f\7\2\2wy\5\20\t\2xz\5:\36"+ + "\2yx\3\2\2\2yz\3\2\2\2z{\3\2\2\2{|\5\4\3\b|\u008c\3\2\2\2}~\f\6\2\2~\u0080"+ + "\5\22\n\2\177\u0081\5:\36\2\u0080\177\3\2\2\2\u0080\u0081\3\2\2\2\u0081"+ + "\u0082\3\2\2\2\u0082\u0083\5\4\3\7\u0083\u008c\3\2\2\2\u0084\u0085\f\5"+ + "\2\2\u0085\u0087\5&\24\2\u0086\u0088\5\"\22\2\u0087\u0086\3\2\2\2\u0087"+ + "\u0088\3\2\2\2\u0088\u008c\3\2\2\2\u0089\u008a\f\4\2\2\u008a\u008c\5$"+ + "\23\2\u008bZ\3\2\2\2\u008ba\3\2\2\2\u008bh\3\2\2\2\u008bo\3\2\2\2\u008b"+ + "v\3\2\2\2\u008b}\3\2\2\2\u008b\u0084\3\2\2\2\u008b\u0089\3\2\2\2\u008c"+ + "\u008f\3\2\2\2\u008d\u008b\3\2\2\2\u008d\u008e\3\2\2\2\u008e\5\3\2\2\2"+ + "\u008f\u008d\3\2\2\2\u0090\u0091\t\2\2\2\u0091\7\3\2\2\2\u0092\u0093\7"+ + "\22\2\2\u0093\t\3\2\2\2\u0094\u0095\t\3\2\2\u0095\13\3\2\2\2\u0096\u0097"+ + "\t\2\2\2\u0097\r\3\2\2\2\u0098\u009a\t\4\2\2\u0099\u009b\7(\2\2\u009a"+ + "\u0099\3\2\2\2\u009a\u009b\3\2\2\2\u009b\17\3\2\2\2\u009c\u009d\t\5\2"+ + "\2\u009d\21\3\2\2\2\u009e\u009f\7\36\2\2\u009f\23\3\2\2\2\u00a0\u00a6"+ + "\5.\30\2\u00a1\u00a6\5\64\33\2\u00a2\u00a6\5\30\r\2\u00a3\u00a6\5N(\2"+ + "\u00a4\u00a6\5\26\f\2\u00a5\u00a0\3\2\2\2\u00a5\u00a1\3\2\2\2\u00a5\u00a2"+ + "\3\2\2\2\u00a5\u00a3\3\2\2\2\u00a5\u00a4\3\2\2\2\u00a6\25\3\2\2\2\u00a7"+ + "\u00a8\7\3\2\2\u00a8\u00a9\5\4\3\2\u00a9\u00aa\7\4\2\2\u00aa\27\3\2\2"+ + "\2\u00ab\u00ad\5\32\16\2\u00ac\u00ae\5\34\17\2\u00ad\u00ac\3\2\2\2\u00ad"+ + "\u00ae\3\2\2\2\u00ae\u00b0\3\2\2\2\u00af\u00b1\5\"\22\2\u00b0\u00af\3"+ + "\2\2\2\u00b0\u00b1\3\2\2\2\u00b1\31\3\2\2\2\u00b2\u00b8\5D#\2\u00b3\u00b5"+ + "\7\5\2\2\u00b4\u00b6\5,\27\2\u00b5\u00b4\3\2\2\2\u00b5\u00b6\3\2\2\2\u00b6"+ + "\u00b7\3\2\2\2\u00b7\u00b9\7\6\2\2\u00b8\u00b3\3\2\2\2\u00b8\u00b9\3\2"+ + "\2\2\u00b9\u00bf\3\2\2\2\u00ba\u00bb\7\5\2\2\u00bb\u00bc\5,\27\2\u00bc"+ + "\u00bd\7\6\2\2\u00bd\u00bf\3\2\2\2\u00be\u00b2\3\2\2\2\u00be\u00ba\3\2"+ + "\2\2\u00bf\33\3\2\2\2\u00c0\u00c1\7\7\2\2\u00c1\u00c2\7,\2\2\u00c2\u00c3"+ + "\7\b\2\2\u00c3\35\3\2\2\2\u00c4\u00c5\7&\2\2\u00c5\u00c6\7,\2\2\u00c6"+ + "\37\3\2\2\2\u00c7\u00c8\7\34\2\2\u00c8\u00c9\t\6\2\2\u00c9!\3\2\2\2\u00ca"+ + "\u00d3\5\36\20\2\u00cb\u00d3\5 \21\2\u00cc\u00cd\5\36\20\2\u00cd\u00ce"+ + "\5 \21\2\u00ce\u00d3\3\2\2\2\u00cf\u00d0\5 \21\2\u00d0\u00d1\5\36\20\2"+ + "\u00d1\u00d3\3\2\2\2\u00d2\u00ca\3\2\2\2\u00d2\u00cb\3\2\2\2\u00d2\u00cc"+ + "\3\2\2\2\u00d2\u00cf\3\2\2\2\u00d3#\3\2\2\2\u00d4\u00d5\7\'\2\2\u00d5"+ + "\u00d6\7\13\2\2\u00d6%\3\2\2\2\u00d7\u00d8\7\7\2\2\u00d8\u00d9\7,\2\2"+ + "\u00d9\u00db\7\t\2\2\u00da\u00dc\7,\2\2\u00db\u00da\3\2\2\2\u00db\u00dc"+ + "\3\2\2\2\u00dc\u00dd\3\2\2\2\u00dd\u00de\7\b\2\2\u00de\'\3\2\2\2\u00df"+ + "\u00e0\5H%\2\u00e0\u00e1\5*\26\2\u00e1\u00e2\7\f\2\2\u00e2)\3\2\2\2\u00e3"+ + "\u00e4\t\7\2\2\u00e4+\3\2\2\2\u00e5\u00ea\5(\25\2\u00e6\u00e7\7\n\2\2"+ + "\u00e7\u00e9\5(\25\2\u00e8\u00e6\3\2\2\2\u00e9\u00ec\3\2\2\2\u00ea\u00e8"+ + "\3\2\2\2\u00ea\u00eb\3\2\2\2\u00eb-\3\2\2\2\u00ec\u00ea\3\2\2\2\u00ed"+ + "\u00ee\7-\2\2\u00ee\u00ef\5\62\32\2\u00ef/\3\2\2\2\u00f0\u00f3\5N(\2\u00f1"+ + "\u00f3\5\4\3\2\u00f2\u00f0\3\2\2\2\u00f2\u00f1\3\2\2\2\u00f3\61\3\2\2"+ + "\2\u00f4\u00fd\7\3\2\2\u00f5\u00fa\5\60\31\2\u00f6\u00f7\7\n\2\2\u00f7"+ + "\u00f9\5\60\31\2\u00f8\u00f6\3\2\2\2\u00f9\u00fc\3\2\2\2\u00fa\u00f8\3"+ + "\2\2\2\u00fa\u00fb\3\2\2\2\u00fb\u00fe\3\2\2\2\u00fc\u00fa\3\2\2\2\u00fd"+ + "\u00f5\3\2\2\2\u00fd\u00fe\3\2\2\2\u00fe\u00ff\3\2\2\2\u00ff\u0100\7\4"+ + "\2\2\u0100\63\3\2\2\2\u0101\u0102\7+\2\2\u0102\u0111\5\62\32\2\u0103\u0106"+ + "\7+\2\2\u0104\u0107\5\66\34\2\u0105\u0107\58\35\2\u0106\u0104\3\2\2\2"+ + "\u0106\u0105\3\2\2\2\u0107\u0108\3\2\2\2\u0108\u0109\5\62\32\2\u0109\u0111"+ + "\3\2\2\2\u010a\u010b\7+\2\2\u010b\u010e\5\62\32\2\u010c\u010f\5\66\34"+ + "\2\u010d\u010f\58\35\2\u010e\u010c\3\2\2\2\u010e\u010d\3\2\2\2\u010f\u0111"+ + "\3\2\2\2\u0110\u0101\3\2\2\2\u0110\u0103\3\2\2\2\u0110\u010a\3\2\2\2\u0111"+ + "\65\3\2\2\2\u0112\u0113\7 \2\2\u0113\u0114\5J&\2\u0114\67\3\2\2\2\u0115"+ + "\u0116\7!\2\2\u0116\u0117\5J&\2\u01179\3\2\2\2\u0118\u011b\5<\37\2\u0119"+ + "\u011b\5> \2\u011a\u0118\3\2\2\2\u011a\u0119\3\2\2\2\u011b\u011e\3\2\2"+ + "\2\u011c\u011f\5@!\2\u011d\u011f\5B\"\2\u011e\u011c\3\2\2\2\u011e\u011d"+ + "\3\2\2\2\u011e\u011f\3\2\2\2\u011f;\3\2\2\2\u0120\u0121\7\"\2\2\u0121"+ + "\u0122\5J&\2\u0122=\3\2\2\2\u0123\u0124\7#\2\2\u0124\u0125\5J&\2\u0125"+ + "?\3\2\2\2\u0126\u0128\7$\2\2\u0127\u0129\5J&\2\u0128\u0127\3\2\2\2\u0128"+ + "\u0129\3\2\2\2\u0129A\3\2\2\2\u012a\u012c\7%\2\2\u012b\u012d\5J&\2\u012c"+ + "\u012b\3\2\2\2\u012c\u012d\3\2\2\2\u012dC\3\2\2\2\u012e\u0132\5F$\2\u012f"+ + "\u0132\7-\2\2\u0130\u0132\7.\2\2\u0131\u012e\3\2\2\2\u0131\u012f\3\2\2"+ + "\2\u0131\u0130\3\2\2\2\u0132E\3\2\2\2\u0133\u0134\t\b\2\2\u0134G\3\2\2"+ + "\2\u0135\u0138\5L\'\2\u0136\u0138\7-\2\2\u0137\u0135\3\2\2\2\u0137\u0136"+ + "\3\2\2\2\u0138I\3\2\2\2\u0139\u0142\7\3\2\2\u013a\u013f\5H%\2\u013b\u013c"+ + "\7\n\2\2\u013c\u013e\5H%\2\u013d\u013b\3\2\2\2\u013e\u0141\3\2\2\2\u013f"+ + "\u013d\3\2\2\2\u013f\u0140\3\2\2\2\u0140\u0143\3\2\2\2\u0141\u013f\3\2"+ + "\2\2\u0142\u013a\3\2\2\2\u0142\u0143\3\2\2\2\u0143\u0145\3\2\2\2\u0144"+ + "\u0146\7\n\2\2\u0145\u0144\3\2\2\2\u0145\u0146\3\2\2\2\u0146\u0147\3\2"+ + "\2\2\u0147\u0148\7\4\2\2\u0148K\3\2\2\2\u0149\u014a\t\t\2\2\u014aM\3\2"+ + "\2\2\u014b\u014c\t\n\2\2\u014cO\3\2\2\2%X]dkry\u0080\u0087\u008b\u008d"+ + "\u009a\u00a5\u00ad\u00b0\u00b5\u00b8\u00be\u00d2\u00db\u00ea\u00f2\u00fa"+ + "\u00fd\u0106\u010e\u0110\u011a\u011e\u0128\u012c\u0131\u0137\u013f\u0142"+ + "\u0145"; public static final ATN _ATN = new ATNDeserializer().deserialize(_serializedATN.toCharArray()); static { diff --git a/prometheus/src/main/java/filodb/prometheus/antlr/PromQLVisitor.java b/prometheus/src/main/java/filodb/prometheus/antlr/PromQLVisitor.java index 375a0c5b3d..b4ad46a2f8 100644 --- a/prometheus/src/main/java/filodb/prometheus/antlr/PromQLVisitor.java +++ b/prometheus/src/main/java/filodb/prometheus/antlr/PromQLVisitor.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.ParseTreeVisitor; @@ -129,6 +129,18 @@ public interface PromQLVisitor extends ParseTreeVisitor { * @return the visitor result */ T visitOffset(PromQLParser.OffsetContext ctx); + /** + * Visit a parse tree produced by {@link PromQLParser#atModifier}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitAtModifier(PromQLParser.AtModifierContext ctx); + /** + * Visit a parse tree produced by {@link PromQLParser#modifier}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitModifier(PromQLParser.ModifierContext ctx); /** * Visit a parse tree produced by {@link PromQLParser#limit}. * @param ctx the parse tree diff --git a/prometheus/src/main/scala/filodb/prometheus/ast/Expressions.scala b/prometheus/src/main/scala/filodb/prometheus/ast/Expressions.scala index 1d0e963f98..553b9162b9 100644 --- a/prometheus/src/main/scala/filodb/prometheus/ast/Expressions.scala +++ b/prometheus/src/main/scala/filodb/prometheus/ast/Expressions.scala @@ -3,8 +3,16 @@ package filodb.prometheus.ast import filodb.core.query.RangeParams import filodb.query._ -case class UnaryExpression(operator: Operator, operand: Expression) extends Expression { +case class UnaryExpression(operator: Operator, operand: Expression) extends Expression with PeriodicSeries { //TODO Need to pass an operator to a series + override def toSeriesPlan(timeParams: TimeRangeParams): PeriodicSeriesPlan = { + if (operator != Add && operator != Sub) { + throw new IllegalArgumentException(s"operator=$operator is not allowed in expression=$operand") + } + // use binary expression to implement the unary operators. + // eg. -foo is implemented through (0 - foo). + BinaryExpression(Scalar(0), operator, None, operand).toSeriesPlan(timeParams) + } } case class PrecedenceExpression(expression: Expression) extends Expression diff --git a/prometheus/src/main/scala/filodb/prometheus/ast/Vectors.scala b/prometheus/src/main/scala/filodb/prometheus/ast/Vectors.scala index 2359d1165d..57149baa85 100644 --- a/prometheus/src/main/scala/filodb/prometheus/ast/Vectors.scala +++ b/prometheus/src/main/scala/filodb/prometheus/ast/Vectors.scala @@ -27,6 +27,29 @@ case class Ignoring(labels: Seq[String]) extends JoinMatching case class On(labels: Seq[String]) extends JoinMatching +sealed trait AtTimestamp { + def getTimestamp(timeParams: TimeRangeParams): Long +} + +case class AtUnix(time: Long) extends AtTimestamp { + override def getTimestamp(timeParams: TimeRangeParams): Long = { + time + } +} + +case class AtStart() extends AtTimestamp { + override def getTimestamp(timeParams: TimeRangeParams): Long = { + timeParams.start + } +} + +case class AtEnd() extends AtTimestamp { + override def getTimestamp(timeParams: TimeRangeParams): Long = { + timeParams.end + } +} + +case class Modifier(offset: Option[Duration], at: Option[AtTimestamp]) sealed trait JoinGrouping { def labels: Seq[String] } @@ -89,7 +112,8 @@ case class VectorMatch(matching: Option[JoinMatching], } case class SubqueryExpression( - subquery: PeriodicSeries, sqcl: SubqueryClause, offset: Option[Duration], limit: Option[Int] + subquery: PeriodicSeries, sqcl: SubqueryClause, offset: Option[Duration], + atTimestamp: Option[AtTimestamp], limit: Option[Int] ) extends Expression with PeriodicSeries { def toSeriesPlan(timeParams: TimeRangeParams): PeriodicSeriesPlan = { @@ -278,7 +302,8 @@ case class VectorSpec() extends Vector{ */ case class InstantExpression(metricName: Option[String], labelSelection: Seq[LabelMatch], - offset: Option[Duration]) extends Vector with PeriodicSeries { + offset: Option[Duration], + atTimestamp: Option[AtTimestamp]) extends Vector with PeriodicSeries { import WindowConstants._ @@ -360,7 +385,8 @@ case class InstantExpression(metricName: Option[String], case class RangeExpression(metricName: Option[String], labelSelection: Seq[LabelMatch], window: Duration, - offset: Option[Duration]) extends Vector with SimpleSeries { + offset: Option[Duration], + atTimestamp: Option[AtTimestamp]) extends Vector with SimpleSeries { private[prometheus] val (columnFilters, column, bucketOpt) = labelMatchesToFilters(mergeNameToLabels) diff --git a/prometheus/src/main/scala/filodb/prometheus/parse/AntlrParser.scala b/prometheus/src/main/scala/filodb/prometheus/parse/AntlrParser.scala index 2bdc664428..0bbf9eb148 100644 --- a/prometheus/src/main/scala/filodb/prometheus/parse/AntlrParser.scala +++ b/prometheus/src/main/scala/filodb/prometheus/parse/AntlrParser.scala @@ -91,13 +91,14 @@ class AntlrParser extends PromQLBaseVisitor[Object] { throw new IllegalArgumentException("Subquery can only be applied to instant queries") } val sqcl = build[SubqueryClause](ctx.subquery()) - val offset: Option[Duration] = if (ctx.offset == null) { - None + val modifier = if (ctx.modifier == null) { + Modifier(None, None) } else { - Some(build[Duration](ctx.offset)) + build[Modifier](ctx.modifier) } val limit: Option[Scalar] = None - SubqueryExpression(lhs.asInstanceOf[PeriodicSeries], sqcl, offset, limit.map(_.toScalar.toInt)) + SubqueryExpression(lhs.asInstanceOf[PeriodicSeries], sqcl, + modifier.offset, modifier.at, limit.map(_.toScalar.toInt)) } override def visitSubquery(ctx: PromQLParser.SubqueryContext): SubqueryClause = { @@ -191,19 +192,17 @@ class AntlrParser extends PromQLBaseVisitor[Object] { build[Seq[LabelMatch]](matcherList) } - val offset: Option[Duration] = if (ctx.offset == null) { - None + val modifier = if (ctx.modifier == null) { + Modifier(None, None) } else { - Some(build[Duration](ctx.offset)) + build[Modifier](ctx.modifier) } - val limit: Option[Scalar] = None - if (ctx.window == null) { - InstantExpression(metricName, labelSelection, offset) + InstantExpression(metricName, labelSelection, modifier.offset, modifier.at) } else { val window = build[Duration](ctx.window) - RangeExpression(metricName, labelSelection, window, offset) + RangeExpression(metricName, labelSelection, window, modifier.offset, modifier.at) } } @@ -215,6 +214,32 @@ class AntlrParser extends PromQLBaseVisitor[Object] { parseDuration(ctx.DURATION) } + + override def visitModifier(ctx: PromQLParser.ModifierContext): Modifier = { + val offset: Option[Duration] = if (ctx.offset == null) { + None + } else { + Some(build[Duration](ctx.offset)) + } + val at: Option[AtTimestamp] = if (ctx.atModifier() == null) { + None + } else { + Some(build[AtTimestamp](ctx.atModifier())) + } + Modifier(offset, at) + } + + override def visitAtModifier(ctx: PromQLParser.AtModifierContext): AtTimestamp = { + ctx.getParent.getParent + if (ctx.NUMBER() != null) { + AtUnix(ctx.NUMBER().getText.toLong) + } else if (ctx.START() != null) { + AtStart() + } else { // ctx.END() != null + AtEnd() + } + } + override def visitLimitOperation(ctx: PromQLParser.LimitOperationContext) = { val name = ctx.limit().LIMIT().getSymbol().getText() val limit = Scalar(java.lang.Double.parseDouble(ctx.limit().NUMBER().getSymbol().getText())) diff --git a/prometheus/src/main/scala/filodb/prometheus/parse/LegacyParser.scala b/prometheus/src/main/scala/filodb/prometheus/parse/LegacyParser.scala index 87c33ca5a6..5cdcad8e38 100644 --- a/prometheus/src/main/scala/filodb/prometheus/parse/LegacyParser.scala +++ b/prometheus/src/main/scala/filodb/prometheus/parse/LegacyParser.scala @@ -264,26 +264,26 @@ trait SelectorParser extends OperatorParser with UnitParser with BaseParser { lazy val instantVectorSelector: PackratParser[InstantExpression] = metricNameIdentifier ~ labelSelection.? ~ offset.? ^^ { case metricName ~ ls ~ opt => - InstantExpression(Some(metricName.str), ls.getOrElse(Seq.empty), opt.map(_.duration)) + InstantExpression(Some(metricName.str), ls.getOrElse(Seq.empty), opt.map(_.duration), None) } lazy val instantVectorSelector2: PackratParser[InstantExpression] = labelSelection ~ offset.? ^^ { case ls ~ opt => - InstantExpression(None, ls, opt.map(_.duration)) + InstantExpression(None, ls, opt.map(_.duration), None) } lazy val rangeVectorSelector: PackratParser[RangeExpression] = metricNameIdentifier ~ labelSelection.? ~ simpleLookback ~ offset.? ^^ { case metricName ~ ls ~ simpleLookback ~ opt => RangeExpression(Some(metricName.str), ls.getOrElse(Seq.empty), simpleLookback.duration, - opt.map(_.duration)) + opt.map(_.duration), None) } lazy val rangeVectorSelector2: PackratParser[RangeExpression] = labelSelection ~ simpleLookback ~ offset.? ^^ { case ls ~ simpleLookback ~ opt => - RangeExpression(None, ls, simpleLookback.duration, opt.map(_.duration)) + RangeExpression(None, ls, simpleLookback.duration, opt.map(_.duration), None) } lazy val vector: PackratParser[Vector] = @@ -414,7 +414,7 @@ trait ExpressionParser extends AggregatesParser with SelectorParser with Numeric lazy val subqueryExpression: PackratParser[SubqueryExpression] = subqueryableExpression ~ subqueryClause ^^ { - case sqe ~ sqc => SubqueryExpression( sqe, sqc, None, None) //we do not support offset in the legacy parser + case sqe ~ sqc => SubqueryExpression( sqe, sqc, None, None, None) //we do not support offset in the legacy parser } } diff --git a/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala b/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala index f4e2e94700..5b8b06f265 100644 --- a/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala +++ b/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala @@ -1,12 +1,12 @@ package filodb.prometheus.parse -import filodb.prometheus.ast.{PeriodicSeries, RangeExpression, SimpleSeries, SubqueryClause, SubqueryExpression, TimeStepParams, VectorSpec} +import filodb.prometheus.ast.{AtEnd, AtStart, AtTimestamp, AtUnix, PeriodicSeries, RangeExpression, SimpleSeries, SubqueryClause, SubqueryExpression, TimeStepParams, VectorSpec} import filodb.prometheus.parse.Parser.{Antlr, Shadow} import filodb.query.{BinaryJoin, LogicalPlan} import org.scalatest.funspec.AnyFunSpec import org.scalatest.matchers.should.Matchers -import java.util.concurrent.TimeUnit.{MINUTES, SECONDS, HOURS, DAYS} +import java.util.concurrent.TimeUnit.{DAYS, HOURS, MINUTES, SECONDS} import scala.concurrent.duration.Duration //noinspection ScalaStyle @@ -235,6 +235,22 @@ class ParserSpec extends AnyFunSpec with Matchers { parseSuccessfully("foo[5m25s] OFFSET 1h30m") parseSuccessfully("foo[1h5m25s] OFFSET 1h30m20s") + parseSuccessfully("foo @1000") + parseSuccessfully("foo @start()") + parseSuccessfully("foo @end()") + + parseSuccessfully("foo[5m] @1000") + parseSuccessfully("foo[5m25s] @start()") + parseSuccessfully("foo[1h5m25s] @end()") + + parseSuccessfully("foo[5m] @1000 OFFSET 1h30m20s") + parseSuccessfully("foo[5m25s] @start() OFFSET 1h30m") + parseSuccessfully("foo[1h5m25s] @end() OFFSET 1h30m") + + parseSuccessfully("rate(foo[5m] @1000 OFFSET 1h30m20s)") + parseSuccessfully("rate(foo[5m25s] @start() OFFSET 1h30m)") + parseSuccessfully("rate(foo[1h5m25s] @end() OFFSET 1h30m)") + parseError("foo[5mm]") parseError("foo[0m]") parseError("foo[1i5m]") @@ -254,6 +270,10 @@ class ParserSpec extends AnyFunSpec with Matchers { parseError("some_metric LIMIT 1m[5m]") parseError("(foo + bar)[5m]") +// parseError("rate(foo[5m]) @1000") +// parseError("rate(foo[5m25s]) @start()") +// parseError("rate(foo[1h5m25s]) @end()") + parseSuccessfully("sum by (foo)(some_metric)") parseSuccessfully("avg by (foo)(some_metric)") parseSuccessfully("max by (foo)(some_metric)") @@ -501,6 +521,13 @@ class ParserSpec extends AnyFunSpec with Matchers { parseSubquery("max_over_time((time() - max(foo) < 1000)[5m:10s] offset 5m) limit 2") parseSubquery("avg_over_time(rate(demo_cpu_usage_seconds_total[1m])[2m:10s])") + parseSubquery("max_over_time((time() - max(foo) < 1000)[5m:10s] @1000)") + parseSubquery("max_over_time((time() - max(foo) < 1000)[5m:10s] @1000) limit 2") + parseSubquery("max_over_time((time() - max(foo) < 1000)[5m:10s] offset 5m @1000)") + parseSubquery("max_over_time((time() - max(foo) < 1000)[5m:10s] offset 5m @1000) limit 2") + parseSubquery("max_over_time((time() - max(foo) < 1000)[5m:10s] @1000 offset 5m)") + parseSubquery("max_over_time((time() - max(foo) < 1000)[5m:10s] @1000 offset 5m) limit 2") + parseSubquery("foo[5m:1m]") parseSubquery("foo[5m:]") parseSubquery("max_over_time(rate(foo[5m])[5m:1m])") @@ -555,6 +582,30 @@ class ParserSpec extends AnyFunSpec with Matchers { ) } + it("parse expressions with unary operators") { + parseWithAntlr("-1", """ScalarBinaryOperation(SUB,Left(0.0),Left(1.0),RangeParams(1524855988,1000,1524855988))""") + parseWithAntlr("-foo", """ScalarVectorBinaryOperation(SUB,ScalarFixedDoublePlan(0.0,RangeParams(1524855988,1000,1524855988)),PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(foo))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),true)""") + parseWithAntlr("foo * -1","""BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(foo))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),MUL,OneToOne,ScalarBinaryOperation(SUB,Left(0.0),Left(1.0),RangeParams(1524855988,1000,1524855988)),List(),List(),List())""") + parseWithAntlr("-1 * foo", """BinaryJoin(ScalarBinaryOperation(SUB,Left(0.0),Left(1.0),RangeParams(1524855988,1000,1524855988)),MUL,OneToOne,PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(foo))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),List(),List(),List())""") + parseWithAntlr("-1 * -foo", """BinaryJoin(ScalarBinaryOperation(SUB,Left(0.0),Left(1.0),RangeParams(1524855988,1000,1524855988)),MUL,OneToOne,ScalarVectorBinaryOperation(SUB,ScalarFixedDoublePlan(0.0,RangeParams(1524855988,1000,1524855988)),PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(foo))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),true),List(),List(),List())""") + parseWithAntlr("sum(foo) < -1", """BinaryJoin(Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(foo))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),List(),None),LSS,OneToOne,ScalarBinaryOperation(SUB,Left(0.0),Left(1.0),RangeParams(1524855988,1000,1524855988)),List(),List(),List())""") + parseWithAntlr("-sum(foo) < -1", "BinaryJoin(ScalarVectorBinaryOperation(SUB,ScalarFixedDoublePlan(0.0,RangeParams(1524855988,1000,1524855988)),Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(foo))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),List(),None),true),LSS,OneToOne,ScalarBinaryOperation(SUB,Left(0.0),Left(1.0),RangeParams(1524855988,1000,1524855988)),List(),List(),List())") + parseWithAntlr("sum(-foo) < -1", """BinaryJoin(Aggregate(Sum,ScalarVectorBinaryOperation(SUB,ScalarFixedDoublePlan(0.0,RangeParams(1524855988,1000,1524855988)),PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(foo))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),true),List(),None),LSS,OneToOne,ScalarBinaryOperation(SUB,Left(0.0),Left(1.0),RangeParams(1524855988,1000,1524855988)),List(),List(),List())""") + parseWithAntlr("-sum(-foo) < -1", """BinaryJoin(ScalarVectorBinaryOperation(SUB,ScalarFixedDoublePlan(0.0,RangeParams(1524855988,1000,1524855988)),Aggregate(Sum,ScalarVectorBinaryOperation(SUB,ScalarFixedDoublePlan(0.0,RangeParams(1524855988,1000,1524855988)),PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(foo))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),true),List(),None),true),LSS,OneToOne,ScalarBinaryOperation(SUB,Left(0.0),Left(1.0),RangeParams(1524855988,1000,1524855988)),List(),List(),List())""") + + parseWithAntlr("+1", """ScalarBinaryOperation(ADD,Left(0.0),Left(1.0),RangeParams(1524855988,1000,1524855988))""") + parseWithAntlr("+foo", """ScalarVectorBinaryOperation(ADD,ScalarFixedDoublePlan(0.0,RangeParams(1524855988,1000,1524855988)),PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(foo))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),true)""") + parseWithAntlr("foo * +1", """BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(foo))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),MUL,OneToOne,ScalarBinaryOperation(ADD,Left(0.0),Left(1.0),RangeParams(1524855988,1000,1524855988)),List(),List(),List())""") + parseWithAntlr("+1 * foo", """BinaryJoin(ScalarBinaryOperation(ADD,Left(0.0),Left(1.0),RangeParams(1524855988,1000,1524855988)),MUL,OneToOne,PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(foo))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),List(),List(),List())""") + parseWithAntlr("+1 * +foo", """BinaryJoin(ScalarBinaryOperation(ADD,Left(0.0),Left(1.0),RangeParams(1524855988,1000,1524855988)),MUL,OneToOne,ScalarVectorBinaryOperation(ADD,ScalarFixedDoublePlan(0.0,RangeParams(1524855988,1000,1524855988)),PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(foo))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),true),List(),List(),List())""") + parseWithAntlr("sum(foo) < +1", """BinaryJoin(Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(foo))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),List(),None),LSS,OneToOne,ScalarBinaryOperation(ADD,Left(0.0),Left(1.0),RangeParams(1524855988,1000,1524855988)),List(),List(),List())""") + parseWithAntlr("+sum(foo) < +1", "BinaryJoin(ScalarVectorBinaryOperation(ADD,ScalarFixedDoublePlan(0.0,RangeParams(1524855988,1000,1524855988)),Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(foo))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),List(),None),true),LSS,OneToOne,ScalarBinaryOperation(ADD,Left(0.0),Left(1.0),RangeParams(1524855988,1000,1524855988)),List(),List(),List())") + parseWithAntlr("sum(+foo) < +1", """BinaryJoin(Aggregate(Sum,ScalarVectorBinaryOperation(ADD,ScalarFixedDoublePlan(0.0,RangeParams(1524855988,1000,1524855988)),PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(foo))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),true),List(),None),LSS,OneToOne,ScalarBinaryOperation(ADD,Left(0.0),Left(1.0),RangeParams(1524855988,1000,1524855988)),List(),List(),List())""") + parseWithAntlr("+sum(+foo) < +1", """BinaryJoin(ScalarVectorBinaryOperation(ADD,ScalarFixedDoublePlan(0.0,RangeParams(1524855988,1000,1524855988)),Aggregate(Sum,ScalarVectorBinaryOperation(ADD,ScalarFixedDoublePlan(0.0,RangeParams(1524855988,1000,1524855988)),PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(foo))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),true),List(),None),true),LSS,OneToOne,ScalarBinaryOperation(ADD,Left(0.0),Left(1.0),RangeParams(1524855988,1000,1524855988)),List(),List(),List())""") + + parseWithAntlr("+-1", """ScalarVectorBinaryOperation(ADD,ScalarFixedDoublePlan(0.0,RangeParams(1524855988,1000,1524855988)),ScalarBinaryOperation(SUB,Left(0.0),Left(1.0),RangeParams(1524855988,1000,1524855988)),true)""") + } + it("Should be able to make logical plans for Series Expressions") { val queryToLpString = Map( "http_requests_total + time()" -> "ScalarVectorBinaryOperation(ADD,ScalarTimeBasedPlan(Time,RangeParams(1524855988,1000,1524855988)),PeriodicSeries(RawSeries(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List(),Some(300000),None),1524855988000,1000000,1524855988000,None),false)", @@ -805,7 +856,8 @@ class ParserSpec extends AnyFunSpec with Matchers { } it("should correctly parse durations with multiple units") { - case class Spec(query: String, windowDuration: Duration, offsetDuration: Duration) + case class Spec(query: String, windowDuration: Duration, offsetDuration: Duration, + atTimestamp: Option[AtTimestamp] = None) val specs = Seq( Spec( """foo{label="bar"}[1m30s] offset 2h15m""", @@ -813,27 +865,43 @@ class ParserSpec extends AnyFunSpec with Matchers { Duration(2, HOURS) + Duration(15, MINUTES) ), Spec( - """foo{label="bar"}[3d2h25m10s] offset 2d12h15m30s""", + """foo{label="bar"}[3d2h25m10s] @600 offset 2d12h15m30s""", Duration(3, DAYS) + Duration(2, HOURS) + Duration(25, MINUTES) + Duration(10, SECONDS), Duration(2, DAYS) + Duration(12, HOURS) + Duration(15, MINUTES) + Duration(30, SECONDS), + Some(AtUnix(600)) ), Spec( - """foo{label="bar"}[3d0h25m0s] offset 0d12h15m30s""", + """foo{label="bar"}[3d0h25m0s] offset 0d12h15m30s @1000""", Duration(3, DAYS) + Duration(25, MINUTES), Duration(12, HOURS) + Duration(15, MINUTES) + Duration(30, SECONDS), + Some(AtUnix(1000)) + ), + Spec( + """foo{label="bar"}[3d0h25m0s] offset 0d12h15m30s @start()""", + Duration(3, DAYS) + Duration(25, MINUTES), + Duration(12, HOURS) + Duration(15, MINUTES) + Duration(30, SECONDS), + Some(AtStart()) + ), + Spec( + """foo{label="bar"}[3d0h25m0s] offset 0d12h15m30s @end()""", + Duration(3, DAYS) + Duration(25, MINUTES), + Duration(12, HOURS) + Duration(15, MINUTES) + Duration(30, SECONDS), + Some(AtEnd()) ) ) specs.foreach{ spec => Parser.parseQuery(spec.query) match { - case RangeExpression(_, _, window, offset) => + case RangeExpression(_, _, window, offset, atTimestamp) => window.millis(0) shouldEqual spec.windowDuration.toMillis offset.get.millis(0) shouldEqual spec.offsetDuration.toMillis + atTimestamp shouldEqual spec.atTimestamp } } } it("should correctly parse subquery durations with multiple units") { - case class Spec(query: String, windowDuration: Duration, stepDuration: Duration, offsetDuration: Duration) + case class Spec(query: String, windowDuration: Duration, stepDuration: Duration, + offsetDuration: Duration, atTimestamp: Option[AtTimestamp] = None) val specs = Seq( Spec( """foo{label="bar"}[3d2h25m10s:1d4h30m4s] offset 2d12h15m30s""", @@ -842,18 +910,20 @@ class ParserSpec extends AnyFunSpec with Matchers { Duration(2, DAYS) + Duration(12, HOURS) + Duration(15, MINUTES) + Duration(30, SECONDS) ), Spec( - """foo{label="bar"}[3d0h25m0s:1d0h2m] offset 0d12h15m30s""", + """foo{label="bar"}[3d0h25m0s:1d0h2m] offset 0d12h15m30s @1000""", Duration(3, DAYS) + Duration(25, MINUTES), Duration(1, DAYS) + Duration(2, MINUTES), - Duration(12, HOURS) + Duration(15, MINUTES) + Duration(30, SECONDS) + Duration(12, HOURS) + Duration(15, MINUTES) + Duration(30, SECONDS), + Some(AtUnix(1000)) ) ) specs.foreach{ spec => Parser.parseQuery(spec.query) match { - case SubqueryExpression(_, SubqueryClause(window, step), offset, _) => + case SubqueryExpression(_, SubqueryClause(window, step), offset, atTimestamp, _) => window.millis(0) shouldEqual spec.windowDuration.toMillis step.map(_.millis(0)).getOrElse(0) shouldEqual spec.stepDuration.toMillis offset.get.millis(0) shouldEqual spec.offsetDuration.toMillis + atTimestamp shouldEqual spec.atTimestamp } } } diff --git a/query/src/main/scala/filodb/query/exec/ExecPlan.scala b/query/src/main/scala/filodb/query/exec/ExecPlan.scala index d1b889a6f6..5abf255e16 100644 --- a/query/src/main/scala/filodb/query/exec/ExecPlan.scala +++ b/query/src/main/scala/filodb/query/exec/ExecPlan.scala @@ -235,34 +235,34 @@ trait ExecPlan extends QueryCommand { @volatile var numResultSamples = 0 // BEWARE - do not modify concurrently!! @volatile var resultSize = 0L val queryResults = rv.doOnStart(_ => Task.eval(span.mark("before-first-materialized-result-rv"))) - .map { - case srvable: SerializableRangeVector => srvable - case rv: RangeVector => - // materialize, and limit rows per RV - val execPlanString = queryWithPlanName(queryContext) - val builder = SerializedRangeVector.newBuilder(maxRecordContainerSize(querySession.queryConfig)) - val srv = SerializedRangeVector(rv, builder, recordSchema, execPlanString, querySession.queryStats) - if (rv.outputRange.isEmpty) - qLogger.debug(s"Empty rangevector found. Rv class is: ${rv.getClass.getSimpleName}, " + - s"execPlan is: $execPlanString, execPlan children ${this.children}") + .bufferTumbling(querySession.queryConfig.numRvsPerResultMessage) + .map { f => + val builder = SerializedRangeVector.newBuilder(maxRecordContainerSize(querySession.queryConfig)) + val srvs = f.map { + case srvable: SerializableRangeVector => srvable + case rv: RangeVector => + val execPlanString = queryWithPlanName(queryContext) + SerializedRangeVector(rv, builder, recordSchema, execPlanString, querySession.queryStats) + } + .map { srv => + // fail the query instead of limiting range vectors and returning incomplete/inaccurate results + numResultSamples += srv.numRowsSerialized + checkSamplesLimit(numResultSamples, querySession.warnings) + val srvBytes = srv.estimatedSerializedBytes + resultSize += srvBytes + querySession.queryStats.getResultBytesCounter(Nil).addAndGet(srvBytes) + checkResultBytes(resultSize, querySession.queryConfig, querySession.warnings) srv + } + .filter(_.numRowsSerialized > 0) + StreamQueryResult(queryContext.queryId, planId, srvs) } - .map { srv => - // fail the query instead of limiting range vectors and returning incomplete/inaccurate results - numResultSamples += srv.numRowsSerialized - checkSamplesLimit(numResultSamples, querySession.warnings) - resultSize += srv.estimatedSerializedBytes - checkResultBytes(resultSize, querySession.queryConfig, querySession.warnings) - srv - } - .filter(_.numRowsSerialized > 0) - .bufferTumbling(querySession.queryConfig.numRvsPerResultMessage) - .map(StreamQueryResult(queryContext.queryId, planId, _)) .guarantee(Task.eval { Kamon.histogram("query-execute-time-elapsed-step2-result-materialized", MeasurementUnit.time.milliseconds) .withTag("plan", getClass.getSimpleName) .record(Math.max(0, System.currentTimeMillis - startExecute)) + SerializedRangeVector.queryResultBytes.record(resultSize) // recording and adding step1 to queryStats at the end of execution since the grouping // for stats is not formed yet at the beginning querySession.queryStats.getCpuNanosCounter(Nil).getAndAdd(step1CpuTime) @@ -439,7 +439,6 @@ trait ExecPlan extends QueryCommand { } } - def makeResult( rv : Observable[RangeVector], recordSchema: RecordSchema, resultSchema: ResultSchema ): Task[QueryResult] = { @@ -462,7 +461,9 @@ trait ExecPlan extends QueryCommand { // fail the query instead of limiting range vectors and returning incomplete/inaccurate results numResultSamples += srv.numRowsSerialized checkSamplesLimit(numResultSamples, querySession.warnings) - resultSize += srv.estimatedSerializedBytes + val srvBytes = srv.estimatedSerializedBytes + resultSize += srvBytes + querySession.queryStats.getResultBytesCounter(Nil).addAndGet(srvBytes) checkResultBytes(resultSize, querySession.queryConfig, querySession.warnings) srv } @@ -470,6 +471,7 @@ trait ExecPlan extends QueryCommand { .guarantee(Task.eval(span.mark("after-last-materialized-result-rv"))) .toListL .map { r => + SerializedRangeVector.queryResultBytes.record(resultSize) Kamon.histogram("query-execute-time-elapsed-step2-result-materialized", MeasurementUnit.time.milliseconds) .withTag("plan", getClass.getSimpleName) @@ -530,7 +532,9 @@ trait ExecPlan extends QueryCommand { } protected def queryWithPlanName(queryContext: QueryContext): String = { - s"${this.getClass.getSimpleName}-${queryContext.origQueryParams}" + // Disabling this since it showed up in local method profiles. Re-enable if needed for debugging + // s"${this.getClass.getSimpleName}-${queryContext.origQueryParams}" + s"${queryContext.queryId}:$planId" } def curNodeText(level: Int): String = diff --git a/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala b/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala index f6a3cb264d..9412813801 100644 --- a/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala +++ b/query/src/main/scala/filodb/query/exec/PromQlRemoteExec.scala @@ -153,7 +153,7 @@ case class PromQlRemoteExec(queryEndpoint: String, } // dont add this size to queryStats since it was already added by callee use dummy QueryStats() SerializedRangeVector(rv, builder, recordSchema.get("default").get, - queryWithPlanName(queryContext), dummyQueryStats) + planId, dummyQueryStats) // TODO: Handle stitching with verbose flag } QueryResult( @@ -235,7 +235,7 @@ case class PromQlRemoteExec(queryEndpoint: String, } // dont add this size to queryStats since it was already added by callee use dummy QueryStats() SerializedRangeVector(rv, builder, recordSchema.get(Avg.entryName).get, - queryWithPlanName(queryContext), dummyQueryStats) + planId, dummyQueryStats) } // TODO: Handle stitching with verbose flag @@ -273,7 +273,7 @@ case class PromQlRemoteExec(queryEndpoint: String, } // dont add this size to queryStats since it was already added by callee use dummy QueryStats() SerializedRangeVector(rv, builder, recordSchema.get(QueryFunctionConstants.stdVal).get, - queryWithPlanName(queryContext), dummyQueryStats) + planId, dummyQueryStats) } // TODO: Handle stitching with verbose flag diff --git a/query/src/main/scala/filodb/query/exec/aggregator/CountValuesRowAggregator.scala b/query/src/main/scala/filodb/query/exec/aggregator/CountValuesRowAggregator.scala index eed6afdd51..22b84ffbed 100644 --- a/query/src/main/scala/filodb/query/exec/aggregator/CountValuesRowAggregator.scala +++ b/query/src/main/scala/filodb/query/exec/aggregator/CountValuesRowAggregator.scala @@ -2,6 +2,7 @@ package filodb.query.exec.aggregator import scala.collection.mutable +import filodb.core.Utils import filodb.core.binaryrecord2.RecordBuilder import filodb.core.memstore.FiloSchedulers import filodb.core.memstore.FiloSchedulers.QuerySchedName @@ -83,36 +84,42 @@ class CountValuesRowAggregator(label: String, limit: Int = 1000) extends RowAggr def present(aggRangeVector: RangeVector, limit: Int, rangeParams: RangeParams, queryStats: QueryStats): Seq[RangeVector] = { - val colSchema = Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), - ColumnInfo("value", ColumnType.DoubleColumn)) - val recSchema = SerializedRangeVector.toSchema(colSchema) - val resRvs = mutable.Map[RangeVectorKey, RecordBuilder]() + val startNs = Utils.currentThreadCpuTimeNanos try { - FiloSchedulers.assertThreadName(QuerySchedName) - // aggRangeVector.rows.take below triggers the ChunkInfoIterator which requires lock/release - ChunkMap.validateNoSharedLocks(s"CountValues-$label") - aggRangeVector.rows.take(limit).foreach { row => - val rowMap = CountValuesSerDeser.deserialize(row.getBlobBase(1), - row.getBlobNumBytes(1), row.getBlobOffset(1)) - rowMap.foreach { (k, v) => - val rvk = CustomRangeVectorKey(aggRangeVector.key.labelValues + - (label.utf8 -> k.toString.utf8)) - val builder = resRvs.getOrElseUpdate(rvk, SerializedRangeVector.newBuilder()) - builder.startNewRecord(recSchema) - builder.addLong(row.getLong(0)) - builder.addDouble(v) - builder.endRecord() + val colSchema = Seq(ColumnInfo("timestamp", ColumnType.TimestampColumn), + ColumnInfo("value", ColumnType.DoubleColumn)) + val recSchema = SerializedRangeVector.toSchema(colSchema) + val resRvs = mutable.Map[RangeVectorKey, RecordBuilder]() + try { + FiloSchedulers.assertThreadName(QuerySchedName) + // aggRangeVector.rows.take below triggers the ChunkInfoIterator which requires lock/release + ChunkMap.validateNoSharedLocks(s"CountValues-$label") + aggRangeVector.rows.take(limit).foreach { row => + val rowMap = CountValuesSerDeser.deserialize(row.getBlobBase(1), + row.getBlobNumBytes(1), row.getBlobOffset(1)) + rowMap.foreach { (k, v) => + val rvk = CustomRangeVectorKey(aggRangeVector.key.labelValues + + (label.utf8 -> k.toString.utf8)) + val builder = resRvs.getOrElseUpdate(rvk, SerializedRangeVector.newBuilder()) + builder.startNewRecord(recSchema) + builder.addLong(row.getLong(0)) + builder.addDouble(v) + builder.endRecord() + } } + } finally { + aggRangeVector.rows.close() + ChunkMap.releaseAllSharedLocks() } + resRvs.map { case (key, builder) => + val numRows = builder.allContainers.map(_.countRecords()).sum + val srv = new SerializedRangeVector(key, numRows, builder.allContainers, recSchema, 0, None) + queryStats.getResultBytesCounter(Nil).getAndAdd(srv.estimatedSerializedBytes) + srv + }.toSeq + } finally { + queryStats.getCpuNanosCounter(Nil).getAndAdd(Utils.currentThreadCpuTimeNanos - startNs) } - finally { - aggRangeVector.rows.close() - ChunkMap.releaseAllSharedLocks() - } - resRvs.map { case (key, builder) => - val numRows = builder.allContainers.map(_.countRecords()).sum - new SerializedRangeVector(key, numRows, builder.allContainers, recSchema, 0, None) - }.toSeq } def reductionSchema(source: ResultSchema): ResultSchema = { diff --git a/query/src/main/scala/filodb/query/exec/aggregator/SumRowAggregator.scala b/query/src/main/scala/filodb/query/exec/aggregator/SumRowAggregator.scala index 638d01e949..a482b69664 100644 --- a/query/src/main/scala/filodb/query/exec/aggregator/SumRowAggregator.scala +++ b/query/src/main/scala/filodb/query/exec/aggregator/SumRowAggregator.scala @@ -21,8 +21,8 @@ object SumRowAggregator extends RowAggregator { def map(rvk: RangeVectorKey, item: RowReader, mapInto: MutableRowReader): RowReader = item def reduceAggregate(acc: SumHolder, aggRes: RowReader): SumHolder = { acc.timestamp = aggRes.getLong(0) - if (!aggRes.getDouble(1).isNaN) { - if (acc.sum.isNaN) acc.sum = 0 + if (!java.lang.Double.isNaN(aggRes.getDouble(1))) { + if (java.lang.Double.isNaN(acc.sum)) acc.sum = 0 acc.sum += aggRes.getDouble(1) } acc diff --git a/query/src/main/scala/filodb/query/exec/aggregator/TopBottomKRowAggregator.scala b/query/src/main/scala/filodb/query/exec/aggregator/TopBottomKRowAggregator.scala index de4479e8d1..726692fafc 100644 --- a/query/src/main/scala/filodb/query/exec/aggregator/TopBottomKRowAggregator.scala +++ b/query/src/main/scala/filodb/query/exec/aggregator/TopBottomKRowAggregator.scala @@ -158,12 +158,10 @@ class TopBottomKRowAggregator(k: Int, bottomK: Boolean) extends RowAggregator wi resRvs.map { case (key, builder) => val numRows = builder.allContainers.map(_.countRecords()).sum logger.debug(s"TopkPresent before creating SRV key = ${key.labelValues.mkString(",")}") - val srv = new SerializedRangeVector(key, numRows, builder.allContainers, recSchema, 0, + new SerializedRangeVector(key, numRows, builder.allContainers, recSchema, 0, Some(RvRange(rangeParams.startSecs * 1000, rangeParams.stepSecs * 1000, rangeParams.endSecs * 1000))) - queryStats.getResultBytesCounter(Nil).getAndAdd(srv.estimatedSerializedBytes) - srv }.toSeq } finally { queryStats.getCpuNanosCounter(Nil).getAndAdd(Utils.currentThreadCpuTimeNanos - startNs) diff --git a/query/src/test/scala/filodb/query/exec/InProcessPlanDispatcherSpec.scala b/query/src/test/scala/filodb/query/exec/InProcessPlanDispatcherSpec.scala index ca6cceb872..607e94d8c3 100644 --- a/query/src/test/scala/filodb/query/exec/InProcessPlanDispatcherSpec.scala +++ b/query/src/test/scala/filodb/query/exec/InProcessPlanDispatcherSpec.scala @@ -36,13 +36,13 @@ class InProcessPlanDispatcherSpec extends AnyFunSpec import filodb.core.{MachineMetricsData => MMD} 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.setup(MMD.dataset1.ref, Schemas(MMD.schema1), 0, TestData.storeConf) + memStore.setup(MMD.dataset1.ref, Schemas(MMD.schema1), 0, TestData.storeConf, 1) memStore.ingest(MMD.dataset1.ref, 0, mmdSomeData) - memStore.setup(MMD.histDataset.ref, Schemas(MMD.histDataset.schema), 0, TestData.storeConf) + memStore.setup(MMD.histDataset.ref, Schemas(MMD.histDataset.schema), 0, TestData.storeConf, 1) memStore.ingest(MMD.histDataset.ref, 0, MMD.records(MMD.histDataset, histData)) - memStore.setup(MMD.histMaxDS.ref, Schemas(MMD.histMaxDS.schema), 0, TestData.storeConf) + memStore.setup(MMD.histMaxDS.ref, Schemas(MMD.histMaxDS.schema), 0, TestData.storeConf, 1) memStore.ingest(MMD.histMaxDS.ref, 0, MMD.records(MMD.histMaxDS, histMaxData)) memStore.refreshIndexForTesting(timeseriesDataset.ref) memStore.refreshIndexForTesting(MMD.dataset1.ref) diff --git a/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala b/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala index 1f19b21fd8..84da458bb7 100644 --- a/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala @@ -86,7 +86,7 @@ class MetadataExecSpec extends AnyFunSpec with Matchers with ScalaFutures with B tuples.map { t => SeqRowReader(Seq(t._1, t._2, metric, partTagsUTF8)) } .foreach(builder.addFromReader(_, Schemas.promCounter)) } - memStore.setup(timeseriesDatasetMultipleShardKeys.ref, Schemas(Schemas.promCounter), ishard, TestData.storeConf) + memStore.setup(timeseriesDatasetMultipleShardKeys.ref, Schemas(Schemas.promCounter), ishard, TestData.storeConf, 1) memStore.ingest(timeseriesDatasetMultipleShardKeys.ref, ishard, SomeData(builder.allContainers.head, 0)) builder.reset() diff --git a/query/src/test/scala/filodb/query/exec/MultiSchemaPartitionsExecSpec.scala b/query/src/test/scala/filodb/query/exec/MultiSchemaPartitionsExecSpec.scala index 7e9ba0d567..092c309583 100644 --- a/query/src/test/scala/filodb/query/exec/MultiSchemaPartitionsExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/MultiSchemaPartitionsExecSpec.scala @@ -88,16 +88,16 @@ class MultiSchemaPartitionsExecSpec extends AnyFunSpec with Matchers with ScalaF implicit val execTimeout = 5.seconds override def beforeAll(): Unit = { - memStore.setup(dsRef, schemas, 0, TestData.storeConf) + memStore.setup(dsRef, schemas, 0, TestData.storeConf, 2) memStore.ingest(dsRef, 0, SomeData(container, 0)) memStore.ingest(dsRef, 0, MMD.records(MMD.histDataset, histData)) // set up shard, but do not ingest data to simulate an empty shard - memStore.setup(dsRef, schemas, 1, TestData.storeConf) + memStore.setup(dsRef, schemas, 1, TestData.storeConf, 2) - memStore.setup(MMD.dataset1.ref, Schemas(MMD.schema1), 0, TestData.storeConf) + memStore.setup(MMD.dataset1.ref, Schemas(MMD.schema1), 0, TestData.storeConf, 1) memStore.ingest(MMD.dataset1.ref, 0, mmdSomeData) - memStore.setup(MMD.histMaxDS.ref, Schemas(MMD.histMaxDS.schema), 0, TestData.storeConf) + memStore.setup(MMD.histMaxDS.ref, Schemas(MMD.histMaxDS.schema), 0, TestData.storeConf, 1) memStore.ingest(MMD.histMaxDS.ref, 0, MMD.records(MMD.histMaxDS, histMaxData)) memStore.refreshIndexForTesting(dsRef) @@ -138,6 +138,7 @@ class MultiSchemaPartitionsExecSpec extends AnyFunSpec with Matchers with ScalaF val execPlan = MultiSchemaPartitionsExec(QueryContext(), dummyDispatcher, dsRef, 0, filters, TimeRangeChunkScan(startTime, endTime), "_metric_") + querySession.queryStats.clear() // so this can be run as a standalone test val resp = execPlan.execute(memStore, querySession).runToFuture.futureValue val result = resp.asInstanceOf[QueryResult] result.result.size shouldEqual 1 @@ -145,6 +146,10 @@ class MultiSchemaPartitionsExecSpec extends AnyFunSpec with Matchers with ScalaF dataRead shouldEqual tuples.take(11) val partKeyRead = result.result(0).key.labelValues.map(lv => (lv._1.asNewString, lv._2.asNewString)) partKeyRead shouldEqual partKeyKVWithMetric + querySession.queryStats.getResultBytesCounter().get() shouldEqual 297 + querySession.queryStats.getCpuNanosCounter().get() > 0 shouldEqual true + querySession.queryStats.getDataBytesScannedCounter().get() shouldEqual 48 + querySession.queryStats.getTimeSeriesScannedCounter().get() shouldEqual 1 } it("should get empty schema if query returns no results") { diff --git a/query/src/test/scala/filodb/query/exec/PromQLGrpcRemoteExecSpec.scala b/query/src/test/scala/filodb/query/exec/PromQLGrpcRemoteExecSpec.scala index 217cae5fd0..5ecf7fea2b 100644 --- a/query/src/test/scala/filodb/query/exec/PromQLGrpcRemoteExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/PromQLGrpcRemoteExecSpec.scala @@ -157,7 +157,7 @@ class PromQLGrpcRemoteExecSpec extends AnyFunSpec with Matchers with ScalaFuture deserializedSrv.numRowsSerialized shouldEqual 4 val res = deserializedSrv.rows.map(r => (r.getLong(0), r.getDouble(1))).toList deserializedSrv.key shouldEqual rvKey - qr.queryStats.getResultBytesCounter(List()).get()shouldEqual 108 + // queryStats ResultBytes counter increment is not done as part of SRV constructor, so skipping that assertion (qr.queryStats.getCpuNanosCounter(List()).get() > 0) shouldEqual true res.length shouldEqual 11 res.map(_._1) shouldEqual (0 to 1000 by 100) diff --git a/query/src/test/scala/filodb/query/exec/RemoteMetadataExecSpec.scala b/query/src/test/scala/filodb/query/exec/RemoteMetadataExecSpec.scala index 18d92cdda1..a13bbb5291 100644 --- a/query/src/test/scala/filodb/query/exec/RemoteMetadataExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/RemoteMetadataExecSpec.scala @@ -85,7 +85,7 @@ class RemoteMetadataExecSpec extends AnyFunSpec with Matchers with ScalaFutures tuples.map { t => SeqRowReader(Seq(t._1, t._2, metric, partTagsUTF8)) } .foreach(builder.addFromReader(_, Schemas.promCounter)) } - memStore.setup(timeseriesDatasetMultipleShardKeys.ref, Schemas(Schemas.promCounter), ishard, TestData.storeConf) + memStore.setup(timeseriesDatasetMultipleShardKeys.ref, Schemas(Schemas.promCounter), ishard, TestData.storeConf, 1) memStore.ingest(timeseriesDatasetMultipleShardKeys.ref, ishard, SomeData(builder.allContainers.head, 0)) } diff --git a/query/src/test/scala/filodb/query/exec/SplitLocalPartitionDistConcatExecSpec.scala b/query/src/test/scala/filodb/query/exec/SplitLocalPartitionDistConcatExecSpec.scala index 652bbc43ed..aab1bc1697 100644 --- a/query/src/test/scala/filodb/query/exec/SplitLocalPartitionDistConcatExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/SplitLocalPartitionDistConcatExecSpec.scala @@ -86,10 +86,10 @@ class SplitLocalPartitionDistConcatExecSpec extends AnyFunSpec with Matchers wit implicit val execTimeout = 5.seconds override def beforeAll(): Unit = { - memStore.setup(dsRef, schemas, 0, TestData.storeConf) + memStore.setup(dsRef, schemas, 0, TestData.storeConf, 1) memStore.ingest(dsRef, 0, SomeData(container, 0)) - memStore.setup(MMD.dataset1.ref, Schemas(MMD.schema1), 0, TestData.storeConf) + memStore.setup(MMD.dataset1.ref, Schemas(MMD.schema1), 0, TestData.storeConf, 1) memStore.ingest(MMD.dataset1.ref, 0, mmdSomeData) memStore.refreshIndexForTesting(dsRef) diff --git a/run_benchmarks.sh b/run_benchmarks.sh index d368e26d59..0572870994 100755 --- a/run_benchmarks.sh +++ b/run_benchmarks.sh @@ -1,7 +1,6 @@ #!/bin/bash -sbt "jmh/jmh:run -rf json -i 15 -wi 10 -f3 -jvmArgsAppend -XX:MaxInlineLevel=20 \ +sbt "jmh/jmh:run -rf json -i 5 -wi 3 -f 1 -jvmArgsAppend -XX:MaxInlineLevel=20 \ -jvmArgsAppend -Xmx4g -jvmArgsAppend -XX:MaxInlineSize=99 \ - -prof jfr:dir=/tmp/filo-jmh \ filodb.jmh.QueryHiCardInMemoryBenchmark \ filodb.jmh.QueryInMemoryBenchmark \ filodb.jmh.QueryAndIngestBenchmark \ diff --git a/spark-jobs/src/test/scala/filodb/downsampler/DownsamplerMainSpec.scala b/spark-jobs/src/test/scala/filodb/downsampler/DownsamplerMainSpec.scala index d66d0eec41..6fa2640d82 100644 --- a/spark-jobs/src/test/scala/filodb/downsampler/DownsamplerMainSpec.scala +++ b/spark-jobs/src/test/scala/filodb/downsampler/DownsamplerMainSpec.scala @@ -1412,7 +1412,7 @@ class DownsamplerMainSpec extends AnyFunSpec with Matchers with BeforeAndAfterAl settings.filodbConfig) downsampleTSStore.setup(batchDownsampler.rawDatasetRef, settings.filodbSettings.schemas, - 0, rawDataStoreConfig, settings.rawDatasetIngestionConfig.downsampleConfig) + 0, rawDataStoreConfig, 1, settings.rawDatasetIngestionConfig.downsampleConfig) downsampleTSStore.recoverIndex(batchDownsampler.rawDatasetRef, 0).futureValue @@ -1459,7 +1459,7 @@ class DownsamplerMainSpec extends AnyFunSpec with Matchers with BeforeAndAfterAl ) downsampleTSStore.setup(batchDownsampler.rawDatasetRef, settings.filodbSettings.schemas, - 0, rawDataStoreConfig, durableIndexSettings.rawDatasetIngestionConfig.downsampleConfig) + 0, rawDataStoreConfig, 1, durableIndexSettings.rawDatasetIngestionConfig.downsampleConfig) val recoveredRecords = downsampleTSStore.recoverIndex(batchDownsampler.rawDatasetRef, 0).futureValue recoveredRecords shouldBe 5 @@ -1474,7 +1474,7 @@ class DownsamplerMainSpec extends AnyFunSpec with Matchers with BeforeAndAfterAl ) downsampleTSStore2.setup(batchDownsampler.rawDatasetRef, settings.filodbSettings.schemas, - 0, rawDataStoreConfig, durableIndexSettings.rawDatasetIngestionConfig.downsampleConfig) + 0, rawDataStoreConfig, 1, durableIndexSettings.rawDatasetIngestionConfig.downsampleConfig) val recoveredRecords2 = downsampleTSStore2.recoverIndex(batchDownsampler.rawDatasetRef, 0).futureValue recoveredRecords2 shouldBe 0 @@ -1509,7 +1509,7 @@ class DownsamplerMainSpec extends AnyFunSpec with Matchers with BeforeAndAfterAl settings.filodbConfig) downsampleTSStore.setup(batchDownsampler.rawDatasetRef, settings.filodbSettings.schemas, - 0, rawDataStoreConfig, settings.rawDatasetIngestionConfig.downsampleConfig) + 0, rawDataStoreConfig, 1, settings.rawDatasetIngestionConfig.downsampleConfig) downsampleTSStore.recoverIndex(batchDownsampler.rawDatasetRef, 0).futureValue @@ -1540,7 +1540,7 @@ class DownsamplerMainSpec extends AnyFunSpec with Matchers with BeforeAndAfterAl settings.filodbConfig) downsampleTSStore.setup(batchDownsampler.rawDatasetRef, settings.filodbSettings.schemas, - 0, rawDataStoreConfig, settings.rawDatasetIngestionConfig.downsampleConfig) + 0, rawDataStoreConfig, 1, settings.rawDatasetIngestionConfig.downsampleConfig) downsampleTSStore.recoverIndex(batchDownsampler.rawDatasetRef, 0).futureValue @@ -1572,7 +1572,7 @@ class DownsamplerMainSpec extends AnyFunSpec with Matchers with BeforeAndAfterAl settings.filodbConfig) downsampleTSStore.setup(batchDownsampler.rawDatasetRef, settings.filodbSettings.schemas, - 0, rawDataStoreConfig, settings.rawDatasetIngestionConfig.downsampleConfig) + 0, rawDataStoreConfig, 1, settings.rawDatasetIngestionConfig.downsampleConfig) downsampleTSStore.recoverIndex(batchDownsampler.rawDatasetRef, 0).futureValue @@ -1602,7 +1602,7 @@ class DownsamplerMainSpec extends AnyFunSpec with Matchers with BeforeAndAfterAl val downsampleTSStore = new DownsampledTimeSeriesStore(downsampleColStore, rawColStore, settings.filodbConfig) downsampleTSStore.setup(batchDownsampler.rawDatasetRef, settings.filodbSettings.schemas, - 0, rawDataStoreConfig, settings.rawDatasetIngestionConfig.downsampleConfig) + 0, rawDataStoreConfig, 1, settings.rawDatasetIngestionConfig.downsampleConfig) downsampleTSStore.recoverIndex(batchDownsampler.rawDatasetRef, 0).futureValue val colFilters = seriesTags.map { case (t, v) => ColumnFilter(t.toString, Equals(v.toString)) }.toSeq val queryFilters = colFilters :+ ColumnFilter("_metric_", Equals(gaugeName))