diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneCountOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneCountOperator.java
new file mode 100644
index 0000000000000..6d02a975cf74f
--- /dev/null
+++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneCountOperator.java
@@ -0,0 +1,148 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0; you may not use this file except in compliance with the Elastic License
+ * 2.0.
+ */
+
+package org.elasticsearch.compute.lucene;
+
+import org.apache.lucene.search.LeafCollector;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.Scorable;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.Weight;
+import org.elasticsearch.compute.data.DocVector;
+import org.elasticsearch.compute.data.IntBlock;
+import org.elasticsearch.compute.data.IntVector;
+import org.elasticsearch.compute.data.Page;
+import org.elasticsearch.compute.operator.DriverContext;
+import org.elasticsearch.compute.operator.SourceOperator;
+import org.elasticsearch.search.internal.SearchContext;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.function.Function;
+
+/**
+ * Source operator that incrementally counts the results in Lucene searches
+ */
+public class LuceneCountOperator extends LuceneOperator {
+
+ private int totalHits = 0;
+ private int remainingDocs;
+
+ private IntVector.Builder docsBuilder;
+ private final LeafCollector leafCollector;
+
+ public static class Factory implements LuceneOperator.Factory {
+ private final DataPartitioning dataPartitioning;
+ private final int taskConcurrency;
+ private final int limit;
+ private final LuceneSliceQueue sliceQueue;
+
+ public Factory(
+ List searchContexts,
+ Function queryFunction,
+ DataPartitioning dataPartitioning,
+ int taskConcurrency,
+ int limit
+ ) {
+ this.limit = limit;
+ this.dataPartitioning = dataPartitioning;
+ var weightFunction = weightFunction(queryFunction, ScoreMode.COMPLETE_NO_SCORES);
+ this.sliceQueue = LuceneSliceQueue.create(searchContexts, weightFunction, dataPartitioning, taskConcurrency);
+ this.taskConcurrency = Math.min(sliceQueue.totalSlices(), taskConcurrency);
+ }
+
+ @Override
+ public SourceOperator get(DriverContext driverContext) {
+ return new LuceneCountOperator(sliceQueue, limit);
+ }
+
+ @Override
+ public int taskConcurrency() {
+ return taskConcurrency;
+ }
+
+ public int limit() {
+ return limit;
+ }
+
+ @Override
+ public String describe() {
+ return "LuceneCountOperator[dataPartitioning = "
+ + dataPartitioning
+ + ", limit = "
+ + limit
+ + "]";
+ }
+ }
+
+ public LuceneCountOperator(LuceneSliceQueue sliceQueue, int limit) {
+ super(Integer.MAX_VALUE, sliceQueue);
+ this.remainingDocs = limit;
+ this.leafCollector = new LeafCollector() {
+ @Override
+ public void setScorer(Scorable scorer) {}
+
+ @Override
+ public void collect(int doc) {
+ if (remainingDocs > 0) {
+ --remainingDocs;
+ totalHits++;
+ }
+ }
+ };
+ }
+
+ @Override
+ public boolean isFinished() {
+ return doneCollecting;
+ }
+
+ @Override
+ public void finish() {
+ doneCollecting = true;
+ }
+
+ @Override
+ public Page getOutput() {
+ if (isFinished()) {
+ assert remainingDocs == 0 : remainingDocs;
+ return null;
+ }
+ try {
+ final LuceneScorer scorer = getCurrentOrLoadNextScorer();
+ if (scorer == null) {
+ return null;
+ }
+ Weight weight = scorer.weight;
+ var leafReaderContext = scorer.leafReaderContext();
+ // see org.apache.lucene.search.TotalHitCountCollector
+ int leafCount = weight == null ? -1 : weight.count(leafReaderContext);
+ if (leafCount != -1) {
+ // check to not go over limit
+ var count = Math.min(leafCount, remainingDocs);
+ totalHits += count;
+ remainingDocs -= count;
+ } else {
+ scorer.scoreNextRange(
+ leafCollector,
+ leafReaderContext.reader().getLiveDocs(),
+ remainingDocs
+ );
+ }
+
+ Page page = null;
+ if (remainingDocs <= 0 || scorer.isDone()) {
+ pagesEmitted++;
+ page = new Page(1, IntBlock.newConstantBlockWith(totalHits, 1));
+ }
+ return page;
+ } catch (IOException e) {
+ throw new UncheckedIOException(e);
+ }
+ }
+}
diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneOperator.java
index e0b25469d5ab2..7513f476c8966 100644
--- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneOperator.java
+++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneOperator.java
@@ -57,7 +57,6 @@ public interface Factory extends SourceOperator.SourceOperatorFactory {
@Override
public void close() {
-
}
LuceneScorer getCurrentOrLoadNextScorer() {
@@ -97,7 +96,7 @@ LuceneScorer getCurrentOrLoadNextScorer() {
static final class LuceneScorer {
private final int shardIndex;
private final SearchContext searchContext;
- private final Weight weight;
+ final Weight weight;
private final LeafReaderContext leafReaderContext;
private BulkScorer bulkScorer;
diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneCountOperatorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneCountOperatorTests.java
new file mode 100644
index 0000000000000..e51e617168538
--- /dev/null
+++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneCountOperatorTests.java
@@ -0,0 +1,160 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0; you may not use this file except in compliance with the Elastic License
+ * 2.0.
+ */
+
+package org.elasticsearch.compute.lucene;
+
+import org.apache.lucene.document.SortedNumericDocValuesField;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.index.NoMergePolicy;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.tests.index.RandomIndexWriter;
+import org.elasticsearch.common.util.BigArrays;
+import org.elasticsearch.compute.data.LongBlock;
+import org.elasticsearch.compute.data.Page;
+import org.elasticsearch.compute.operator.AnyOperatorTestCase;
+import org.elasticsearch.compute.operator.Driver;
+import org.elasticsearch.compute.operator.DriverContext;
+import org.elasticsearch.compute.operator.OperatorTestCase;
+import org.elasticsearch.compute.operator.PageConsumerOperator;
+import org.elasticsearch.core.IOUtils;
+import org.elasticsearch.index.cache.query.TrivialQueryCachingPolicy;
+import org.elasticsearch.index.query.SearchExecutionContext;
+import org.elasticsearch.search.internal.ContextIndexSearcher;
+import org.elasticsearch.search.internal.SearchContext;
+import org.junit.After;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.function.Function;
+
+import static org.hamcrest.Matchers.both;
+import static org.hamcrest.Matchers.greaterThanOrEqualTo;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.lessThan;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class LuceneCountOperatorTests extends AnyOperatorTestCase {
+ private Directory directory = newDirectory();
+ private IndexReader reader;
+
+ @After
+ public void closeIndex() throws IOException {
+ IOUtils.close(reader, directory);
+ }
+
+ @Override
+ protected LuceneCountOperator.Factory simple(BigArrays bigArrays) {
+ return simple(bigArrays, randomFrom(DataPartitioning.values()), between(1, 10_000), 100);
+ }
+
+ private LuceneCountOperator.Factory simple(BigArrays bigArrays, DataPartitioning dataPartitioning, int numDocs, int limit) {
+ int commitEvery = Math.max(1, numDocs / 10);
+ try (
+ RandomIndexWriter writer = new RandomIndexWriter(
+ random(),
+ directory,
+ newIndexWriterConfig().setMergePolicy(NoMergePolicy.INSTANCE)
+ )
+ ) {
+ for (int d = 0; d < numDocs; d++) {
+ List doc = new ArrayList<>();
+ doc.add(new SortedNumericDocValuesField("s", d));
+ writer.addDocument(doc);
+ if (d % commitEvery == 0) {
+ writer.commit();
+ }
+ }
+ reader = writer.getReader();
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+
+ SearchContext ctx = mockSearchContext(reader);
+ SearchExecutionContext ectx = mock(SearchExecutionContext.class);
+ when(ctx.getSearchExecutionContext()).thenReturn(ectx);
+ when(ectx.getIndexReader()).thenReturn(reader);
+ Function queryFunction = c -> new MatchAllDocsQuery();
+ return new LuceneCountOperator.Factory(List.of(ctx), queryFunction, dataPartitioning, 1, limit);
+ }
+
+ @Override
+ protected String expectedToStringOfSimple() {
+ assumeFalse("can't support variable maxPageSize", true); // TODO allow testing this
+ return "LuceneCountOperator[shardId=0, maxPageSize=**random**]";
+ }
+
+ @Override
+ protected String expectedDescriptionOfSimple() {
+ assumeFalse("can't support variable maxPageSize", true); // TODO allow testing this
+ return """
+ LuceneCountOperator[dataPartitioning = SHARD, maxPageSize = **random**, limit = 100, sorts = [{"s":{"order":"asc"}}]]""";
+ }
+
+ // TODO tests for the other data partitioning configurations
+
+ public void testShardDataPartitioning() {
+ int size = between(1_000, 20_000);
+ int limit = between(10, size);
+ testSimple(size, limit);
+ }
+
+ public void testEmpty() {
+ testSimple(0, between(10, 10_000));
+ }
+
+ private void testSimple(int size, int limit) {
+ DriverContext ctx = new DriverContext();
+ LuceneCountOperator.Factory factory = simple(nonBreakingBigArrays(), DataPartitioning.SHARD, size, limit);
+
+ List results = new ArrayList<>();
+ OperatorTestCase.runDriver(
+ new Driver(ctx, factory.get(ctx), List.of(), new PageConsumerOperator(page -> results.add(page)), () -> {})
+ );
+ OperatorTestCase.assertDriverContext(ctx);
+
+ for (Page page : results) {
+ assertThat(page.getPositionCount(), is(1));
+ }
+
+ for (Page page : results) {
+ LongBlock sBlock = page.getBlock(1);
+ for (int p = 0; p < page.getPositionCount(); p++) {
+ assertThat(sBlock.getLong(sBlock.getFirstValueIndex(p)), both(greaterThanOrEqualTo(0L)).and(lessThan((long) size)));
+ }
+ }
+ assertThat(results, hasSize(1));
+ }
+
+ /**
+ * Creates a mock search context with the given index reader.
+ * The returned mock search context can be used to test with {@link LuceneOperator}.
+ */
+ public static SearchContext mockSearchContext(IndexReader reader) {
+ try {
+ ContextIndexSearcher searcher = new ContextIndexSearcher(
+ reader,
+ IndexSearcher.getDefaultSimilarity(),
+ IndexSearcher.getDefaultQueryCache(),
+ TrivialQueryCachingPolicy.NEVER,
+ true
+ );
+ SearchContext searchContext = mock(SearchContext.class);
+ when(searchContext.searcher()).thenReturn(searcher);
+ return searchContext;
+ } catch (IOException e) {
+ throw new UncheckedIOException(e);
+ }
+ }
+}
diff --git a/x-pack/plugin/esql/src/main/antlr/EsqlBaseParser.g4 b/x-pack/plugin/esql/src/main/antlr/EsqlBaseParser.g4
index 8f07a8a5dcdea..044e920744375 100644
--- a/x-pack/plugin/esql/src/main/antlr/EsqlBaseParser.g4
+++ b/x-pack/plugin/esql/src/main/antlr/EsqlBaseParser.g4
@@ -76,8 +76,12 @@ operatorExpression
primaryExpression
: constant #constantDefault
| qualifiedName #dereference
+ | functionExpression #function
| LP booleanExpression RP #parenthesizedExpression
- | identifier LP (booleanExpression (COMMA booleanExpression)*)? RP #functionExpression
+ ;
+
+functionExpression
+ : identifier LP (ASTERISK | (booleanExpression (COMMA booleanExpression)*))? RP
;
rowCommand
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/EsqlFunctionRegistry.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/EsqlFunctionRegistry.java
index d4029fd8ac4f0..d3a0ac0431cf4 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/EsqlFunctionRegistry.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/EsqlFunctionRegistry.java
@@ -187,6 +187,10 @@ private FunctionDefinition[][] functions() {
@Override
protected String normalize(String name) {
+ return normalizeName(name);
+ }
+
+ public static String normalizeName(String name) {
return name.toLowerCase(Locale.ROOT);
}
}
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParser.interp b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParser.interp
index 349f31f7c476d..658e09ca4b190 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParser.interp
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParser.interp
@@ -177,6 +177,7 @@ regexBooleanExpression
valueExpression
operatorExpression
primaryExpression
+functionExpression
rowCommand
fields
field
@@ -216,4 +217,4 @@ enrichWithClause
atn:
-[4, 1, 81, 501, 2, 0, 7, 0, 2, 1, 7, 1, 2, 2, 7, 2, 2, 3, 7, 3, 2, 4, 7, 4, 2, 5, 7, 5, 2, 6, 7, 6, 2, 7, 7, 7, 2, 8, 7, 8, 2, 9, 7, 9, 2, 10, 7, 10, 2, 11, 7, 11, 2, 12, 7, 12, 2, 13, 7, 13, 2, 14, 7, 14, 2, 15, 7, 15, 2, 16, 7, 16, 2, 17, 7, 17, 2, 18, 7, 18, 2, 19, 7, 19, 2, 20, 7, 20, 2, 21, 7, 21, 2, 22, 7, 22, 2, 23, 7, 23, 2, 24, 7, 24, 2, 25, 7, 25, 2, 26, 7, 26, 2, 27, 7, 27, 2, 28, 7, 28, 2, 29, 7, 29, 2, 30, 7, 30, 2, 31, 7, 31, 2, 32, 7, 32, 2, 33, 7, 33, 2, 34, 7, 34, 2, 35, 7, 35, 2, 36, 7, 36, 2, 37, 7, 37, 2, 38, 7, 38, 2, 39, 7, 39, 2, 40, 7, 40, 2, 41, 7, 41, 2, 42, 7, 42, 2, 43, 7, 43, 2, 44, 7, 44, 2, 45, 7, 45, 1, 0, 1, 0, 1, 0, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 5, 1, 102, 8, 1, 10, 1, 12, 1, 105, 9, 1, 1, 2, 1, 2, 1, 2, 1, 2, 3, 2, 111, 8, 2, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 3, 3, 126, 8, 3, 1, 4, 1, 4, 1, 4, 1, 5, 1, 5, 1, 5, 1, 5, 1, 5, 1, 5, 1, 5, 3, 5, 138, 8, 5, 1, 5, 1, 5, 1, 5, 1, 5, 1, 5, 5, 5, 145, 8, 5, 10, 5, 12, 5, 148, 9, 5, 1, 5, 1, 5, 1, 5, 1, 5, 1, 5, 3, 5, 155, 8, 5, 1, 5, 1, 5, 3, 5, 159, 8, 5, 1, 5, 1, 5, 1, 5, 1, 5, 1, 5, 1, 5, 5, 5, 167, 8, 5, 10, 5, 12, 5, 170, 9, 5, 1, 6, 1, 6, 3, 6, 174, 8, 6, 1, 6, 1, 6, 1, 6, 1, 6, 1, 6, 3, 6, 181, 8, 6, 1, 6, 1, 6, 1, 6, 3, 6, 186, 8, 6, 1, 7, 1, 7, 1, 7, 1, 7, 1, 7, 3, 7, 193, 8, 7, 1, 8, 1, 8, 1, 8, 1, 8, 3, 8, 199, 8, 8, 1, 8, 1, 8, 1, 8, 1, 8, 1, 8, 1, 8, 5, 8, 207, 8, 8, 10, 8, 12, 8, 210, 9, 8, 1, 9, 1, 9, 1, 9, 1, 9, 1, 9, 1, 9, 1, 9, 1, 9, 1, 9, 1, 9, 1, 9, 5, 9, 223, 8, 9, 10, 9, 12, 9, 226, 9, 9, 3, 9, 228, 8, 9, 1, 9, 1, 9, 3, 9, 232, 8, 9, 1, 10, 1, 10, 1, 10, 1, 11, 1, 11, 1, 11, 5, 11, 240, 8, 11, 10, 11, 12, 11, 243, 9, 11, 1, 12, 1, 12, 1, 12, 1, 12, 1, 12, 3, 12, 250, 8, 12, 1, 13, 1, 13, 1, 13, 1, 13, 5, 13, 256, 8, 13, 10, 13, 12, 13, 259, 9, 13, 1, 13, 3, 13, 262, 8, 13, 1, 14, 1, 14, 1, 14, 1, 14, 1, 14, 5, 14, 269, 8, 14, 10, 14, 12, 14, 272, 9, 14, 1, 14, 1, 14, 1, 15, 1, 15, 1, 15, 1, 16, 1, 16, 3, 16, 281, 8, 16, 1, 16, 1, 16, 3, 16, 285, 8, 16, 1, 17, 1, 17, 1, 17, 1, 17, 3, 17, 291, 8, 17, 1, 18, 1, 18, 1, 18, 5, 18, 296, 8, 18, 10, 18, 12, 18, 299, 9, 18, 1, 19, 1, 19, 1, 20, 1, 20, 1, 20, 5, 20, 306, 8, 20, 10, 20, 12, 20, 309, 9, 20, 1, 21, 1, 21, 1, 22, 1, 22, 1, 22, 1, 22, 1, 22, 1, 22, 1, 22, 1, 22, 1, 22, 1, 22, 1, 22, 1, 22, 1, 22, 5, 22, 326, 8, 22, 10, 22, 12, 22, 329, 9, 22, 1, 22, 1, 22, 1, 22, 1, 22, 1, 22, 1, 22, 5, 22, 337, 8, 22, 10, 22, 12, 22, 340, 9, 22, 1, 22, 1, 22, 1, 22, 1, 22, 1, 22, 1, 22, 5, 22, 348, 8, 22, 10, 22, 12, 22, 351, 9, 22, 1, 22, 1, 22, 3, 22, 355, 8, 22, 1, 23, 1, 23, 1, 23, 1, 24, 1, 24, 1, 24, 1, 24, 5, 24, 364, 8, 24, 10, 24, 12, 24, 367, 9, 24, 1, 25, 1, 25, 3, 25, 371, 8, 25, 1, 25, 1, 25, 3, 25, 375, 8, 25, 1, 26, 1, 26, 1, 26, 1, 26, 5, 26, 381, 8, 26, 10, 26, 12, 26, 384, 9, 26, 1, 26, 1, 26, 1, 26, 1, 26, 5, 26, 390, 8, 26, 10, 26, 12, 26, 393, 9, 26, 3, 26, 395, 8, 26, 1, 27, 1, 27, 1, 27, 1, 27, 5, 27, 401, 8, 27, 10, 27, 12, 27, 404, 9, 27, 1, 28, 1, 28, 1, 28, 1, 28, 5, 28, 410, 8, 28, 10, 28, 12, 28, 413, 9, 28, 1, 29, 1, 29, 1, 29, 1, 29, 1, 30, 1, 30, 1, 30, 1, 30, 3, 30, 423, 8, 30, 1, 31, 1, 31, 1, 31, 1, 31, 1, 32, 1, 32, 1, 32, 1, 33, 1, 33, 1, 33, 5, 33, 435, 8, 33, 10, 33, 12, 33, 438, 9, 33, 1, 34, 1, 34, 1, 34, 1, 34, 1, 35, 1, 35, 1, 36, 1, 36, 3, 36, 448, 8, 36, 1, 37, 3, 37, 451, 8, 37, 1, 37, 1, 37, 1, 38, 3, 38, 456, 8, 38, 1, 38, 1, 38, 1, 39, 1, 39, 1, 40, 1, 40, 1, 41, 1, 41, 1, 41, 1, 42, 1, 42, 1, 42, 1, 42, 1, 43, 1, 43, 1, 43, 1, 43, 3, 43, 475, 8, 43, 1, 44, 1, 44, 1, 44, 1, 44, 3, 44, 481, 8, 44, 1, 44, 1, 44, 1, 44, 1, 44, 5, 44, 487, 8, 44, 10, 44, 12, 44, 490, 9, 44, 3, 44, 492, 8, 44, 1, 45, 1, 45, 1, 45, 3, 45, 497, 8, 45, 1, 45, 1, 45, 1, 45, 0, 3, 2, 10, 16, 46, 0, 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, 80, 82, 84, 86, 88, 90, 0, 8, 1, 0, 60, 61, 1, 0, 62, 64, 1, 0, 76, 77, 1, 0, 67, 68, 2, 0, 32, 32, 35, 35, 1, 0, 38, 39, 2, 0, 37, 37, 51, 51, 1, 0, 54, 59, 531, 0, 92, 1, 0, 0, 0, 2, 95, 1, 0, 0, 0, 4, 110, 1, 0, 0, 0, 6, 125, 1, 0, 0, 0, 8, 127, 1, 0, 0, 0, 10, 158, 1, 0, 0, 0, 12, 185, 1, 0, 0, 0, 14, 192, 1, 0, 0, 0, 16, 198, 1, 0, 0, 0, 18, 231, 1, 0, 0, 0, 20, 233, 1, 0, 0, 0, 22, 236, 1, 0, 0, 0, 24, 249, 1, 0, 0, 0, 26, 251, 1, 0, 0, 0, 28, 263, 1, 0, 0, 0, 30, 275, 1, 0, 0, 0, 32, 278, 1, 0, 0, 0, 34, 286, 1, 0, 0, 0, 36, 292, 1, 0, 0, 0, 38, 300, 1, 0, 0, 0, 40, 302, 1, 0, 0, 0, 42, 310, 1, 0, 0, 0, 44, 354, 1, 0, 0, 0, 46, 356, 1, 0, 0, 0, 48, 359, 1, 0, 0, 0, 50, 368, 1, 0, 0, 0, 52, 394, 1, 0, 0, 0, 54, 396, 1, 0, 0, 0, 56, 405, 1, 0, 0, 0, 58, 414, 1, 0, 0, 0, 60, 418, 1, 0, 0, 0, 62, 424, 1, 0, 0, 0, 64, 428, 1, 0, 0, 0, 66, 431, 1, 0, 0, 0, 68, 439, 1, 0, 0, 0, 70, 443, 1, 0, 0, 0, 72, 447, 1, 0, 0, 0, 74, 450, 1, 0, 0, 0, 76, 455, 1, 0, 0, 0, 78, 459, 1, 0, 0, 0, 80, 461, 1, 0, 0, 0, 82, 463, 1, 0, 0, 0, 84, 466, 1, 0, 0, 0, 86, 474, 1, 0, 0, 0, 88, 476, 1, 0, 0, 0, 90, 496, 1, 0, 0, 0, 92, 93, 3, 2, 1, 0, 93, 94, 5, 0, 0, 1, 94, 1, 1, 0, 0, 0, 95, 96, 6, 1, -1, 0, 96, 97, 3, 4, 2, 0, 97, 103, 1, 0, 0, 0, 98, 99, 10, 1, 0, 0, 99, 100, 5, 26, 0, 0, 100, 102, 3, 6, 3, 0, 101, 98, 1, 0, 0, 0, 102, 105, 1, 0, 0, 0, 103, 101, 1, 0, 0, 0, 103, 104, 1, 0, 0, 0, 104, 3, 1, 0, 0, 0, 105, 103, 1, 0, 0, 0, 106, 111, 3, 82, 41, 0, 107, 111, 3, 26, 13, 0, 108, 111, 3, 20, 10, 0, 109, 111, 3, 86, 43, 0, 110, 106, 1, 0, 0, 0, 110, 107, 1, 0, 0, 0, 110, 108, 1, 0, 0, 0, 110, 109, 1, 0, 0, 0, 111, 5, 1, 0, 0, 0, 112, 126, 3, 30, 15, 0, 113, 126, 3, 34, 17, 0, 114, 126, 3, 46, 23, 0, 115, 126, 3, 52, 26, 0, 116, 126, 3, 48, 24, 0, 117, 126, 3, 32, 16, 0, 118, 126, 3, 8, 4, 0, 119, 126, 3, 54, 27, 0, 120, 126, 3, 56, 28, 0, 121, 126, 3, 60, 30, 0, 122, 126, 3, 62, 31, 0, 123, 126, 3, 88, 44, 0, 124, 126, 3, 64, 32, 0, 125, 112, 1, 0, 0, 0, 125, 113, 1, 0, 0, 0, 125, 114, 1, 0, 0, 0, 125, 115, 1, 0, 0, 0, 125, 116, 1, 0, 0, 0, 125, 117, 1, 0, 0, 0, 125, 118, 1, 0, 0, 0, 125, 119, 1, 0, 0, 0, 125, 120, 1, 0, 0, 0, 125, 121, 1, 0, 0, 0, 125, 122, 1, 0, 0, 0, 125, 123, 1, 0, 0, 0, 125, 124, 1, 0, 0, 0, 126, 7, 1, 0, 0, 0, 127, 128, 5, 18, 0, 0, 128, 129, 3, 10, 5, 0, 129, 9, 1, 0, 0, 0, 130, 131, 6, 5, -1, 0, 131, 132, 5, 44, 0, 0, 132, 159, 3, 10, 5, 7, 133, 159, 3, 14, 7, 0, 134, 159, 3, 12, 6, 0, 135, 137, 3, 14, 7, 0, 136, 138, 5, 44, 0, 0, 137, 136, 1, 0, 0, 0, 137, 138, 1, 0, 0, 0, 138, 139, 1, 0, 0, 0, 139, 140, 5, 41, 0, 0, 140, 141, 5, 40, 0, 0, 141, 146, 3, 14, 7, 0, 142, 143, 5, 34, 0, 0, 143, 145, 3, 14, 7, 0, 144, 142, 1, 0, 0, 0, 145, 148, 1, 0, 0, 0, 146, 144, 1, 0, 0, 0, 146, 147, 1, 0, 0, 0, 147, 149, 1, 0, 0, 0, 148, 146, 1, 0, 0, 0, 149, 150, 5, 50, 0, 0, 150, 159, 1, 0, 0, 0, 151, 152, 3, 14, 7, 0, 152, 154, 5, 42, 0, 0, 153, 155, 5, 44, 0, 0, 154, 153, 1, 0, 0, 0, 154, 155, 1, 0, 0, 0, 155, 156, 1, 0, 0, 0, 156, 157, 5, 45, 0, 0, 157, 159, 1, 0, 0, 0, 158, 130, 1, 0, 0, 0, 158, 133, 1, 0, 0, 0, 158, 134, 1, 0, 0, 0, 158, 135, 1, 0, 0, 0, 158, 151, 1, 0, 0, 0, 159, 168, 1, 0, 0, 0, 160, 161, 10, 4, 0, 0, 161, 162, 5, 31, 0, 0, 162, 167, 3, 10, 5, 5, 163, 164, 10, 3, 0, 0, 164, 165, 5, 47, 0, 0, 165, 167, 3, 10, 5, 4, 166, 160, 1, 0, 0, 0, 166, 163, 1, 0, 0, 0, 167, 170, 1, 0, 0, 0, 168, 166, 1, 0, 0, 0, 168, 169, 1, 0, 0, 0, 169, 11, 1, 0, 0, 0, 170, 168, 1, 0, 0, 0, 171, 173, 3, 14, 7, 0, 172, 174, 5, 44, 0, 0, 173, 172, 1, 0, 0, 0, 173, 174, 1, 0, 0, 0, 174, 175, 1, 0, 0, 0, 175, 176, 5, 43, 0, 0, 176, 177, 3, 78, 39, 0, 177, 186, 1, 0, 0, 0, 178, 180, 3, 14, 7, 0, 179, 181, 5, 44, 0, 0, 180, 179, 1, 0, 0, 0, 180, 181, 1, 0, 0, 0, 181, 182, 1, 0, 0, 0, 182, 183, 5, 49, 0, 0, 183, 184, 3, 78, 39, 0, 184, 186, 1, 0, 0, 0, 185, 171, 1, 0, 0, 0, 185, 178, 1, 0, 0, 0, 186, 13, 1, 0, 0, 0, 187, 193, 3, 16, 8, 0, 188, 189, 3, 16, 8, 0, 189, 190, 3, 80, 40, 0, 190, 191, 3, 16, 8, 0, 191, 193, 1, 0, 0, 0, 192, 187, 1, 0, 0, 0, 192, 188, 1, 0, 0, 0, 193, 15, 1, 0, 0, 0, 194, 195, 6, 8, -1, 0, 195, 199, 3, 18, 9, 0, 196, 197, 7, 0, 0, 0, 197, 199, 3, 16, 8, 3, 198, 194, 1, 0, 0, 0, 198, 196, 1, 0, 0, 0, 199, 208, 1, 0, 0, 0, 200, 201, 10, 2, 0, 0, 201, 202, 7, 1, 0, 0, 202, 207, 3, 16, 8, 3, 203, 204, 10, 1, 0, 0, 204, 205, 7, 0, 0, 0, 205, 207, 3, 16, 8, 2, 206, 200, 1, 0, 0, 0, 206, 203, 1, 0, 0, 0, 207, 210, 1, 0, 0, 0, 208, 206, 1, 0, 0, 0, 208, 209, 1, 0, 0, 0, 209, 17, 1, 0, 0, 0, 210, 208, 1, 0, 0, 0, 211, 232, 3, 44, 22, 0, 212, 232, 3, 40, 20, 0, 213, 214, 5, 40, 0, 0, 214, 215, 3, 10, 5, 0, 215, 216, 5, 50, 0, 0, 216, 232, 1, 0, 0, 0, 217, 218, 3, 42, 21, 0, 218, 227, 5, 40, 0, 0, 219, 224, 3, 10, 5, 0, 220, 221, 5, 34, 0, 0, 221, 223, 3, 10, 5, 0, 222, 220, 1, 0, 0, 0, 223, 226, 1, 0, 0, 0, 224, 222, 1, 0, 0, 0, 224, 225, 1, 0, 0, 0, 225, 228, 1, 0, 0, 0, 226, 224, 1, 0, 0, 0, 227, 219, 1, 0, 0, 0, 227, 228, 1, 0, 0, 0, 228, 229, 1, 0, 0, 0, 229, 230, 5, 50, 0, 0, 230, 232, 1, 0, 0, 0, 231, 211, 1, 0, 0, 0, 231, 212, 1, 0, 0, 0, 231, 213, 1, 0, 0, 0, 231, 217, 1, 0, 0, 0, 232, 19, 1, 0, 0, 0, 233, 234, 5, 14, 0, 0, 234, 235, 3, 22, 11, 0, 235, 21, 1, 0, 0, 0, 236, 241, 3, 24, 12, 0, 237, 238, 5, 34, 0, 0, 238, 240, 3, 24, 12, 0, 239, 237, 1, 0, 0, 0, 240, 243, 1, 0, 0, 0, 241, 239, 1, 0, 0, 0, 241, 242, 1, 0, 0, 0, 242, 23, 1, 0, 0, 0, 243, 241, 1, 0, 0, 0, 244, 250, 3, 10, 5, 0, 245, 246, 3, 40, 20, 0, 246, 247, 5, 33, 0, 0, 247, 248, 3, 10, 5, 0, 248, 250, 1, 0, 0, 0, 249, 244, 1, 0, 0, 0, 249, 245, 1, 0, 0, 0, 250, 25, 1, 0, 0, 0, 251, 252, 5, 6, 0, 0, 252, 257, 3, 38, 19, 0, 253, 254, 5, 34, 0, 0, 254, 256, 3, 38, 19, 0, 255, 253, 1, 0, 0, 0, 256, 259, 1, 0, 0, 0, 257, 255, 1, 0, 0, 0, 257, 258, 1, 0, 0, 0, 258, 261, 1, 0, 0, 0, 259, 257, 1, 0, 0, 0, 260, 262, 3, 28, 14, 0, 261, 260, 1, 0, 0, 0, 261, 262, 1, 0, 0, 0, 262, 27, 1, 0, 0, 0, 263, 264, 5, 65, 0, 0, 264, 265, 5, 73, 0, 0, 265, 270, 3, 38, 19, 0, 266, 267, 5, 34, 0, 0, 267, 269, 3, 38, 19, 0, 268, 266, 1, 0, 0, 0, 269, 272, 1, 0, 0, 0, 270, 268, 1, 0, 0, 0, 270, 271, 1, 0, 0, 0, 271, 273, 1, 0, 0, 0, 272, 270, 1, 0, 0, 0, 273, 274, 5, 66, 0, 0, 274, 29, 1, 0, 0, 0, 275, 276, 5, 4, 0, 0, 276, 277, 3, 22, 11, 0, 277, 31, 1, 0, 0, 0, 278, 280, 5, 17, 0, 0, 279, 281, 3, 22, 11, 0, 280, 279, 1, 0, 0, 0, 280, 281, 1, 0, 0, 0, 281, 284, 1, 0, 0, 0, 282, 283, 5, 30, 0, 0, 283, 285, 3, 36, 18, 0, 284, 282, 1, 0, 0, 0, 284, 285, 1, 0, 0, 0, 285, 33, 1, 0, 0, 0, 286, 287, 5, 8, 0, 0, 287, 290, 3, 22, 11, 0, 288, 289, 5, 30, 0, 0, 289, 291, 3, 36, 18, 0, 290, 288, 1, 0, 0, 0, 290, 291, 1, 0, 0, 0, 291, 35, 1, 0, 0, 0, 292, 297, 3, 40, 20, 0, 293, 294, 5, 34, 0, 0, 294, 296, 3, 40, 20, 0, 295, 293, 1, 0, 0, 0, 296, 299, 1, 0, 0, 0, 297, 295, 1, 0, 0, 0, 297, 298, 1, 0, 0, 0, 298, 37, 1, 0, 0, 0, 299, 297, 1, 0, 0, 0, 300, 301, 7, 2, 0, 0, 301, 39, 1, 0, 0, 0, 302, 307, 3, 42, 21, 0, 303, 304, 5, 36, 0, 0, 304, 306, 3, 42, 21, 0, 305, 303, 1, 0, 0, 0, 306, 309, 1, 0, 0, 0, 307, 305, 1, 0, 0, 0, 307, 308, 1, 0, 0, 0, 308, 41, 1, 0, 0, 0, 309, 307, 1, 0, 0, 0, 310, 311, 7, 3, 0, 0, 311, 43, 1, 0, 0, 0, 312, 355, 5, 45, 0, 0, 313, 314, 3, 76, 38, 0, 314, 315, 5, 67, 0, 0, 315, 355, 1, 0, 0, 0, 316, 355, 3, 74, 37, 0, 317, 355, 3, 76, 38, 0, 318, 355, 3, 70, 35, 0, 319, 355, 5, 48, 0, 0, 320, 355, 3, 78, 39, 0, 321, 322, 5, 65, 0, 0, 322, 327, 3, 72, 36, 0, 323, 324, 5, 34, 0, 0, 324, 326, 3, 72, 36, 0, 325, 323, 1, 0, 0, 0, 326, 329, 1, 0, 0, 0, 327, 325, 1, 0, 0, 0, 327, 328, 1, 0, 0, 0, 328, 330, 1, 0, 0, 0, 329, 327, 1, 0, 0, 0, 330, 331, 5, 66, 0, 0, 331, 355, 1, 0, 0, 0, 332, 333, 5, 65, 0, 0, 333, 338, 3, 70, 35, 0, 334, 335, 5, 34, 0, 0, 335, 337, 3, 70, 35, 0, 336, 334, 1, 0, 0, 0, 337, 340, 1, 0, 0, 0, 338, 336, 1, 0, 0, 0, 338, 339, 1, 0, 0, 0, 339, 341, 1, 0, 0, 0, 340, 338, 1, 0, 0, 0, 341, 342, 5, 66, 0, 0, 342, 355, 1, 0, 0, 0, 343, 344, 5, 65, 0, 0, 344, 349, 3, 78, 39, 0, 345, 346, 5, 34, 0, 0, 346, 348, 3, 78, 39, 0, 347, 345, 1, 0, 0, 0, 348, 351, 1, 0, 0, 0, 349, 347, 1, 0, 0, 0, 349, 350, 1, 0, 0, 0, 350, 352, 1, 0, 0, 0, 351, 349, 1, 0, 0, 0, 352, 353, 5, 66, 0, 0, 353, 355, 1, 0, 0, 0, 354, 312, 1, 0, 0, 0, 354, 313, 1, 0, 0, 0, 354, 316, 1, 0, 0, 0, 354, 317, 1, 0, 0, 0, 354, 318, 1, 0, 0, 0, 354, 319, 1, 0, 0, 0, 354, 320, 1, 0, 0, 0, 354, 321, 1, 0, 0, 0, 354, 332, 1, 0, 0, 0, 354, 343, 1, 0, 0, 0, 355, 45, 1, 0, 0, 0, 356, 357, 5, 10, 0, 0, 357, 358, 5, 28, 0, 0, 358, 47, 1, 0, 0, 0, 359, 360, 5, 16, 0, 0, 360, 365, 3, 50, 25, 0, 361, 362, 5, 34, 0, 0, 362, 364, 3, 50, 25, 0, 363, 361, 1, 0, 0, 0, 364, 367, 1, 0, 0, 0, 365, 363, 1, 0, 0, 0, 365, 366, 1, 0, 0, 0, 366, 49, 1, 0, 0, 0, 367, 365, 1, 0, 0, 0, 368, 370, 3, 10, 5, 0, 369, 371, 7, 4, 0, 0, 370, 369, 1, 0, 0, 0, 370, 371, 1, 0, 0, 0, 371, 374, 1, 0, 0, 0, 372, 373, 5, 46, 0, 0, 373, 375, 7, 5, 0, 0, 374, 372, 1, 0, 0, 0, 374, 375, 1, 0, 0, 0, 375, 51, 1, 0, 0, 0, 376, 377, 5, 9, 0, 0, 377, 382, 3, 38, 19, 0, 378, 379, 5, 34, 0, 0, 379, 381, 3, 38, 19, 0, 380, 378, 1, 0, 0, 0, 381, 384, 1, 0, 0, 0, 382, 380, 1, 0, 0, 0, 382, 383, 1, 0, 0, 0, 383, 395, 1, 0, 0, 0, 384, 382, 1, 0, 0, 0, 385, 386, 5, 12, 0, 0, 386, 391, 3, 38, 19, 0, 387, 388, 5, 34, 0, 0, 388, 390, 3, 38, 19, 0, 389, 387, 1, 0, 0, 0, 390, 393, 1, 0, 0, 0, 391, 389, 1, 0, 0, 0, 391, 392, 1, 0, 0, 0, 392, 395, 1, 0, 0, 0, 393, 391, 1, 0, 0, 0, 394, 376, 1, 0, 0, 0, 394, 385, 1, 0, 0, 0, 395, 53, 1, 0, 0, 0, 396, 397, 5, 2, 0, 0, 397, 402, 3, 38, 19, 0, 398, 399, 5, 34, 0, 0, 399, 401, 3, 38, 19, 0, 400, 398, 1, 0, 0, 0, 401, 404, 1, 0, 0, 0, 402, 400, 1, 0, 0, 0, 402, 403, 1, 0, 0, 0, 403, 55, 1, 0, 0, 0, 404, 402, 1, 0, 0, 0, 405, 406, 5, 13, 0, 0, 406, 411, 3, 58, 29, 0, 407, 408, 5, 34, 0, 0, 408, 410, 3, 58, 29, 0, 409, 407, 1, 0, 0, 0, 410, 413, 1, 0, 0, 0, 411, 409, 1, 0, 0, 0, 411, 412, 1, 0, 0, 0, 412, 57, 1, 0, 0, 0, 413, 411, 1, 0, 0, 0, 414, 415, 3, 38, 19, 0, 415, 416, 5, 72, 0, 0, 416, 417, 3, 38, 19, 0, 417, 59, 1, 0, 0, 0, 418, 419, 5, 1, 0, 0, 419, 420, 3, 18, 9, 0, 420, 422, 3, 78, 39, 0, 421, 423, 3, 66, 33, 0, 422, 421, 1, 0, 0, 0, 422, 423, 1, 0, 0, 0, 423, 61, 1, 0, 0, 0, 424, 425, 5, 7, 0, 0, 425, 426, 3, 18, 9, 0, 426, 427, 3, 78, 39, 0, 427, 63, 1, 0, 0, 0, 428, 429, 5, 11, 0, 0, 429, 430, 3, 38, 19, 0, 430, 65, 1, 0, 0, 0, 431, 436, 3, 68, 34, 0, 432, 433, 5, 34, 0, 0, 433, 435, 3, 68, 34, 0, 434, 432, 1, 0, 0, 0, 435, 438, 1, 0, 0, 0, 436, 434, 1, 0, 0, 0, 436, 437, 1, 0, 0, 0, 437, 67, 1, 0, 0, 0, 438, 436, 1, 0, 0, 0, 439, 440, 3, 42, 21, 0, 440, 441, 5, 33, 0, 0, 441, 442, 3, 44, 22, 0, 442, 69, 1, 0, 0, 0, 443, 444, 7, 6, 0, 0, 444, 71, 1, 0, 0, 0, 445, 448, 3, 74, 37, 0, 446, 448, 3, 76, 38, 0, 447, 445, 1, 0, 0, 0, 447, 446, 1, 0, 0, 0, 448, 73, 1, 0, 0, 0, 449, 451, 7, 0, 0, 0, 450, 449, 1, 0, 0, 0, 450, 451, 1, 0, 0, 0, 451, 452, 1, 0, 0, 0, 452, 453, 5, 29, 0, 0, 453, 75, 1, 0, 0, 0, 454, 456, 7, 0, 0, 0, 455, 454, 1, 0, 0, 0, 455, 456, 1, 0, 0, 0, 456, 457, 1, 0, 0, 0, 457, 458, 5, 28, 0, 0, 458, 77, 1, 0, 0, 0, 459, 460, 5, 27, 0, 0, 460, 79, 1, 0, 0, 0, 461, 462, 7, 7, 0, 0, 462, 81, 1, 0, 0, 0, 463, 464, 5, 5, 0, 0, 464, 465, 3, 84, 42, 0, 465, 83, 1, 0, 0, 0, 466, 467, 5, 65, 0, 0, 467, 468, 3, 2, 1, 0, 468, 469, 5, 66, 0, 0, 469, 85, 1, 0, 0, 0, 470, 471, 5, 15, 0, 0, 471, 475, 5, 52, 0, 0, 472, 473, 5, 15, 0, 0, 473, 475, 5, 53, 0, 0, 474, 470, 1, 0, 0, 0, 474, 472, 1, 0, 0, 0, 475, 87, 1, 0, 0, 0, 476, 477, 5, 3, 0, 0, 477, 480, 3, 38, 19, 0, 478, 479, 5, 74, 0, 0, 479, 481, 3, 38, 19, 0, 480, 478, 1, 0, 0, 0, 480, 481, 1, 0, 0, 0, 481, 491, 1, 0, 0, 0, 482, 483, 5, 75, 0, 0, 483, 488, 3, 90, 45, 0, 484, 485, 5, 34, 0, 0, 485, 487, 3, 90, 45, 0, 486, 484, 1, 0, 0, 0, 487, 490, 1, 0, 0, 0, 488, 486, 1, 0, 0, 0, 488, 489, 1, 0, 0, 0, 489, 492, 1, 0, 0, 0, 490, 488, 1, 0, 0, 0, 491, 482, 1, 0, 0, 0, 491, 492, 1, 0, 0, 0, 492, 89, 1, 0, 0, 0, 493, 494, 3, 38, 19, 0, 494, 495, 5, 33, 0, 0, 495, 497, 1, 0, 0, 0, 496, 493, 1, 0, 0, 0, 496, 497, 1, 0, 0, 0, 497, 498, 1, 0, 0, 0, 498, 499, 3, 38, 19, 0, 499, 91, 1, 0, 0, 0, 51, 103, 110, 125, 137, 146, 154, 158, 166, 168, 173, 180, 185, 192, 198, 206, 208, 224, 227, 231, 241, 249, 257, 261, 270, 280, 284, 290, 297, 307, 327, 338, 349, 354, 365, 370, 374, 382, 391, 394, 402, 411, 422, 436, 447, 450, 455, 474, 480, 488, 491, 496]
\ No newline at end of file
+[4, 1, 81, 505, 2, 0, 7, 0, 2, 1, 7, 1, 2, 2, 7, 2, 2, 3, 7, 3, 2, 4, 7, 4, 2, 5, 7, 5, 2, 6, 7, 6, 2, 7, 7, 7, 2, 8, 7, 8, 2, 9, 7, 9, 2, 10, 7, 10, 2, 11, 7, 11, 2, 12, 7, 12, 2, 13, 7, 13, 2, 14, 7, 14, 2, 15, 7, 15, 2, 16, 7, 16, 2, 17, 7, 17, 2, 18, 7, 18, 2, 19, 7, 19, 2, 20, 7, 20, 2, 21, 7, 21, 2, 22, 7, 22, 2, 23, 7, 23, 2, 24, 7, 24, 2, 25, 7, 25, 2, 26, 7, 26, 2, 27, 7, 27, 2, 28, 7, 28, 2, 29, 7, 29, 2, 30, 7, 30, 2, 31, 7, 31, 2, 32, 7, 32, 2, 33, 7, 33, 2, 34, 7, 34, 2, 35, 7, 35, 2, 36, 7, 36, 2, 37, 7, 37, 2, 38, 7, 38, 2, 39, 7, 39, 2, 40, 7, 40, 2, 41, 7, 41, 2, 42, 7, 42, 2, 43, 7, 43, 2, 44, 7, 44, 2, 45, 7, 45, 2, 46, 7, 46, 1, 0, 1, 0, 1, 0, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 5, 1, 104, 8, 1, 10, 1, 12, 1, 107, 9, 1, 1, 2, 1, 2, 1, 2, 1, 2, 3, 2, 113, 8, 2, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 3, 3, 128, 8, 3, 1, 4, 1, 4, 1, 4, 1, 5, 1, 5, 1, 5, 1, 5, 1, 5, 1, 5, 1, 5, 3, 5, 140, 8, 5, 1, 5, 1, 5, 1, 5, 1, 5, 1, 5, 5, 5, 147, 8, 5, 10, 5, 12, 5, 150, 9, 5, 1, 5, 1, 5, 1, 5, 1, 5, 1, 5, 3, 5, 157, 8, 5, 1, 5, 1, 5, 3, 5, 161, 8, 5, 1, 5, 1, 5, 1, 5, 1, 5, 1, 5, 1, 5, 5, 5, 169, 8, 5, 10, 5, 12, 5, 172, 9, 5, 1, 6, 1, 6, 3, 6, 176, 8, 6, 1, 6, 1, 6, 1, 6, 1, 6, 1, 6, 3, 6, 183, 8, 6, 1, 6, 1, 6, 1, 6, 3, 6, 188, 8, 6, 1, 7, 1, 7, 1, 7, 1, 7, 1, 7, 3, 7, 195, 8, 7, 1, 8, 1, 8, 1, 8, 1, 8, 3, 8, 201, 8, 8, 1, 8, 1, 8, 1, 8, 1, 8, 1, 8, 1, 8, 5, 8, 209, 8, 8, 10, 8, 12, 8, 212, 9, 8, 1, 9, 1, 9, 1, 9, 1, 9, 1, 9, 1, 9, 1, 9, 3, 9, 221, 8, 9, 1, 10, 1, 10, 1, 10, 1, 10, 1, 10, 1, 10, 5, 10, 229, 8, 10, 10, 10, 12, 10, 232, 9, 10, 3, 10, 234, 8, 10, 1, 10, 1, 10, 1, 11, 1, 11, 1, 11, 1, 12, 1, 12, 1, 12, 5, 12, 244, 8, 12, 10, 12, 12, 12, 247, 9, 12, 1, 13, 1, 13, 1, 13, 1, 13, 1, 13, 3, 13, 254, 8, 13, 1, 14, 1, 14, 1, 14, 1, 14, 5, 14, 260, 8, 14, 10, 14, 12, 14, 263, 9, 14, 1, 14, 3, 14, 266, 8, 14, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 5, 15, 273, 8, 15, 10, 15, 12, 15, 276, 9, 15, 1, 15, 1, 15, 1, 16, 1, 16, 1, 16, 1, 17, 1, 17, 3, 17, 285, 8, 17, 1, 17, 1, 17, 3, 17, 289, 8, 17, 1, 18, 1, 18, 1, 18, 1, 18, 3, 18, 295, 8, 18, 1, 19, 1, 19, 1, 19, 5, 19, 300, 8, 19, 10, 19, 12, 19, 303, 9, 19, 1, 20, 1, 20, 1, 21, 1, 21, 1, 21, 5, 21, 310, 8, 21, 10, 21, 12, 21, 313, 9, 21, 1, 22, 1, 22, 1, 23, 1, 23, 1, 23, 1, 23, 1, 23, 1, 23, 1, 23, 1, 23, 1, 23, 1, 23, 1, 23, 1, 23, 1, 23, 5, 23, 330, 8, 23, 10, 23, 12, 23, 333, 9, 23, 1, 23, 1, 23, 1, 23, 1, 23, 1, 23, 1, 23, 5, 23, 341, 8, 23, 10, 23, 12, 23, 344, 9, 23, 1, 23, 1, 23, 1, 23, 1, 23, 1, 23, 1, 23, 5, 23, 352, 8, 23, 10, 23, 12, 23, 355, 9, 23, 1, 23, 1, 23, 3, 23, 359, 8, 23, 1, 24, 1, 24, 1, 24, 1, 25, 1, 25, 1, 25, 1, 25, 5, 25, 368, 8, 25, 10, 25, 12, 25, 371, 9, 25, 1, 26, 1, 26, 3, 26, 375, 8, 26, 1, 26, 1, 26, 3, 26, 379, 8, 26, 1, 27, 1, 27, 1, 27, 1, 27, 5, 27, 385, 8, 27, 10, 27, 12, 27, 388, 9, 27, 1, 27, 1, 27, 1, 27, 1, 27, 5, 27, 394, 8, 27, 10, 27, 12, 27, 397, 9, 27, 3, 27, 399, 8, 27, 1, 28, 1, 28, 1, 28, 1, 28, 5, 28, 405, 8, 28, 10, 28, 12, 28, 408, 9, 28, 1, 29, 1, 29, 1, 29, 1, 29, 5, 29, 414, 8, 29, 10, 29, 12, 29, 417, 9, 29, 1, 30, 1, 30, 1, 30, 1, 30, 1, 31, 1, 31, 1, 31, 1, 31, 3, 31, 427, 8, 31, 1, 32, 1, 32, 1, 32, 1, 32, 1, 33, 1, 33, 1, 33, 1, 34, 1, 34, 1, 34, 5, 34, 439, 8, 34, 10, 34, 12, 34, 442, 9, 34, 1, 35, 1, 35, 1, 35, 1, 35, 1, 36, 1, 36, 1, 37, 1, 37, 3, 37, 452, 8, 37, 1, 38, 3, 38, 455, 8, 38, 1, 38, 1, 38, 1, 39, 3, 39, 460, 8, 39, 1, 39, 1, 39, 1, 40, 1, 40, 1, 41, 1, 41, 1, 42, 1, 42, 1, 42, 1, 43, 1, 43, 1, 43, 1, 43, 1, 44, 1, 44, 1, 44, 1, 44, 3, 44, 479, 8, 44, 1, 45, 1, 45, 1, 45, 1, 45, 3, 45, 485, 8, 45, 1, 45, 1, 45, 1, 45, 1, 45, 5, 45, 491, 8, 45, 10, 45, 12, 45, 494, 9, 45, 3, 45, 496, 8, 45, 1, 46, 1, 46, 1, 46, 3, 46, 501, 8, 46, 1, 46, 1, 46, 1, 46, 0, 3, 2, 10, 16, 47, 0, 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, 80, 82, 84, 86, 88, 90, 92, 0, 8, 1, 0, 60, 61, 1, 0, 62, 64, 1, 0, 76, 77, 1, 0, 67, 68, 2, 0, 32, 32, 35, 35, 1, 0, 38, 39, 2, 0, 37, 37, 51, 51, 1, 0, 54, 59, 535, 0, 94, 1, 0, 0, 0, 2, 97, 1, 0, 0, 0, 4, 112, 1, 0, 0, 0, 6, 127, 1, 0, 0, 0, 8, 129, 1, 0, 0, 0, 10, 160, 1, 0, 0, 0, 12, 187, 1, 0, 0, 0, 14, 194, 1, 0, 0, 0, 16, 200, 1, 0, 0, 0, 18, 220, 1, 0, 0, 0, 20, 222, 1, 0, 0, 0, 22, 237, 1, 0, 0, 0, 24, 240, 1, 0, 0, 0, 26, 253, 1, 0, 0, 0, 28, 255, 1, 0, 0, 0, 30, 267, 1, 0, 0, 0, 32, 279, 1, 0, 0, 0, 34, 282, 1, 0, 0, 0, 36, 290, 1, 0, 0, 0, 38, 296, 1, 0, 0, 0, 40, 304, 1, 0, 0, 0, 42, 306, 1, 0, 0, 0, 44, 314, 1, 0, 0, 0, 46, 358, 1, 0, 0, 0, 48, 360, 1, 0, 0, 0, 50, 363, 1, 0, 0, 0, 52, 372, 1, 0, 0, 0, 54, 398, 1, 0, 0, 0, 56, 400, 1, 0, 0, 0, 58, 409, 1, 0, 0, 0, 60, 418, 1, 0, 0, 0, 62, 422, 1, 0, 0, 0, 64, 428, 1, 0, 0, 0, 66, 432, 1, 0, 0, 0, 68, 435, 1, 0, 0, 0, 70, 443, 1, 0, 0, 0, 72, 447, 1, 0, 0, 0, 74, 451, 1, 0, 0, 0, 76, 454, 1, 0, 0, 0, 78, 459, 1, 0, 0, 0, 80, 463, 1, 0, 0, 0, 82, 465, 1, 0, 0, 0, 84, 467, 1, 0, 0, 0, 86, 470, 1, 0, 0, 0, 88, 478, 1, 0, 0, 0, 90, 480, 1, 0, 0, 0, 92, 500, 1, 0, 0, 0, 94, 95, 3, 2, 1, 0, 95, 96, 5, 0, 0, 1, 96, 1, 1, 0, 0, 0, 97, 98, 6, 1, -1, 0, 98, 99, 3, 4, 2, 0, 99, 105, 1, 0, 0, 0, 100, 101, 10, 1, 0, 0, 101, 102, 5, 26, 0, 0, 102, 104, 3, 6, 3, 0, 103, 100, 1, 0, 0, 0, 104, 107, 1, 0, 0, 0, 105, 103, 1, 0, 0, 0, 105, 106, 1, 0, 0, 0, 106, 3, 1, 0, 0, 0, 107, 105, 1, 0, 0, 0, 108, 113, 3, 84, 42, 0, 109, 113, 3, 28, 14, 0, 110, 113, 3, 22, 11, 0, 111, 113, 3, 88, 44, 0, 112, 108, 1, 0, 0, 0, 112, 109, 1, 0, 0, 0, 112, 110, 1, 0, 0, 0, 112, 111, 1, 0, 0, 0, 113, 5, 1, 0, 0, 0, 114, 128, 3, 32, 16, 0, 115, 128, 3, 36, 18, 0, 116, 128, 3, 48, 24, 0, 117, 128, 3, 54, 27, 0, 118, 128, 3, 50, 25, 0, 119, 128, 3, 34, 17, 0, 120, 128, 3, 8, 4, 0, 121, 128, 3, 56, 28, 0, 122, 128, 3, 58, 29, 0, 123, 128, 3, 62, 31, 0, 124, 128, 3, 64, 32, 0, 125, 128, 3, 90, 45, 0, 126, 128, 3, 66, 33, 0, 127, 114, 1, 0, 0, 0, 127, 115, 1, 0, 0, 0, 127, 116, 1, 0, 0, 0, 127, 117, 1, 0, 0, 0, 127, 118, 1, 0, 0, 0, 127, 119, 1, 0, 0, 0, 127, 120, 1, 0, 0, 0, 127, 121, 1, 0, 0, 0, 127, 122, 1, 0, 0, 0, 127, 123, 1, 0, 0, 0, 127, 124, 1, 0, 0, 0, 127, 125, 1, 0, 0, 0, 127, 126, 1, 0, 0, 0, 128, 7, 1, 0, 0, 0, 129, 130, 5, 18, 0, 0, 130, 131, 3, 10, 5, 0, 131, 9, 1, 0, 0, 0, 132, 133, 6, 5, -1, 0, 133, 134, 5, 44, 0, 0, 134, 161, 3, 10, 5, 7, 135, 161, 3, 14, 7, 0, 136, 161, 3, 12, 6, 0, 137, 139, 3, 14, 7, 0, 138, 140, 5, 44, 0, 0, 139, 138, 1, 0, 0, 0, 139, 140, 1, 0, 0, 0, 140, 141, 1, 0, 0, 0, 141, 142, 5, 41, 0, 0, 142, 143, 5, 40, 0, 0, 143, 148, 3, 14, 7, 0, 144, 145, 5, 34, 0, 0, 145, 147, 3, 14, 7, 0, 146, 144, 1, 0, 0, 0, 147, 150, 1, 0, 0, 0, 148, 146, 1, 0, 0, 0, 148, 149, 1, 0, 0, 0, 149, 151, 1, 0, 0, 0, 150, 148, 1, 0, 0, 0, 151, 152, 5, 50, 0, 0, 152, 161, 1, 0, 0, 0, 153, 154, 3, 14, 7, 0, 154, 156, 5, 42, 0, 0, 155, 157, 5, 44, 0, 0, 156, 155, 1, 0, 0, 0, 156, 157, 1, 0, 0, 0, 157, 158, 1, 0, 0, 0, 158, 159, 5, 45, 0, 0, 159, 161, 1, 0, 0, 0, 160, 132, 1, 0, 0, 0, 160, 135, 1, 0, 0, 0, 160, 136, 1, 0, 0, 0, 160, 137, 1, 0, 0, 0, 160, 153, 1, 0, 0, 0, 161, 170, 1, 0, 0, 0, 162, 163, 10, 4, 0, 0, 163, 164, 5, 31, 0, 0, 164, 169, 3, 10, 5, 5, 165, 166, 10, 3, 0, 0, 166, 167, 5, 47, 0, 0, 167, 169, 3, 10, 5, 4, 168, 162, 1, 0, 0, 0, 168, 165, 1, 0, 0, 0, 169, 172, 1, 0, 0, 0, 170, 168, 1, 0, 0, 0, 170, 171, 1, 0, 0, 0, 171, 11, 1, 0, 0, 0, 172, 170, 1, 0, 0, 0, 173, 175, 3, 14, 7, 0, 174, 176, 5, 44, 0, 0, 175, 174, 1, 0, 0, 0, 175, 176, 1, 0, 0, 0, 176, 177, 1, 0, 0, 0, 177, 178, 5, 43, 0, 0, 178, 179, 3, 80, 40, 0, 179, 188, 1, 0, 0, 0, 180, 182, 3, 14, 7, 0, 181, 183, 5, 44, 0, 0, 182, 181, 1, 0, 0, 0, 182, 183, 1, 0, 0, 0, 183, 184, 1, 0, 0, 0, 184, 185, 5, 49, 0, 0, 185, 186, 3, 80, 40, 0, 186, 188, 1, 0, 0, 0, 187, 173, 1, 0, 0, 0, 187, 180, 1, 0, 0, 0, 188, 13, 1, 0, 0, 0, 189, 195, 3, 16, 8, 0, 190, 191, 3, 16, 8, 0, 191, 192, 3, 82, 41, 0, 192, 193, 3, 16, 8, 0, 193, 195, 1, 0, 0, 0, 194, 189, 1, 0, 0, 0, 194, 190, 1, 0, 0, 0, 195, 15, 1, 0, 0, 0, 196, 197, 6, 8, -1, 0, 197, 201, 3, 18, 9, 0, 198, 199, 7, 0, 0, 0, 199, 201, 3, 16, 8, 3, 200, 196, 1, 0, 0, 0, 200, 198, 1, 0, 0, 0, 201, 210, 1, 0, 0, 0, 202, 203, 10, 2, 0, 0, 203, 204, 7, 1, 0, 0, 204, 209, 3, 16, 8, 3, 205, 206, 10, 1, 0, 0, 206, 207, 7, 0, 0, 0, 207, 209, 3, 16, 8, 2, 208, 202, 1, 0, 0, 0, 208, 205, 1, 0, 0, 0, 209, 212, 1, 0, 0, 0, 210, 208, 1, 0, 0, 0, 210, 211, 1, 0, 0, 0, 211, 17, 1, 0, 0, 0, 212, 210, 1, 0, 0, 0, 213, 221, 3, 46, 23, 0, 214, 221, 3, 42, 21, 0, 215, 221, 3, 20, 10, 0, 216, 217, 5, 40, 0, 0, 217, 218, 3, 10, 5, 0, 218, 219, 5, 50, 0, 0, 219, 221, 1, 0, 0, 0, 220, 213, 1, 0, 0, 0, 220, 214, 1, 0, 0, 0, 220, 215, 1, 0, 0, 0, 220, 216, 1, 0, 0, 0, 221, 19, 1, 0, 0, 0, 222, 223, 3, 44, 22, 0, 223, 233, 5, 40, 0, 0, 224, 234, 5, 62, 0, 0, 225, 230, 3, 10, 5, 0, 226, 227, 5, 34, 0, 0, 227, 229, 3, 10, 5, 0, 228, 226, 1, 0, 0, 0, 229, 232, 1, 0, 0, 0, 230, 228, 1, 0, 0, 0, 230, 231, 1, 0, 0, 0, 231, 234, 1, 0, 0, 0, 232, 230, 1, 0, 0, 0, 233, 224, 1, 0, 0, 0, 233, 225, 1, 0, 0, 0, 233, 234, 1, 0, 0, 0, 234, 235, 1, 0, 0, 0, 235, 236, 5, 50, 0, 0, 236, 21, 1, 0, 0, 0, 237, 238, 5, 14, 0, 0, 238, 239, 3, 24, 12, 0, 239, 23, 1, 0, 0, 0, 240, 245, 3, 26, 13, 0, 241, 242, 5, 34, 0, 0, 242, 244, 3, 26, 13, 0, 243, 241, 1, 0, 0, 0, 244, 247, 1, 0, 0, 0, 245, 243, 1, 0, 0, 0, 245, 246, 1, 0, 0, 0, 246, 25, 1, 0, 0, 0, 247, 245, 1, 0, 0, 0, 248, 254, 3, 10, 5, 0, 249, 250, 3, 42, 21, 0, 250, 251, 5, 33, 0, 0, 251, 252, 3, 10, 5, 0, 252, 254, 1, 0, 0, 0, 253, 248, 1, 0, 0, 0, 253, 249, 1, 0, 0, 0, 254, 27, 1, 0, 0, 0, 255, 256, 5, 6, 0, 0, 256, 261, 3, 40, 20, 0, 257, 258, 5, 34, 0, 0, 258, 260, 3, 40, 20, 0, 259, 257, 1, 0, 0, 0, 260, 263, 1, 0, 0, 0, 261, 259, 1, 0, 0, 0, 261, 262, 1, 0, 0, 0, 262, 265, 1, 0, 0, 0, 263, 261, 1, 0, 0, 0, 264, 266, 3, 30, 15, 0, 265, 264, 1, 0, 0, 0, 265, 266, 1, 0, 0, 0, 266, 29, 1, 0, 0, 0, 267, 268, 5, 65, 0, 0, 268, 269, 5, 73, 0, 0, 269, 274, 3, 40, 20, 0, 270, 271, 5, 34, 0, 0, 271, 273, 3, 40, 20, 0, 272, 270, 1, 0, 0, 0, 273, 276, 1, 0, 0, 0, 274, 272, 1, 0, 0, 0, 274, 275, 1, 0, 0, 0, 275, 277, 1, 0, 0, 0, 276, 274, 1, 0, 0, 0, 277, 278, 5, 66, 0, 0, 278, 31, 1, 0, 0, 0, 279, 280, 5, 4, 0, 0, 280, 281, 3, 24, 12, 0, 281, 33, 1, 0, 0, 0, 282, 284, 5, 17, 0, 0, 283, 285, 3, 24, 12, 0, 284, 283, 1, 0, 0, 0, 284, 285, 1, 0, 0, 0, 285, 288, 1, 0, 0, 0, 286, 287, 5, 30, 0, 0, 287, 289, 3, 38, 19, 0, 288, 286, 1, 0, 0, 0, 288, 289, 1, 0, 0, 0, 289, 35, 1, 0, 0, 0, 290, 291, 5, 8, 0, 0, 291, 294, 3, 24, 12, 0, 292, 293, 5, 30, 0, 0, 293, 295, 3, 38, 19, 0, 294, 292, 1, 0, 0, 0, 294, 295, 1, 0, 0, 0, 295, 37, 1, 0, 0, 0, 296, 301, 3, 42, 21, 0, 297, 298, 5, 34, 0, 0, 298, 300, 3, 42, 21, 0, 299, 297, 1, 0, 0, 0, 300, 303, 1, 0, 0, 0, 301, 299, 1, 0, 0, 0, 301, 302, 1, 0, 0, 0, 302, 39, 1, 0, 0, 0, 303, 301, 1, 0, 0, 0, 304, 305, 7, 2, 0, 0, 305, 41, 1, 0, 0, 0, 306, 311, 3, 44, 22, 0, 307, 308, 5, 36, 0, 0, 308, 310, 3, 44, 22, 0, 309, 307, 1, 0, 0, 0, 310, 313, 1, 0, 0, 0, 311, 309, 1, 0, 0, 0, 311, 312, 1, 0, 0, 0, 312, 43, 1, 0, 0, 0, 313, 311, 1, 0, 0, 0, 314, 315, 7, 3, 0, 0, 315, 45, 1, 0, 0, 0, 316, 359, 5, 45, 0, 0, 317, 318, 3, 78, 39, 0, 318, 319, 5, 67, 0, 0, 319, 359, 1, 0, 0, 0, 320, 359, 3, 76, 38, 0, 321, 359, 3, 78, 39, 0, 322, 359, 3, 72, 36, 0, 323, 359, 5, 48, 0, 0, 324, 359, 3, 80, 40, 0, 325, 326, 5, 65, 0, 0, 326, 331, 3, 74, 37, 0, 327, 328, 5, 34, 0, 0, 328, 330, 3, 74, 37, 0, 329, 327, 1, 0, 0, 0, 330, 333, 1, 0, 0, 0, 331, 329, 1, 0, 0, 0, 331, 332, 1, 0, 0, 0, 332, 334, 1, 0, 0, 0, 333, 331, 1, 0, 0, 0, 334, 335, 5, 66, 0, 0, 335, 359, 1, 0, 0, 0, 336, 337, 5, 65, 0, 0, 337, 342, 3, 72, 36, 0, 338, 339, 5, 34, 0, 0, 339, 341, 3, 72, 36, 0, 340, 338, 1, 0, 0, 0, 341, 344, 1, 0, 0, 0, 342, 340, 1, 0, 0, 0, 342, 343, 1, 0, 0, 0, 343, 345, 1, 0, 0, 0, 344, 342, 1, 0, 0, 0, 345, 346, 5, 66, 0, 0, 346, 359, 1, 0, 0, 0, 347, 348, 5, 65, 0, 0, 348, 353, 3, 80, 40, 0, 349, 350, 5, 34, 0, 0, 350, 352, 3, 80, 40, 0, 351, 349, 1, 0, 0, 0, 352, 355, 1, 0, 0, 0, 353, 351, 1, 0, 0, 0, 353, 354, 1, 0, 0, 0, 354, 356, 1, 0, 0, 0, 355, 353, 1, 0, 0, 0, 356, 357, 5, 66, 0, 0, 357, 359, 1, 0, 0, 0, 358, 316, 1, 0, 0, 0, 358, 317, 1, 0, 0, 0, 358, 320, 1, 0, 0, 0, 358, 321, 1, 0, 0, 0, 358, 322, 1, 0, 0, 0, 358, 323, 1, 0, 0, 0, 358, 324, 1, 0, 0, 0, 358, 325, 1, 0, 0, 0, 358, 336, 1, 0, 0, 0, 358, 347, 1, 0, 0, 0, 359, 47, 1, 0, 0, 0, 360, 361, 5, 10, 0, 0, 361, 362, 5, 28, 0, 0, 362, 49, 1, 0, 0, 0, 363, 364, 5, 16, 0, 0, 364, 369, 3, 52, 26, 0, 365, 366, 5, 34, 0, 0, 366, 368, 3, 52, 26, 0, 367, 365, 1, 0, 0, 0, 368, 371, 1, 0, 0, 0, 369, 367, 1, 0, 0, 0, 369, 370, 1, 0, 0, 0, 370, 51, 1, 0, 0, 0, 371, 369, 1, 0, 0, 0, 372, 374, 3, 10, 5, 0, 373, 375, 7, 4, 0, 0, 374, 373, 1, 0, 0, 0, 374, 375, 1, 0, 0, 0, 375, 378, 1, 0, 0, 0, 376, 377, 5, 46, 0, 0, 377, 379, 7, 5, 0, 0, 378, 376, 1, 0, 0, 0, 378, 379, 1, 0, 0, 0, 379, 53, 1, 0, 0, 0, 380, 381, 5, 9, 0, 0, 381, 386, 3, 40, 20, 0, 382, 383, 5, 34, 0, 0, 383, 385, 3, 40, 20, 0, 384, 382, 1, 0, 0, 0, 385, 388, 1, 0, 0, 0, 386, 384, 1, 0, 0, 0, 386, 387, 1, 0, 0, 0, 387, 399, 1, 0, 0, 0, 388, 386, 1, 0, 0, 0, 389, 390, 5, 12, 0, 0, 390, 395, 3, 40, 20, 0, 391, 392, 5, 34, 0, 0, 392, 394, 3, 40, 20, 0, 393, 391, 1, 0, 0, 0, 394, 397, 1, 0, 0, 0, 395, 393, 1, 0, 0, 0, 395, 396, 1, 0, 0, 0, 396, 399, 1, 0, 0, 0, 397, 395, 1, 0, 0, 0, 398, 380, 1, 0, 0, 0, 398, 389, 1, 0, 0, 0, 399, 55, 1, 0, 0, 0, 400, 401, 5, 2, 0, 0, 401, 406, 3, 40, 20, 0, 402, 403, 5, 34, 0, 0, 403, 405, 3, 40, 20, 0, 404, 402, 1, 0, 0, 0, 405, 408, 1, 0, 0, 0, 406, 404, 1, 0, 0, 0, 406, 407, 1, 0, 0, 0, 407, 57, 1, 0, 0, 0, 408, 406, 1, 0, 0, 0, 409, 410, 5, 13, 0, 0, 410, 415, 3, 60, 30, 0, 411, 412, 5, 34, 0, 0, 412, 414, 3, 60, 30, 0, 413, 411, 1, 0, 0, 0, 414, 417, 1, 0, 0, 0, 415, 413, 1, 0, 0, 0, 415, 416, 1, 0, 0, 0, 416, 59, 1, 0, 0, 0, 417, 415, 1, 0, 0, 0, 418, 419, 3, 40, 20, 0, 419, 420, 5, 72, 0, 0, 420, 421, 3, 40, 20, 0, 421, 61, 1, 0, 0, 0, 422, 423, 5, 1, 0, 0, 423, 424, 3, 18, 9, 0, 424, 426, 3, 80, 40, 0, 425, 427, 3, 68, 34, 0, 426, 425, 1, 0, 0, 0, 426, 427, 1, 0, 0, 0, 427, 63, 1, 0, 0, 0, 428, 429, 5, 7, 0, 0, 429, 430, 3, 18, 9, 0, 430, 431, 3, 80, 40, 0, 431, 65, 1, 0, 0, 0, 432, 433, 5, 11, 0, 0, 433, 434, 3, 40, 20, 0, 434, 67, 1, 0, 0, 0, 435, 440, 3, 70, 35, 0, 436, 437, 5, 34, 0, 0, 437, 439, 3, 70, 35, 0, 438, 436, 1, 0, 0, 0, 439, 442, 1, 0, 0, 0, 440, 438, 1, 0, 0, 0, 440, 441, 1, 0, 0, 0, 441, 69, 1, 0, 0, 0, 442, 440, 1, 0, 0, 0, 443, 444, 3, 44, 22, 0, 444, 445, 5, 33, 0, 0, 445, 446, 3, 46, 23, 0, 446, 71, 1, 0, 0, 0, 447, 448, 7, 6, 0, 0, 448, 73, 1, 0, 0, 0, 449, 452, 3, 76, 38, 0, 450, 452, 3, 78, 39, 0, 451, 449, 1, 0, 0, 0, 451, 450, 1, 0, 0, 0, 452, 75, 1, 0, 0, 0, 453, 455, 7, 0, 0, 0, 454, 453, 1, 0, 0, 0, 454, 455, 1, 0, 0, 0, 455, 456, 1, 0, 0, 0, 456, 457, 5, 29, 0, 0, 457, 77, 1, 0, 0, 0, 458, 460, 7, 0, 0, 0, 459, 458, 1, 0, 0, 0, 459, 460, 1, 0, 0, 0, 460, 461, 1, 0, 0, 0, 461, 462, 5, 28, 0, 0, 462, 79, 1, 0, 0, 0, 463, 464, 5, 27, 0, 0, 464, 81, 1, 0, 0, 0, 465, 466, 7, 7, 0, 0, 466, 83, 1, 0, 0, 0, 467, 468, 5, 5, 0, 0, 468, 469, 3, 86, 43, 0, 469, 85, 1, 0, 0, 0, 470, 471, 5, 65, 0, 0, 471, 472, 3, 2, 1, 0, 472, 473, 5, 66, 0, 0, 473, 87, 1, 0, 0, 0, 474, 475, 5, 15, 0, 0, 475, 479, 5, 52, 0, 0, 476, 477, 5, 15, 0, 0, 477, 479, 5, 53, 0, 0, 478, 474, 1, 0, 0, 0, 478, 476, 1, 0, 0, 0, 479, 89, 1, 0, 0, 0, 480, 481, 5, 3, 0, 0, 481, 484, 3, 40, 20, 0, 482, 483, 5, 74, 0, 0, 483, 485, 3, 40, 20, 0, 484, 482, 1, 0, 0, 0, 484, 485, 1, 0, 0, 0, 485, 495, 1, 0, 0, 0, 486, 487, 5, 75, 0, 0, 487, 492, 3, 92, 46, 0, 488, 489, 5, 34, 0, 0, 489, 491, 3, 92, 46, 0, 490, 488, 1, 0, 0, 0, 491, 494, 1, 0, 0, 0, 492, 490, 1, 0, 0, 0, 492, 493, 1, 0, 0, 0, 493, 496, 1, 0, 0, 0, 494, 492, 1, 0, 0, 0, 495, 486, 1, 0, 0, 0, 495, 496, 1, 0, 0, 0, 496, 91, 1, 0, 0, 0, 497, 498, 3, 40, 20, 0, 498, 499, 5, 33, 0, 0, 499, 501, 1, 0, 0, 0, 500, 497, 1, 0, 0, 0, 500, 501, 1, 0, 0, 0, 501, 502, 1, 0, 0, 0, 502, 503, 3, 40, 20, 0, 503, 93, 1, 0, 0, 0, 51, 105, 112, 127, 139, 148, 156, 160, 168, 170, 175, 182, 187, 194, 200, 208, 210, 220, 230, 233, 245, 253, 261, 265, 274, 284, 288, 294, 301, 311, 331, 342, 353, 358, 369, 374, 378, 386, 395, 398, 406, 415, 426, 440, 451, 454, 459, 478, 484, 492, 495, 500]
\ No newline at end of file
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParser.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParser.java
index b5eac5f58f9f6..49d9abcc087c7 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParser.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParser.java
@@ -34,28 +34,29 @@ public class EsqlBaseParser extends Parser {
RULE_singleStatement = 0, RULE_query = 1, RULE_sourceCommand = 2, RULE_processingCommand = 3,
RULE_whereCommand = 4, RULE_booleanExpression = 5, RULE_regexBooleanExpression = 6,
RULE_valueExpression = 7, RULE_operatorExpression = 8, RULE_primaryExpression = 9,
- RULE_rowCommand = 10, RULE_fields = 11, RULE_field = 12, RULE_fromCommand = 13,
- RULE_metadata = 14, RULE_evalCommand = 15, RULE_statsCommand = 16, RULE_inlinestatsCommand = 17,
- RULE_grouping = 18, RULE_sourceIdentifier = 19, RULE_qualifiedName = 20,
- RULE_identifier = 21, RULE_constant = 22, RULE_limitCommand = 23, RULE_sortCommand = 24,
- RULE_orderExpression = 25, RULE_keepCommand = 26, RULE_dropCommand = 27,
- RULE_renameCommand = 28, RULE_renameClause = 29, RULE_dissectCommand = 30,
- RULE_grokCommand = 31, RULE_mvExpandCommand = 32, RULE_commandOptions = 33,
- RULE_commandOption = 34, RULE_booleanValue = 35, RULE_numericValue = 36,
- RULE_decimalValue = 37, RULE_integerValue = 38, RULE_string = 39, RULE_comparisonOperator = 40,
- RULE_explainCommand = 41, RULE_subqueryExpression = 42, RULE_showCommand = 43,
- RULE_enrichCommand = 44, RULE_enrichWithClause = 45;
+ RULE_functionExpression = 10, RULE_rowCommand = 11, RULE_fields = 12,
+ RULE_field = 13, RULE_fromCommand = 14, RULE_metadata = 15, RULE_evalCommand = 16,
+ RULE_statsCommand = 17, RULE_inlinestatsCommand = 18, RULE_grouping = 19,
+ RULE_sourceIdentifier = 20, RULE_qualifiedName = 21, RULE_identifier = 22,
+ RULE_constant = 23, RULE_limitCommand = 24, RULE_sortCommand = 25, RULE_orderExpression = 26,
+ RULE_keepCommand = 27, RULE_dropCommand = 28, RULE_renameCommand = 29,
+ RULE_renameClause = 30, RULE_dissectCommand = 31, RULE_grokCommand = 32,
+ RULE_mvExpandCommand = 33, RULE_commandOptions = 34, RULE_commandOption = 35,
+ RULE_booleanValue = 36, RULE_numericValue = 37, RULE_decimalValue = 38,
+ RULE_integerValue = 39, RULE_string = 40, RULE_comparisonOperator = 41,
+ RULE_explainCommand = 42, RULE_subqueryExpression = 43, RULE_showCommand = 44,
+ RULE_enrichCommand = 45, RULE_enrichWithClause = 46;
private static String[] makeRuleNames() {
return new String[] {
"singleStatement", "query", "sourceCommand", "processingCommand", "whereCommand",
"booleanExpression", "regexBooleanExpression", "valueExpression", "operatorExpression",
- "primaryExpression", "rowCommand", "fields", "field", "fromCommand",
- "metadata", "evalCommand", "statsCommand", "inlinestatsCommand", "grouping",
- "sourceIdentifier", "qualifiedName", "identifier", "constant", "limitCommand",
- "sortCommand", "orderExpression", "keepCommand", "dropCommand", "renameCommand",
- "renameClause", "dissectCommand", "grokCommand", "mvExpandCommand", "commandOptions",
- "commandOption", "booleanValue", "numericValue", "decimalValue", "integerValue",
- "string", "comparisonOperator", "explainCommand", "subqueryExpression",
+ "primaryExpression", "functionExpression", "rowCommand", "fields", "field",
+ "fromCommand", "metadata", "evalCommand", "statsCommand", "inlinestatsCommand",
+ "grouping", "sourceIdentifier", "qualifiedName", "identifier", "constant",
+ "limitCommand", "sortCommand", "orderExpression", "keepCommand", "dropCommand",
+ "renameCommand", "renameClause", "dissectCommand", "grokCommand", "mvExpandCommand",
+ "commandOptions", "commandOption", "booleanValue", "numericValue", "decimalValue",
+ "integerValue", "string", "comparisonOperator", "explainCommand", "subqueryExpression",
"showCommand", "enrichCommand", "enrichWithClause"
};
}
@@ -174,9 +175,9 @@ public final SingleStatementContext singleStatement() throws RecognitionExceptio
try {
enterOuterAlt(_localctx, 1);
{
- setState(92);
+ setState(94);
query(0);
- setState(93);
+ setState(95);
match(EOF);
}
}
@@ -268,11 +269,11 @@ private QueryContext query(int _p) throws RecognitionException {
_ctx = _localctx;
_prevctx = _localctx;
- setState(96);
+ setState(98);
sourceCommand();
}
_ctx.stop = _input.LT(-1);
- setState(103);
+ setState(105);
_errHandler.sync(this);
_alt = getInterpreter().adaptivePredict(_input,0,_ctx);
while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) {
@@ -283,16 +284,16 @@ private QueryContext query(int _p) throws RecognitionException {
{
_localctx = new CompositeQueryContext(new QueryContext(_parentctx, _parentState));
pushNewRecursionContext(_localctx, _startState, RULE_query);
- setState(98);
+ setState(100);
if (!(precpred(_ctx, 1))) throw new FailedPredicateException(this, "precpred(_ctx, 1)");
- setState(99);
+ setState(101);
match(PIPE);
- setState(100);
+ setState(102);
processingCommand();
}
}
}
- setState(105);
+ setState(107);
_errHandler.sync(this);
_alt = getInterpreter().adaptivePredict(_input,0,_ctx);
}
@@ -346,34 +347,34 @@ public final SourceCommandContext sourceCommand() throws RecognitionException {
SourceCommandContext _localctx = new SourceCommandContext(_ctx, getState());
enterRule(_localctx, 4, RULE_sourceCommand);
try {
- setState(110);
+ setState(112);
_errHandler.sync(this);
switch (_input.LA(1)) {
case EXPLAIN:
enterOuterAlt(_localctx, 1);
{
- setState(106);
+ setState(108);
explainCommand();
}
break;
case FROM:
enterOuterAlt(_localctx, 2);
{
- setState(107);
+ setState(109);
fromCommand();
}
break;
case ROW:
enterOuterAlt(_localctx, 3);
{
- setState(108);
+ setState(110);
rowCommand();
}
break;
case SHOW:
enterOuterAlt(_localctx, 4);
{
- setState(109);
+ setState(111);
showCommand();
}
break;
@@ -456,27 +457,27 @@ public final ProcessingCommandContext processingCommand() throws RecognitionExce
ProcessingCommandContext _localctx = new ProcessingCommandContext(_ctx, getState());
enterRule(_localctx, 6, RULE_processingCommand);
try {
- setState(125);
+ setState(127);
_errHandler.sync(this);
switch (_input.LA(1)) {
case EVAL:
enterOuterAlt(_localctx, 1);
{
- setState(112);
+ setState(114);
evalCommand();
}
break;
case INLINESTATS:
enterOuterAlt(_localctx, 2);
{
- setState(113);
+ setState(115);
inlinestatsCommand();
}
break;
case LIMIT:
enterOuterAlt(_localctx, 3);
{
- setState(114);
+ setState(116);
limitCommand();
}
break;
@@ -484,70 +485,70 @@ public final ProcessingCommandContext processingCommand() throws RecognitionExce
case PROJECT:
enterOuterAlt(_localctx, 4);
{
- setState(115);
+ setState(117);
keepCommand();
}
break;
case SORT:
enterOuterAlt(_localctx, 5);
{
- setState(116);
+ setState(118);
sortCommand();
}
break;
case STATS:
enterOuterAlt(_localctx, 6);
{
- setState(117);
+ setState(119);
statsCommand();
}
break;
case WHERE:
enterOuterAlt(_localctx, 7);
{
- setState(118);
+ setState(120);
whereCommand();
}
break;
case DROP:
enterOuterAlt(_localctx, 8);
{
- setState(119);
+ setState(121);
dropCommand();
}
break;
case RENAME:
enterOuterAlt(_localctx, 9);
{
- setState(120);
+ setState(122);
renameCommand();
}
break;
case DISSECT:
enterOuterAlt(_localctx, 10);
{
- setState(121);
+ setState(123);
dissectCommand();
}
break;
case GROK:
enterOuterAlt(_localctx, 11);
{
- setState(122);
+ setState(124);
grokCommand();
}
break;
case ENRICH:
enterOuterAlt(_localctx, 12);
{
- setState(123);
+ setState(125);
enrichCommand();
}
break;
case MV_EXPAND:
enterOuterAlt(_localctx, 13);
{
- setState(124);
+ setState(126);
mvExpandCommand();
}
break;
@@ -597,9 +598,9 @@ public final WhereCommandContext whereCommand() throws RecognitionException {
try {
enterOuterAlt(_localctx, 1);
{
- setState(127);
+ setState(129);
match(WHERE);
- setState(128);
+ setState(130);
booleanExpression(0);
}
}
@@ -786,7 +787,7 @@ private BooleanExpressionContext booleanExpression(int _p) throws RecognitionExc
int _alt;
enterOuterAlt(_localctx, 1);
{
- setState(158);
+ setState(160);
_errHandler.sync(this);
switch ( getInterpreter().adaptivePredict(_input,6,_ctx) ) {
case 1:
@@ -795,9 +796,9 @@ private BooleanExpressionContext booleanExpression(int _p) throws RecognitionExc
_ctx = _localctx;
_prevctx = _localctx;
- setState(131);
+ setState(133);
match(NOT);
- setState(132);
+ setState(134);
booleanExpression(7);
}
break;
@@ -806,7 +807,7 @@ private BooleanExpressionContext booleanExpression(int _p) throws RecognitionExc
_localctx = new BooleanDefaultContext(_localctx);
_ctx = _localctx;
_prevctx = _localctx;
- setState(133);
+ setState(135);
valueExpression();
}
break;
@@ -815,7 +816,7 @@ private BooleanExpressionContext booleanExpression(int _p) throws RecognitionExc
_localctx = new RegexExpressionContext(_localctx);
_ctx = _localctx;
_prevctx = _localctx;
- setState(134);
+ setState(136);
regexBooleanExpression();
}
break;
@@ -824,41 +825,41 @@ private BooleanExpressionContext booleanExpression(int _p) throws RecognitionExc
_localctx = new LogicalInContext(_localctx);
_ctx = _localctx;
_prevctx = _localctx;
- setState(135);
- valueExpression();
setState(137);
+ valueExpression();
+ setState(139);
_errHandler.sync(this);
_la = _input.LA(1);
if (_la==NOT) {
{
- setState(136);
+ setState(138);
match(NOT);
}
}
- setState(139);
+ setState(141);
match(IN);
- setState(140);
+ setState(142);
match(LP);
- setState(141);
+ setState(143);
valueExpression();
- setState(146);
+ setState(148);
_errHandler.sync(this);
_la = _input.LA(1);
while (_la==COMMA) {
{
{
- setState(142);
+ setState(144);
match(COMMA);
- setState(143);
+ setState(145);
valueExpression();
}
}
- setState(148);
+ setState(150);
_errHandler.sync(this);
_la = _input.LA(1);
}
- setState(149);
+ setState(151);
match(RP);
}
break;
@@ -867,27 +868,27 @@ private BooleanExpressionContext booleanExpression(int _p) throws RecognitionExc
_localctx = new IsNullContext(_localctx);
_ctx = _localctx;
_prevctx = _localctx;
- setState(151);
+ setState(153);
valueExpression();
- setState(152);
- match(IS);
setState(154);
+ match(IS);
+ setState(156);
_errHandler.sync(this);
_la = _input.LA(1);
if (_la==NOT) {
{
- setState(153);
+ setState(155);
match(NOT);
}
}
- setState(156);
+ setState(158);
match(NULL);
}
break;
}
_ctx.stop = _input.LT(-1);
- setState(168);
+ setState(170);
_errHandler.sync(this);
_alt = getInterpreter().adaptivePredict(_input,8,_ctx);
while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) {
@@ -895,7 +896,7 @@ private BooleanExpressionContext booleanExpression(int _p) throws RecognitionExc
if ( _parseListeners!=null ) triggerExitRuleEvent();
_prevctx = _localctx;
{
- setState(166);
+ setState(168);
_errHandler.sync(this);
switch ( getInterpreter().adaptivePredict(_input,7,_ctx) ) {
case 1:
@@ -903,11 +904,11 @@ private BooleanExpressionContext booleanExpression(int _p) throws RecognitionExc
_localctx = new LogicalBinaryContext(new BooleanExpressionContext(_parentctx, _parentState));
((LogicalBinaryContext)_localctx).left = _prevctx;
pushNewRecursionContext(_localctx, _startState, RULE_booleanExpression);
- setState(160);
+ setState(162);
if (!(precpred(_ctx, 4))) throw new FailedPredicateException(this, "precpred(_ctx, 4)");
- setState(161);
+ setState(163);
((LogicalBinaryContext)_localctx).operator = match(AND);
- setState(162);
+ setState(164);
((LogicalBinaryContext)_localctx).right = booleanExpression(5);
}
break;
@@ -916,18 +917,18 @@ private BooleanExpressionContext booleanExpression(int _p) throws RecognitionExc
_localctx = new LogicalBinaryContext(new BooleanExpressionContext(_parentctx, _parentState));
((LogicalBinaryContext)_localctx).left = _prevctx;
pushNewRecursionContext(_localctx, _startState, RULE_booleanExpression);
- setState(163);
+ setState(165);
if (!(precpred(_ctx, 3))) throw new FailedPredicateException(this, "precpred(_ctx, 3)");
- setState(164);
+ setState(166);
((LogicalBinaryContext)_localctx).operator = match(OR);
- setState(165);
+ setState(167);
((LogicalBinaryContext)_localctx).right = booleanExpression(4);
}
break;
}
}
}
- setState(170);
+ setState(172);
_errHandler.sync(this);
_alt = getInterpreter().adaptivePredict(_input,8,_ctx);
}
@@ -981,48 +982,48 @@ public final RegexBooleanExpressionContext regexBooleanExpression() throws Recog
enterRule(_localctx, 12, RULE_regexBooleanExpression);
int _la;
try {
- setState(185);
+ setState(187);
_errHandler.sync(this);
switch ( getInterpreter().adaptivePredict(_input,11,_ctx) ) {
case 1:
enterOuterAlt(_localctx, 1);
{
- setState(171);
- valueExpression();
setState(173);
+ valueExpression();
+ setState(175);
_errHandler.sync(this);
_la = _input.LA(1);
if (_la==NOT) {
{
- setState(172);
+ setState(174);
match(NOT);
}
}
- setState(175);
+ setState(177);
((RegexBooleanExpressionContext)_localctx).kind = match(LIKE);
- setState(176);
+ setState(178);
((RegexBooleanExpressionContext)_localctx).pattern = string();
}
break;
case 2:
enterOuterAlt(_localctx, 2);
{
- setState(178);
- valueExpression();
setState(180);
+ valueExpression();
+ setState(182);
_errHandler.sync(this);
_la = _input.LA(1);
if (_la==NOT) {
{
- setState(179);
+ setState(181);
match(NOT);
}
}
- setState(182);
+ setState(184);
((RegexBooleanExpressionContext)_localctx).kind = match(RLIKE);
- setState(183);
+ setState(185);
((RegexBooleanExpressionContext)_localctx).pattern = string();
}
break;
@@ -1104,14 +1105,14 @@ public final ValueExpressionContext valueExpression() throws RecognitionExceptio
ValueExpressionContext _localctx = new ValueExpressionContext(_ctx, getState());
enterRule(_localctx, 14, RULE_valueExpression);
try {
- setState(192);
+ setState(194);
_errHandler.sync(this);
switch ( getInterpreter().adaptivePredict(_input,12,_ctx) ) {
case 1:
_localctx = new ValueExpressionDefaultContext(_localctx);
enterOuterAlt(_localctx, 1);
{
- setState(187);
+ setState(189);
operatorExpression(0);
}
break;
@@ -1119,11 +1120,11 @@ public final ValueExpressionContext valueExpression() throws RecognitionExceptio
_localctx = new ComparisonContext(_localctx);
enterOuterAlt(_localctx, 2);
{
- setState(188);
+ setState(190);
((ComparisonContext)_localctx).left = operatorExpression(0);
- setState(189);
+ setState(191);
comparisonOperator();
- setState(190);
+ setState(192);
((ComparisonContext)_localctx).right = operatorExpression(0);
}
break;
@@ -1243,7 +1244,7 @@ private OperatorExpressionContext operatorExpression(int _p) throws RecognitionE
int _alt;
enterOuterAlt(_localctx, 1);
{
- setState(198);
+ setState(200);
_errHandler.sync(this);
switch ( getInterpreter().adaptivePredict(_input,13,_ctx) ) {
case 1:
@@ -1252,7 +1253,7 @@ private OperatorExpressionContext operatorExpression(int _p) throws RecognitionE
_ctx = _localctx;
_prevctx = _localctx;
- setState(195);
+ setState(197);
primaryExpression();
}
break;
@@ -1261,7 +1262,7 @@ private OperatorExpressionContext operatorExpression(int _p) throws RecognitionE
_localctx = new ArithmeticUnaryContext(_localctx);
_ctx = _localctx;
_prevctx = _localctx;
- setState(196);
+ setState(198);
((ArithmeticUnaryContext)_localctx).operator = _input.LT(1);
_la = _input.LA(1);
if ( !(_la==PLUS || _la==MINUS) ) {
@@ -1272,13 +1273,13 @@ private OperatorExpressionContext operatorExpression(int _p) throws RecognitionE
_errHandler.reportMatch(this);
consume();
}
- setState(197);
+ setState(199);
operatorExpression(3);
}
break;
}
_ctx.stop = _input.LT(-1);
- setState(208);
+ setState(210);
_errHandler.sync(this);
_alt = getInterpreter().adaptivePredict(_input,15,_ctx);
while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) {
@@ -1286,7 +1287,7 @@ private OperatorExpressionContext operatorExpression(int _p) throws RecognitionE
if ( _parseListeners!=null ) triggerExitRuleEvent();
_prevctx = _localctx;
{
- setState(206);
+ setState(208);
_errHandler.sync(this);
switch ( getInterpreter().adaptivePredict(_input,14,_ctx) ) {
case 1:
@@ -1294,9 +1295,9 @@ private OperatorExpressionContext operatorExpression(int _p) throws RecognitionE
_localctx = new ArithmeticBinaryContext(new OperatorExpressionContext(_parentctx, _parentState));
((ArithmeticBinaryContext)_localctx).left = _prevctx;
pushNewRecursionContext(_localctx, _startState, RULE_operatorExpression);
- setState(200);
+ setState(202);
if (!(precpred(_ctx, 2))) throw new FailedPredicateException(this, "precpred(_ctx, 2)");
- setState(201);
+ setState(203);
((ArithmeticBinaryContext)_localctx).operator = _input.LT(1);
_la = _input.LA(1);
if ( !((((_la - 62)) & ~0x3f) == 0 && ((1L << (_la - 62)) & 7L) != 0) ) {
@@ -1307,7 +1308,7 @@ private OperatorExpressionContext operatorExpression(int _p) throws RecognitionE
_errHandler.reportMatch(this);
consume();
}
- setState(202);
+ setState(204);
((ArithmeticBinaryContext)_localctx).right = operatorExpression(3);
}
break;
@@ -1316,9 +1317,9 @@ private OperatorExpressionContext operatorExpression(int _p) throws RecognitionE
_localctx = new ArithmeticBinaryContext(new OperatorExpressionContext(_parentctx, _parentState));
((ArithmeticBinaryContext)_localctx).left = _prevctx;
pushNewRecursionContext(_localctx, _startState, RULE_operatorExpression);
- setState(203);
+ setState(205);
if (!(precpred(_ctx, 1))) throw new FailedPredicateException(this, "precpred(_ctx, 1)");
- setState(204);
+ setState(206);
((ArithmeticBinaryContext)_localctx).operator = _input.LT(1);
_la = _input.LA(1);
if ( !(_la==PLUS || _la==MINUS) ) {
@@ -1329,14 +1330,14 @@ private OperatorExpressionContext operatorExpression(int _p) throws RecognitionE
_errHandler.reportMatch(this);
consume();
}
- setState(205);
+ setState(207);
((ArithmeticBinaryContext)_localctx).right = operatorExpression(2);
}
break;
}
}
}
- setState(210);
+ setState(212);
_errHandler.sync(this);
_alt = getInterpreter().adaptivePredict(_input,15,_ctx);
}
@@ -1428,34 +1429,22 @@ public T accept(ParseTreeVisitor extends T> visitor) {
}
}
@SuppressWarnings("CheckReturnValue")
- public static class FunctionExpressionContext extends PrimaryExpressionContext {
- public IdentifierContext identifier() {
- return getRuleContext(IdentifierContext.class,0);
- }
- public TerminalNode LP() { return getToken(EsqlBaseParser.LP, 0); }
- public TerminalNode RP() { return getToken(EsqlBaseParser.RP, 0); }
- public List booleanExpression() {
- return getRuleContexts(BooleanExpressionContext.class);
+ public static class FunctionContext extends PrimaryExpressionContext {
+ public FunctionExpressionContext functionExpression() {
+ return getRuleContext(FunctionExpressionContext.class,0);
}
- public BooleanExpressionContext booleanExpression(int i) {
- return getRuleContext(BooleanExpressionContext.class,i);
- }
- public List COMMA() { return getTokens(EsqlBaseParser.COMMA); }
- public TerminalNode COMMA(int i) {
- return getToken(EsqlBaseParser.COMMA, i);
- }
- public FunctionExpressionContext(PrimaryExpressionContext ctx) { copyFrom(ctx); }
+ public FunctionContext(PrimaryExpressionContext ctx) { copyFrom(ctx); }
@Override
public void enterRule(ParseTreeListener listener) {
- if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterFunctionExpression(this);
+ if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterFunction(this);
}
@Override
public void exitRule(ParseTreeListener listener) {
- if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitFunctionExpression(this);
+ if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitFunction(this);
}
@Override
public T accept(ParseTreeVisitor extends T> visitor) {
- if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor extends T>)visitor).visitFunctionExpression(this);
+ if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor extends T>)visitor).visitFunction(this);
else return visitor.visitChildren(this);
}
}
@@ -1463,16 +1452,15 @@ public T accept(ParseTreeVisitor extends T> visitor) {
public final PrimaryExpressionContext primaryExpression() throws RecognitionException {
PrimaryExpressionContext _localctx = new PrimaryExpressionContext(_ctx, getState());
enterRule(_localctx, 18, RULE_primaryExpression);
- int _la;
try {
- setState(231);
+ setState(220);
_errHandler.sync(this);
- switch ( getInterpreter().adaptivePredict(_input,18,_ctx) ) {
+ switch ( getInterpreter().adaptivePredict(_input,16,_ctx) ) {
case 1:
_localctx = new ConstantDefaultContext(_localctx);
enterOuterAlt(_localctx, 1);
{
- setState(211);
+ setState(213);
constant();
}
break;
@@ -1480,60 +1468,144 @@ public final PrimaryExpressionContext primaryExpression() throws RecognitionExce
_localctx = new DereferenceContext(_localctx);
enterOuterAlt(_localctx, 2);
{
- setState(212);
+ setState(214);
qualifiedName();
}
break;
case 3:
- _localctx = new ParenthesizedExpressionContext(_localctx);
+ _localctx = new FunctionContext(_localctx);
enterOuterAlt(_localctx, 3);
{
- setState(213);
- match(LP);
- setState(214);
- booleanExpression(0);
setState(215);
- match(RP);
+ functionExpression();
}
break;
case 4:
- _localctx = new FunctionExpressionContext(_localctx);
+ _localctx = new ParenthesizedExpressionContext(_localctx);
enterOuterAlt(_localctx, 4);
{
+ setState(216);
+ match(LP);
setState(217);
- identifier();
+ booleanExpression(0);
setState(218);
- match(LP);
- setState(227);
+ match(RP);
+ }
+ break;
+ }
+ }
+ catch (RecognitionException re) {
+ _localctx.exception = re;
+ _errHandler.reportError(this, re);
+ _errHandler.recover(this, re);
+ }
+ finally {
+ exitRule();
+ }
+ return _localctx;
+ }
+
+ @SuppressWarnings("CheckReturnValue")
+ public static class FunctionExpressionContext extends ParserRuleContext {
+ public IdentifierContext identifier() {
+ return getRuleContext(IdentifierContext.class,0);
+ }
+ public TerminalNode LP() { return getToken(EsqlBaseParser.LP, 0); }
+ public TerminalNode RP() { return getToken(EsqlBaseParser.RP, 0); }
+ public TerminalNode ASTERISK() { return getToken(EsqlBaseParser.ASTERISK, 0); }
+ public List booleanExpression() {
+ return getRuleContexts(BooleanExpressionContext.class);
+ }
+ public BooleanExpressionContext booleanExpression(int i) {
+ return getRuleContext(BooleanExpressionContext.class,i);
+ }
+ public List COMMA() { return getTokens(EsqlBaseParser.COMMA); }
+ public TerminalNode COMMA(int i) {
+ return getToken(EsqlBaseParser.COMMA, i);
+ }
+ public FunctionExpressionContext(ParserRuleContext parent, int invokingState) {
+ super(parent, invokingState);
+ }
+ @Override public int getRuleIndex() { return RULE_functionExpression; }
+ @Override
+ public void enterRule(ParseTreeListener listener) {
+ if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterFunctionExpression(this);
+ }
+ @Override
+ public void exitRule(ParseTreeListener listener) {
+ if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitFunctionExpression(this);
+ }
+ @Override
+ public T accept(ParseTreeVisitor extends T> visitor) {
+ if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor extends T>)visitor).visitFunctionExpression(this);
+ else return visitor.visitChildren(this);
+ }
+ }
+
+ public final FunctionExpressionContext functionExpression() throws RecognitionException {
+ FunctionExpressionContext _localctx = new FunctionExpressionContext(_ctx, getState());
+ enterRule(_localctx, 20, RULE_functionExpression);
+ int _la;
+ try {
+ enterOuterAlt(_localctx, 1);
+ {
+ setState(222);
+ identifier();
+ setState(223);
+ match(LP);
+ setState(233);
+ _errHandler.sync(this);
+ switch (_input.LA(1)) {
+ case ASTERISK:
+ {
+ setState(224);
+ match(ASTERISK);
+ }
+ break;
+ case STRING:
+ case INTEGER_LITERAL:
+ case DECIMAL_LITERAL:
+ case FALSE:
+ case LP:
+ case NOT:
+ case NULL:
+ case PARAM:
+ case TRUE:
+ case PLUS:
+ case MINUS:
+ case OPENING_BRACKET:
+ case UNQUOTED_IDENTIFIER:
+ case QUOTED_IDENTIFIER:
+ {
+ {
+ setState(225);
+ booleanExpression(0);
+ setState(230);
_errHandler.sync(this);
_la = _input.LA(1);
- if ((((_la - 27)) & ~0x3f) == 0 && ((1L << (_la - 27)) & 3599201870855L) != 0) {
+ while (_la==COMMA) {
+ {
{
- setState(219);
+ setState(226);
+ match(COMMA);
+ setState(227);
booleanExpression(0);
- setState(224);
- _errHandler.sync(this);
- _la = _input.LA(1);
- while (_la==COMMA) {
- {
- {
- setState(220);
- match(COMMA);
- setState(221);
- booleanExpression(0);
- }
- }
- setState(226);
- _errHandler.sync(this);
- _la = _input.LA(1);
}
}
+ setState(232);
+ _errHandler.sync(this);
+ _la = _input.LA(1);
+ }
}
-
- setState(229);
- match(RP);
}
break;
+ case RP:
+ break;
+ default:
+ break;
+ }
+ setState(235);
+ match(RP);
}
}
catch (RecognitionException re) {
@@ -1574,13 +1646,13 @@ public T accept(ParseTreeVisitor extends T> visitor) {
public final RowCommandContext rowCommand() throws RecognitionException {
RowCommandContext _localctx = new RowCommandContext(_ctx, getState());
- enterRule(_localctx, 20, RULE_rowCommand);
+ enterRule(_localctx, 22, RULE_rowCommand);
try {
enterOuterAlt(_localctx, 1);
{
- setState(233);
+ setState(237);
match(ROW);
- setState(234);
+ setState(238);
fields();
}
}
@@ -1628,28 +1700,28 @@ public T accept(ParseTreeVisitor extends T> visitor) {
public final FieldsContext fields() throws RecognitionException {
FieldsContext _localctx = new FieldsContext(_ctx, getState());
- enterRule(_localctx, 22, RULE_fields);
+ enterRule(_localctx, 24, RULE_fields);
try {
int _alt;
enterOuterAlt(_localctx, 1);
{
- setState(236);
+ setState(240);
field();
- setState(241);
+ setState(245);
_errHandler.sync(this);
_alt = getInterpreter().adaptivePredict(_input,19,_ctx);
while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) {
if ( _alt==1 ) {
{
{
- setState(237);
+ setState(241);
match(COMMA);
- setState(238);
+ setState(242);
field();
}
}
}
- setState(243);
+ setState(247);
_errHandler.sync(this);
_alt = getInterpreter().adaptivePredict(_input,19,_ctx);
}
@@ -1696,26 +1768,26 @@ public T accept(ParseTreeVisitor extends T> visitor) {
public final FieldContext field() throws RecognitionException {
FieldContext _localctx = new FieldContext(_ctx, getState());
- enterRule(_localctx, 24, RULE_field);
+ enterRule(_localctx, 26, RULE_field);
try {
- setState(249);
+ setState(253);
_errHandler.sync(this);
switch ( getInterpreter().adaptivePredict(_input,20,_ctx) ) {
case 1:
enterOuterAlt(_localctx, 1);
{
- setState(244);
+ setState(248);
booleanExpression(0);
}
break;
case 2:
enterOuterAlt(_localctx, 2);
{
- setState(245);
+ setState(249);
qualifiedName();
- setState(246);
+ setState(250);
match(ASSIGN);
- setState(247);
+ setState(251);
booleanExpression(0);
}
break;
@@ -1769,39 +1841,39 @@ public T accept(ParseTreeVisitor extends T> visitor) {
public final FromCommandContext fromCommand() throws RecognitionException {
FromCommandContext _localctx = new FromCommandContext(_ctx, getState());
- enterRule(_localctx, 26, RULE_fromCommand);
+ enterRule(_localctx, 28, RULE_fromCommand);
try {
int _alt;
enterOuterAlt(_localctx, 1);
{
- setState(251);
+ setState(255);
match(FROM);
- setState(252);
+ setState(256);
sourceIdentifier();
- setState(257);
+ setState(261);
_errHandler.sync(this);
_alt = getInterpreter().adaptivePredict(_input,21,_ctx);
while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) {
if ( _alt==1 ) {
{
{
- setState(253);
+ setState(257);
match(COMMA);
- setState(254);
+ setState(258);
sourceIdentifier();
}
}
}
- setState(259);
+ setState(263);
_errHandler.sync(this);
_alt = getInterpreter().adaptivePredict(_input,21,_ctx);
}
- setState(261);
+ setState(265);
_errHandler.sync(this);
switch ( getInterpreter().adaptivePredict(_input,22,_ctx) ) {
case 1:
{
- setState(260);
+ setState(264);
metadata();
}
break;
@@ -1855,34 +1927,34 @@ public T accept(ParseTreeVisitor extends T> visitor) {
public final MetadataContext metadata() throws RecognitionException {
MetadataContext _localctx = new MetadataContext(_ctx, getState());
- enterRule(_localctx, 28, RULE_metadata);
+ enterRule(_localctx, 30, RULE_metadata);
int _la;
try {
enterOuterAlt(_localctx, 1);
{
- setState(263);
+ setState(267);
match(OPENING_BRACKET);
- setState(264);
+ setState(268);
match(METADATA);
- setState(265);
+ setState(269);
sourceIdentifier();
- setState(270);
+ setState(274);
_errHandler.sync(this);
_la = _input.LA(1);
while (_la==COMMA) {
{
{
- setState(266);
+ setState(270);
match(COMMA);
- setState(267);
+ setState(271);
sourceIdentifier();
}
}
- setState(272);
+ setState(276);
_errHandler.sync(this);
_la = _input.LA(1);
}
- setState(273);
+ setState(277);
match(CLOSING_BRACKET);
}
}
@@ -1924,13 +1996,13 @@ public T accept(ParseTreeVisitor extends T> visitor) {
public final EvalCommandContext evalCommand() throws RecognitionException {
EvalCommandContext _localctx = new EvalCommandContext(_ctx, getState());
- enterRule(_localctx, 30, RULE_evalCommand);
+ enterRule(_localctx, 32, RULE_evalCommand);
try {
enterOuterAlt(_localctx, 1);
{
- setState(275);
+ setState(279);
match(EVAL);
- setState(276);
+ setState(280);
fields();
}
}
@@ -1976,30 +2048,30 @@ public T accept(ParseTreeVisitor extends T> visitor) {
public final StatsCommandContext statsCommand() throws RecognitionException {
StatsCommandContext _localctx = new StatsCommandContext(_ctx, getState());
- enterRule(_localctx, 32, RULE_statsCommand);
+ enterRule(_localctx, 34, RULE_statsCommand);
try {
enterOuterAlt(_localctx, 1);
{
- setState(278);
+ setState(282);
match(STATS);
- setState(280);
+ setState(284);
_errHandler.sync(this);
switch ( getInterpreter().adaptivePredict(_input,24,_ctx) ) {
case 1:
{
- setState(279);
+ setState(283);
fields();
}
break;
}
- setState(284);
+ setState(288);
_errHandler.sync(this);
switch ( getInterpreter().adaptivePredict(_input,25,_ctx) ) {
case 1:
{
- setState(282);
+ setState(286);
match(BY);
- setState(283);
+ setState(287);
grouping();
}
break;
@@ -2048,22 +2120,22 @@ public T accept(ParseTreeVisitor extends T> visitor) {
public final InlinestatsCommandContext inlinestatsCommand() throws RecognitionException {
InlinestatsCommandContext _localctx = new InlinestatsCommandContext(_ctx, getState());
- enterRule(_localctx, 34, RULE_inlinestatsCommand);
+ enterRule(_localctx, 36, RULE_inlinestatsCommand);
try {
enterOuterAlt(_localctx, 1);
{
- setState(286);
+ setState(290);
match(INLINESTATS);
- setState(287);
+ setState(291);
fields();
- setState(290);
+ setState(294);
_errHandler.sync(this);
switch ( getInterpreter().adaptivePredict(_input,26,_ctx) ) {
case 1:
{
- setState(288);
+ setState(292);
match(BY);
- setState(289);
+ setState(293);
grouping();
}
break;
@@ -2114,28 +2186,28 @@ public T accept(ParseTreeVisitor extends T> visitor) {
public final GroupingContext grouping() throws RecognitionException {
GroupingContext _localctx = new GroupingContext(_ctx, getState());
- enterRule(_localctx, 36, RULE_grouping);
+ enterRule(_localctx, 38, RULE_grouping);
try {
int _alt;
enterOuterAlt(_localctx, 1);
{
- setState(292);
+ setState(296);
qualifiedName();
- setState(297);
+ setState(301);
_errHandler.sync(this);
_alt = getInterpreter().adaptivePredict(_input,27,_ctx);
while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) {
if ( _alt==1 ) {
{
{
- setState(293);
+ setState(297);
match(COMMA);
- setState(294);
+ setState(298);
qualifiedName();
}
}
}
- setState(299);
+ setState(303);
_errHandler.sync(this);
_alt = getInterpreter().adaptivePredict(_input,27,_ctx);
}
@@ -2177,12 +2249,12 @@ public T accept(ParseTreeVisitor extends T> visitor) {
public final SourceIdentifierContext sourceIdentifier() throws RecognitionException {
SourceIdentifierContext _localctx = new SourceIdentifierContext(_ctx, getState());
- enterRule(_localctx, 38, RULE_sourceIdentifier);
+ enterRule(_localctx, 40, RULE_sourceIdentifier);
int _la;
try {
enterOuterAlt(_localctx, 1);
{
- setState(300);
+ setState(304);
_la = _input.LA(1);
if ( !(_la==SRC_UNQUOTED_IDENTIFIER || _la==SRC_QUOTED_IDENTIFIER) ) {
_errHandler.recoverInline(this);
@@ -2238,28 +2310,28 @@ public T accept(ParseTreeVisitor extends T> visitor) {
public final QualifiedNameContext qualifiedName() throws RecognitionException {
QualifiedNameContext _localctx = new QualifiedNameContext(_ctx, getState());
- enterRule(_localctx, 40, RULE_qualifiedName);
+ enterRule(_localctx, 42, RULE_qualifiedName);
try {
int _alt;
enterOuterAlt(_localctx, 1);
{
- setState(302);
+ setState(306);
identifier();
- setState(307);
+ setState(311);
_errHandler.sync(this);
_alt = getInterpreter().adaptivePredict(_input,28,_ctx);
while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) {
if ( _alt==1 ) {
{
{
- setState(303);
+ setState(307);
match(DOT);
- setState(304);
+ setState(308);
identifier();
}
}
}
- setState(309);
+ setState(313);
_errHandler.sync(this);
_alt = getInterpreter().adaptivePredict(_input,28,_ctx);
}
@@ -2301,12 +2373,12 @@ public T accept(ParseTreeVisitor extends T> visitor) {
public final IdentifierContext identifier() throws RecognitionException {
IdentifierContext _localctx = new IdentifierContext(_ctx, getState());
- enterRule(_localctx, 42, RULE_identifier);
+ enterRule(_localctx, 44, RULE_identifier);
int _la;
try {
enterOuterAlt(_localctx, 1);
{
- setState(310);
+ setState(314);
_la = _input.LA(1);
if ( !(_la==UNQUOTED_IDENTIFIER || _la==QUOTED_IDENTIFIER) ) {
_errHandler.recoverInline(this);
@@ -2568,17 +2640,17 @@ public T accept(ParseTreeVisitor extends T> visitor) {
public final ConstantContext constant() throws RecognitionException {
ConstantContext _localctx = new ConstantContext(_ctx, getState());
- enterRule(_localctx, 44, RULE_constant);
+ enterRule(_localctx, 46, RULE_constant);
int _la;
try {
- setState(354);
+ setState(358);
_errHandler.sync(this);
switch ( getInterpreter().adaptivePredict(_input,32,_ctx) ) {
case 1:
_localctx = new NullLiteralContext(_localctx);
enterOuterAlt(_localctx, 1);
{
- setState(312);
+ setState(316);
match(NULL);
}
break;
@@ -2586,9 +2658,9 @@ public final ConstantContext constant() throws RecognitionException {
_localctx = new QualifiedIntegerLiteralContext(_localctx);
enterOuterAlt(_localctx, 2);
{
- setState(313);
+ setState(317);
integerValue();
- setState(314);
+ setState(318);
match(UNQUOTED_IDENTIFIER);
}
break;
@@ -2596,7 +2668,7 @@ public final ConstantContext constant() throws RecognitionException {
_localctx = new DecimalLiteralContext(_localctx);
enterOuterAlt(_localctx, 3);
{
- setState(316);
+ setState(320);
decimalValue();
}
break;
@@ -2604,7 +2676,7 @@ public final ConstantContext constant() throws RecognitionException {
_localctx = new IntegerLiteralContext(_localctx);
enterOuterAlt(_localctx, 4);
{
- setState(317);
+ setState(321);
integerValue();
}
break;
@@ -2612,7 +2684,7 @@ public final ConstantContext constant() throws RecognitionException {
_localctx = new BooleanLiteralContext(_localctx);
enterOuterAlt(_localctx, 5);
{
- setState(318);
+ setState(322);
booleanValue();
}
break;
@@ -2620,7 +2692,7 @@ public final ConstantContext constant() throws RecognitionException {
_localctx = new InputParamContext(_localctx);
enterOuterAlt(_localctx, 6);
{
- setState(319);
+ setState(323);
match(PARAM);
}
break;
@@ -2628,7 +2700,7 @@ public final ConstantContext constant() throws RecognitionException {
_localctx = new StringLiteralContext(_localctx);
enterOuterAlt(_localctx, 7);
{
- setState(320);
+ setState(324);
string();
}
break;
@@ -2636,27 +2708,27 @@ public final ConstantContext constant() throws RecognitionException {
_localctx = new NumericArrayLiteralContext(_localctx);
enterOuterAlt(_localctx, 8);
{
- setState(321);
+ setState(325);
match(OPENING_BRACKET);
- setState(322);
+ setState(326);
numericValue();
- setState(327);
+ setState(331);
_errHandler.sync(this);
_la = _input.LA(1);
while (_la==COMMA) {
{
{
- setState(323);
+ setState(327);
match(COMMA);
- setState(324);
+ setState(328);
numericValue();
}
}
- setState(329);
+ setState(333);
_errHandler.sync(this);
_la = _input.LA(1);
}
- setState(330);
+ setState(334);
match(CLOSING_BRACKET);
}
break;
@@ -2664,27 +2736,27 @@ public final ConstantContext constant() throws RecognitionException {
_localctx = new BooleanArrayLiteralContext(_localctx);
enterOuterAlt(_localctx, 9);
{
- setState(332);
+ setState(336);
match(OPENING_BRACKET);
- setState(333);
+ setState(337);
booleanValue();
- setState(338);
+ setState(342);
_errHandler.sync(this);
_la = _input.LA(1);
while (_la==COMMA) {
{
{
- setState(334);
+ setState(338);
match(COMMA);
- setState(335);
+ setState(339);
booleanValue();
}
}
- setState(340);
+ setState(344);
_errHandler.sync(this);
_la = _input.LA(1);
}
- setState(341);
+ setState(345);
match(CLOSING_BRACKET);
}
break;
@@ -2692,27 +2764,27 @@ public final ConstantContext constant() throws RecognitionException {
_localctx = new StringArrayLiteralContext(_localctx);
enterOuterAlt(_localctx, 10);
{
- setState(343);
+ setState(347);
match(OPENING_BRACKET);
- setState(344);
+ setState(348);
string();
- setState(349);
+ setState(353);
_errHandler.sync(this);
_la = _input.LA(1);
while (_la==COMMA) {
{
{
- setState(345);
+ setState(349);
match(COMMA);
- setState(346);
+ setState(350);
string();
}
}
- setState(351);
+ setState(355);
_errHandler.sync(this);
_la = _input.LA(1);
}
- setState(352);
+ setState(356);
match(CLOSING_BRACKET);
}
break;
@@ -2754,13 +2826,13 @@ public T accept(ParseTreeVisitor extends T> visitor) {
public final LimitCommandContext limitCommand() throws RecognitionException {
LimitCommandContext _localctx = new LimitCommandContext(_ctx, getState());
- enterRule(_localctx, 46, RULE_limitCommand);
+ enterRule(_localctx, 48, RULE_limitCommand);
try {
enterOuterAlt(_localctx, 1);
{
- setState(356);
+ setState(360);
match(LIMIT);
- setState(357);
+ setState(361);
match(INTEGER_LITERAL);
}
}
@@ -2809,30 +2881,30 @@ public T accept(ParseTreeVisitor extends T> visitor) {
public final SortCommandContext sortCommand() throws RecognitionException {
SortCommandContext _localctx = new SortCommandContext(_ctx, getState());
- enterRule(_localctx, 48, RULE_sortCommand);
+ enterRule(_localctx, 50, RULE_sortCommand);
try {
int _alt;
enterOuterAlt(_localctx, 1);
{
- setState(359);
+ setState(363);
match(SORT);
- setState(360);
+ setState(364);
orderExpression();
- setState(365);
+ setState(369);
_errHandler.sync(this);
_alt = getInterpreter().adaptivePredict(_input,33,_ctx);
while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) {
if ( _alt==1 ) {
{
{
- setState(361);
+ setState(365);
match(COMMA);
- setState(362);
+ setState(366);
orderExpression();
}
}
}
- setState(367);
+ setState(371);
_errHandler.sync(this);
_alt = getInterpreter().adaptivePredict(_input,33,_ctx);
}
@@ -2882,19 +2954,19 @@ public T accept(ParseTreeVisitor extends T> visitor) {
public final OrderExpressionContext orderExpression() throws RecognitionException {
OrderExpressionContext _localctx = new OrderExpressionContext(_ctx, getState());
- enterRule(_localctx, 50, RULE_orderExpression);
+ enterRule(_localctx, 52, RULE_orderExpression);
int _la;
try {
enterOuterAlt(_localctx, 1);
{
- setState(368);
+ setState(372);
booleanExpression(0);
- setState(370);
+ setState(374);
_errHandler.sync(this);
switch ( getInterpreter().adaptivePredict(_input,34,_ctx) ) {
case 1:
{
- setState(369);
+ setState(373);
((OrderExpressionContext)_localctx).ordering = _input.LT(1);
_la = _input.LA(1);
if ( !(_la==ASC || _la==DESC) ) {
@@ -2908,14 +2980,14 @@ public final OrderExpressionContext orderExpression() throws RecognitionExceptio
}
break;
}
- setState(374);
+ setState(378);
_errHandler.sync(this);
switch ( getInterpreter().adaptivePredict(_input,35,_ctx) ) {
case 1:
{
- setState(372);
+ setState(376);
match(NULLS);
- setState(373);
+ setState(377);
((OrderExpressionContext)_localctx).nullOrdering = _input.LT(1);
_la = _input.LA(1);
if ( !(_la==FIRST || _la==LAST) ) {
@@ -2977,34 +3049,34 @@ public T accept(ParseTreeVisitor extends T> visitor) {
public final KeepCommandContext keepCommand() throws RecognitionException {
KeepCommandContext _localctx = new KeepCommandContext(_ctx, getState());
- enterRule(_localctx, 52, RULE_keepCommand);
+ enterRule(_localctx, 54, RULE_keepCommand);
try {
int _alt;
- setState(394);
+ setState(398);
_errHandler.sync(this);
switch (_input.LA(1)) {
case KEEP:
enterOuterAlt(_localctx, 1);
{
- setState(376);
+ setState(380);
match(KEEP);
- setState(377);
+ setState(381);
sourceIdentifier();
- setState(382);
+ setState(386);
_errHandler.sync(this);
_alt = getInterpreter().adaptivePredict(_input,36,_ctx);
while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) {
if ( _alt==1 ) {
{
{
- setState(378);
+ setState(382);
match(COMMA);
- setState(379);
+ setState(383);
sourceIdentifier();
}
}
}
- setState(384);
+ setState(388);
_errHandler.sync(this);
_alt = getInterpreter().adaptivePredict(_input,36,_ctx);
}
@@ -3013,25 +3085,25 @@ public final KeepCommandContext keepCommand() throws RecognitionException {
case PROJECT:
enterOuterAlt(_localctx, 2);
{
- setState(385);
+ setState(389);
match(PROJECT);
- setState(386);
+ setState(390);
sourceIdentifier();
- setState(391);
+ setState(395);
_errHandler.sync(this);
_alt = getInterpreter().adaptivePredict(_input,37,_ctx);
while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) {
if ( _alt==1 ) {
{
{
- setState(387);
+ setState(391);
match(COMMA);
- setState(388);
+ setState(392);
sourceIdentifier();
}
}
}
- setState(393);
+ setState(397);
_errHandler.sync(this);
_alt = getInterpreter().adaptivePredict(_input,37,_ctx);
}
@@ -3086,30 +3158,30 @@ public T accept(ParseTreeVisitor extends T> visitor) {
public final DropCommandContext dropCommand() throws RecognitionException {
DropCommandContext _localctx = new DropCommandContext(_ctx, getState());
- enterRule(_localctx, 54, RULE_dropCommand);
+ enterRule(_localctx, 56, RULE_dropCommand);
try {
int _alt;
enterOuterAlt(_localctx, 1);
{
- setState(396);
+ setState(400);
match(DROP);
- setState(397);
+ setState(401);
sourceIdentifier();
- setState(402);
+ setState(406);
_errHandler.sync(this);
_alt = getInterpreter().adaptivePredict(_input,39,_ctx);
while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) {
if ( _alt==1 ) {
{
{
- setState(398);
+ setState(402);
match(COMMA);
- setState(399);
+ setState(403);
sourceIdentifier();
}
}
}
- setState(404);
+ setState(408);
_errHandler.sync(this);
_alt = getInterpreter().adaptivePredict(_input,39,_ctx);
}
@@ -3160,30 +3232,30 @@ public T accept(ParseTreeVisitor extends T> visitor) {
public final RenameCommandContext renameCommand() throws RecognitionException {
RenameCommandContext _localctx = new RenameCommandContext(_ctx, getState());
- enterRule(_localctx, 56, RULE_renameCommand);
+ enterRule(_localctx, 58, RULE_renameCommand);
try {
int _alt;
enterOuterAlt(_localctx, 1);
{
- setState(405);
+ setState(409);
match(RENAME);
- setState(406);
+ setState(410);
renameClause();
- setState(411);
+ setState(415);
_errHandler.sync(this);
_alt = getInterpreter().adaptivePredict(_input,40,_ctx);
while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) {
if ( _alt==1 ) {
{
{
- setState(407);
+ setState(411);
match(COMMA);
- setState(408);
+ setState(412);
renameClause();
}
}
}
- setState(413);
+ setState(417);
_errHandler.sync(this);
_alt = getInterpreter().adaptivePredict(_input,40,_ctx);
}
@@ -3232,15 +3304,15 @@ public T accept(ParseTreeVisitor extends T> visitor) {
public final RenameClauseContext renameClause() throws RecognitionException {
RenameClauseContext _localctx = new RenameClauseContext(_ctx, getState());
- enterRule(_localctx, 58, RULE_renameClause);
+ enterRule(_localctx, 60, RULE_renameClause);
try {
enterOuterAlt(_localctx, 1);
{
- setState(414);
+ setState(418);
((RenameClauseContext)_localctx).oldName = sourceIdentifier();
- setState(415);
+ setState(419);
match(AS);
- setState(416);
+ setState(420);
((RenameClauseContext)_localctx).newName = sourceIdentifier();
}
}
@@ -3288,22 +3360,22 @@ public T accept(ParseTreeVisitor extends T> visitor) {
public final DissectCommandContext dissectCommand() throws RecognitionException {
DissectCommandContext _localctx = new DissectCommandContext(_ctx, getState());
- enterRule(_localctx, 60, RULE_dissectCommand);
+ enterRule(_localctx, 62, RULE_dissectCommand);
try {
enterOuterAlt(_localctx, 1);
{
- setState(418);
+ setState(422);
match(DISSECT);
- setState(419);
+ setState(423);
primaryExpression();
- setState(420);
+ setState(424);
string();
- setState(422);
+ setState(426);
_errHandler.sync(this);
switch ( getInterpreter().adaptivePredict(_input,41,_ctx) ) {
case 1:
{
- setState(421);
+ setState(425);
commandOptions();
}
break;
@@ -3351,15 +3423,15 @@ public T accept(ParseTreeVisitor extends T> visitor) {
public final GrokCommandContext grokCommand() throws RecognitionException {
GrokCommandContext _localctx = new GrokCommandContext(_ctx, getState());
- enterRule(_localctx, 62, RULE_grokCommand);
+ enterRule(_localctx, 64, RULE_grokCommand);
try {
enterOuterAlt(_localctx, 1);
{
- setState(424);
+ setState(428);
match(GROK);
- setState(425);
+ setState(429);
primaryExpression();
- setState(426);
+ setState(430);
string();
}
}
@@ -3401,13 +3473,13 @@ public T accept(ParseTreeVisitor extends T> visitor) {
public final MvExpandCommandContext mvExpandCommand() throws RecognitionException {
MvExpandCommandContext _localctx = new MvExpandCommandContext(_ctx, getState());
- enterRule(_localctx, 64, RULE_mvExpandCommand);
+ enterRule(_localctx, 66, RULE_mvExpandCommand);
try {
enterOuterAlt(_localctx, 1);
{
- setState(428);
+ setState(432);
match(MV_EXPAND);
- setState(429);
+ setState(433);
sourceIdentifier();
}
}
@@ -3455,28 +3527,28 @@ public T accept(ParseTreeVisitor extends T> visitor) {
public final CommandOptionsContext commandOptions() throws RecognitionException {
CommandOptionsContext _localctx = new CommandOptionsContext(_ctx, getState());
- enterRule(_localctx, 66, RULE_commandOptions);
+ enterRule(_localctx, 68, RULE_commandOptions);
try {
int _alt;
enterOuterAlt(_localctx, 1);
{
- setState(431);
+ setState(435);
commandOption();
- setState(436);
+ setState(440);
_errHandler.sync(this);
_alt = getInterpreter().adaptivePredict(_input,42,_ctx);
while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) {
if ( _alt==1 ) {
{
{
- setState(432);
+ setState(436);
match(COMMA);
- setState(433);
+ setState(437);
commandOption();
}
}
}
- setState(438);
+ setState(442);
_errHandler.sync(this);
_alt = getInterpreter().adaptivePredict(_input,42,_ctx);
}
@@ -3523,15 +3595,15 @@ public T accept(ParseTreeVisitor extends T> visitor) {
public final CommandOptionContext commandOption() throws RecognitionException {
CommandOptionContext _localctx = new CommandOptionContext(_ctx, getState());
- enterRule(_localctx, 68, RULE_commandOption);
+ enterRule(_localctx, 70, RULE_commandOption);
try {
enterOuterAlt(_localctx, 1);
{
- setState(439);
+ setState(443);
identifier();
- setState(440);
+ setState(444);
match(ASSIGN);
- setState(441);
+ setState(445);
constant();
}
}
@@ -3571,12 +3643,12 @@ public T accept(ParseTreeVisitor extends T> visitor) {
public final BooleanValueContext booleanValue() throws RecognitionException {
BooleanValueContext _localctx = new BooleanValueContext(_ctx, getState());
- enterRule(_localctx, 70, RULE_booleanValue);
+ enterRule(_localctx, 72, RULE_booleanValue);
int _la;
try {
enterOuterAlt(_localctx, 1);
{
- setState(443);
+ setState(447);
_la = _input.LA(1);
if ( !(_la==FALSE || _la==TRUE) ) {
_errHandler.recoverInline(this);
@@ -3628,22 +3700,22 @@ public T accept(ParseTreeVisitor extends T> visitor) {
public final NumericValueContext numericValue() throws RecognitionException {
NumericValueContext _localctx = new NumericValueContext(_ctx, getState());
- enterRule(_localctx, 72, RULE_numericValue);
+ enterRule(_localctx, 74, RULE_numericValue);
try {
- setState(447);
+ setState(451);
_errHandler.sync(this);
switch ( getInterpreter().adaptivePredict(_input,43,_ctx) ) {
case 1:
enterOuterAlt(_localctx, 1);
{
- setState(445);
+ setState(449);
decimalValue();
}
break;
case 2:
enterOuterAlt(_localctx, 2);
{
- setState(446);
+ setState(450);
integerValue();
}
break;
@@ -3686,17 +3758,17 @@ public T accept(ParseTreeVisitor extends T> visitor) {
public final DecimalValueContext decimalValue() throws RecognitionException {
DecimalValueContext _localctx = new DecimalValueContext(_ctx, getState());
- enterRule(_localctx, 74, RULE_decimalValue);
+ enterRule(_localctx, 76, RULE_decimalValue);
int _la;
try {
enterOuterAlt(_localctx, 1);
{
- setState(450);
+ setState(454);
_errHandler.sync(this);
_la = _input.LA(1);
if (_la==PLUS || _la==MINUS) {
{
- setState(449);
+ setState(453);
_la = _input.LA(1);
if ( !(_la==PLUS || _la==MINUS) ) {
_errHandler.recoverInline(this);
@@ -3709,7 +3781,7 @@ public final DecimalValueContext decimalValue() throws RecognitionException {
}
}
- setState(452);
+ setState(456);
match(DECIMAL_LITERAL);
}
}
@@ -3750,17 +3822,17 @@ public T accept(ParseTreeVisitor extends T> visitor) {
public final IntegerValueContext integerValue() throws RecognitionException {
IntegerValueContext _localctx = new IntegerValueContext(_ctx, getState());
- enterRule(_localctx, 76, RULE_integerValue);
+ enterRule(_localctx, 78, RULE_integerValue);
int _la;
try {
enterOuterAlt(_localctx, 1);
{
- setState(455);
+ setState(459);
_errHandler.sync(this);
_la = _input.LA(1);
if (_la==PLUS || _la==MINUS) {
{
- setState(454);
+ setState(458);
_la = _input.LA(1);
if ( !(_la==PLUS || _la==MINUS) ) {
_errHandler.recoverInline(this);
@@ -3773,7 +3845,7 @@ public final IntegerValueContext integerValue() throws RecognitionException {
}
}
- setState(457);
+ setState(461);
match(INTEGER_LITERAL);
}
}
@@ -3812,11 +3884,11 @@ public T accept(ParseTreeVisitor extends T> visitor) {
public final StringContext string() throws RecognitionException {
StringContext _localctx = new StringContext(_ctx, getState());
- enterRule(_localctx, 78, RULE_string);
+ enterRule(_localctx, 80, RULE_string);
try {
enterOuterAlt(_localctx, 1);
{
- setState(459);
+ setState(463);
match(STRING);
}
}
@@ -3860,12 +3932,12 @@ public T accept(ParseTreeVisitor extends T> visitor) {
public final ComparisonOperatorContext comparisonOperator() throws RecognitionException {
ComparisonOperatorContext _localctx = new ComparisonOperatorContext(_ctx, getState());
- enterRule(_localctx, 80, RULE_comparisonOperator);
+ enterRule(_localctx, 82, RULE_comparisonOperator);
int _la;
try {
enterOuterAlt(_localctx, 1);
{
- setState(461);
+ setState(465);
_la = _input.LA(1);
if ( !(((_la) & ~0x3f) == 0 && ((1L << _la) & 1134907106097364992L) != 0) ) {
_errHandler.recoverInline(this);
@@ -3915,13 +3987,13 @@ public T accept(ParseTreeVisitor extends T> visitor) {
public final ExplainCommandContext explainCommand() throws RecognitionException {
ExplainCommandContext _localctx = new ExplainCommandContext(_ctx, getState());
- enterRule(_localctx, 82, RULE_explainCommand);
+ enterRule(_localctx, 84, RULE_explainCommand);
try {
enterOuterAlt(_localctx, 1);
{
- setState(463);
+ setState(467);
match(EXPLAIN);
- setState(464);
+ setState(468);
subqueryExpression();
}
}
@@ -3964,15 +4036,15 @@ public T accept(ParseTreeVisitor extends T> visitor) {
public final SubqueryExpressionContext subqueryExpression() throws RecognitionException {
SubqueryExpressionContext _localctx = new SubqueryExpressionContext(_ctx, getState());
- enterRule(_localctx, 84, RULE_subqueryExpression);
+ enterRule(_localctx, 86, RULE_subqueryExpression);
try {
enterOuterAlt(_localctx, 1);
{
- setState(466);
+ setState(470);
match(OPENING_BRACKET);
- setState(467);
+ setState(471);
query(0);
- setState(468);
+ setState(472);
match(CLOSING_BRACKET);
}
}
@@ -4040,18 +4112,18 @@ public T accept(ParseTreeVisitor extends T> visitor) {
public final ShowCommandContext showCommand() throws RecognitionException {
ShowCommandContext _localctx = new ShowCommandContext(_ctx, getState());
- enterRule(_localctx, 86, RULE_showCommand);
+ enterRule(_localctx, 88, RULE_showCommand);
try {
- setState(474);
+ setState(478);
_errHandler.sync(this);
switch ( getInterpreter().adaptivePredict(_input,46,_ctx) ) {
case 1:
_localctx = new ShowInfoContext(_localctx);
enterOuterAlt(_localctx, 1);
{
- setState(470);
+ setState(474);
match(SHOW);
- setState(471);
+ setState(475);
match(INFO);
}
break;
@@ -4059,9 +4131,9 @@ public final ShowCommandContext showCommand() throws RecognitionException {
_localctx = new ShowFunctionsContext(_localctx);
enterOuterAlt(_localctx, 2);
{
- setState(472);
+ setState(476);
match(SHOW);
- setState(473);
+ setState(477);
match(FUNCTIONS);
}
break;
@@ -4122,51 +4194,51 @@ public T accept(ParseTreeVisitor extends T> visitor) {
public final EnrichCommandContext enrichCommand() throws RecognitionException {
EnrichCommandContext _localctx = new EnrichCommandContext(_ctx, getState());
- enterRule(_localctx, 88, RULE_enrichCommand);
+ enterRule(_localctx, 90, RULE_enrichCommand);
try {
int _alt;
enterOuterAlt(_localctx, 1);
{
- setState(476);
+ setState(480);
match(ENRICH);
- setState(477);
+ setState(481);
((EnrichCommandContext)_localctx).policyName = sourceIdentifier();
- setState(480);
+ setState(484);
_errHandler.sync(this);
switch ( getInterpreter().adaptivePredict(_input,47,_ctx) ) {
case 1:
{
- setState(478);
+ setState(482);
match(ON);
- setState(479);
+ setState(483);
((EnrichCommandContext)_localctx).matchField = sourceIdentifier();
}
break;
}
- setState(491);
+ setState(495);
_errHandler.sync(this);
switch ( getInterpreter().adaptivePredict(_input,49,_ctx) ) {
case 1:
{
- setState(482);
+ setState(486);
match(WITH);
- setState(483);
+ setState(487);
enrichWithClause();
- setState(488);
+ setState(492);
_errHandler.sync(this);
_alt = getInterpreter().adaptivePredict(_input,48,_ctx);
while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) {
if ( _alt==1 ) {
{
{
- setState(484);
+ setState(488);
match(COMMA);
- setState(485);
+ setState(489);
enrichWithClause();
}
}
}
- setState(490);
+ setState(494);
_errHandler.sync(this);
_alt = getInterpreter().adaptivePredict(_input,48,_ctx);
}
@@ -4218,23 +4290,23 @@ public T accept(ParseTreeVisitor extends T> visitor) {
public final EnrichWithClauseContext enrichWithClause() throws RecognitionException {
EnrichWithClauseContext _localctx = new EnrichWithClauseContext(_ctx, getState());
- enterRule(_localctx, 90, RULE_enrichWithClause);
+ enterRule(_localctx, 92, RULE_enrichWithClause);
try {
enterOuterAlt(_localctx, 1);
{
- setState(496);
+ setState(500);
_errHandler.sync(this);
switch ( getInterpreter().adaptivePredict(_input,50,_ctx) ) {
case 1:
{
- setState(493);
+ setState(497);
((EnrichWithClauseContext)_localctx).newName = sourceIdentifier();
- setState(494);
+ setState(498);
match(ASSIGN);
}
break;
}
- setState(498);
+ setState(502);
((EnrichWithClauseContext)_localctx).enrichField = sourceIdentifier();
}
}
@@ -4287,7 +4359,7 @@ private boolean operatorExpression_sempred(OperatorExpressionContext _localctx,
}
public static final String _serializedATN =
- "\u0004\u0001Q\u01f5\u0002\u0000\u0007\u0000\u0002\u0001\u0007\u0001\u0002"+
+ "\u0004\u0001Q\u01f9\u0002\u0000\u0007\u0000\u0002\u0001\u0007\u0001\u0002"+
"\u0002\u0007\u0002\u0002\u0003\u0007\u0003\u0002\u0004\u0007\u0004\u0002"+
"\u0005\u0007\u0005\u0002\u0006\u0007\u0006\u0002\u0007\u0007\u0007\u0002"+
"\b\u0007\b\u0002\t\u0007\t\u0002\n\u0007\n\u0002\u000b\u0007\u000b\u0002"+
@@ -4300,316 +4372,319 @@ private boolean operatorExpression_sempred(OperatorExpressionContext _localctx,
"\u0002\u001f\u0007\u001f\u0002 \u0007 \u0002!\u0007!\u0002\"\u0007\"\u0002"+
"#\u0007#\u0002$\u0007$\u0002%\u0007%\u0002&\u0007&\u0002\'\u0007\'\u0002"+
"(\u0007(\u0002)\u0007)\u0002*\u0007*\u0002+\u0007+\u0002,\u0007,\u0002"+
- "-\u0007-\u0001\u0000\u0001\u0000\u0001\u0000\u0001\u0001\u0001\u0001\u0001"+
- "\u0001\u0001\u0001\u0001\u0001\u0001\u0001\u0005\u0001f\b\u0001\n\u0001"+
- "\f\u0001i\t\u0001\u0001\u0002\u0001\u0002\u0001\u0002\u0001\u0002\u0003"+
- "\u0002o\b\u0002\u0001\u0003\u0001\u0003\u0001\u0003\u0001\u0003\u0001"+
- "\u0003\u0001\u0003\u0001\u0003\u0001\u0003\u0001\u0003\u0001\u0003\u0001"+
- "\u0003\u0001\u0003\u0001\u0003\u0003\u0003~\b\u0003\u0001\u0004\u0001"+
- "\u0004\u0001\u0004\u0001\u0005\u0001\u0005\u0001\u0005\u0001\u0005\u0001"+
- "\u0005\u0001\u0005\u0001\u0005\u0003\u0005\u008a\b\u0005\u0001\u0005\u0001"+
- "\u0005\u0001\u0005\u0001\u0005\u0001\u0005\u0005\u0005\u0091\b\u0005\n"+
- "\u0005\f\u0005\u0094\t\u0005\u0001\u0005\u0001\u0005\u0001\u0005\u0001"+
- "\u0005\u0001\u0005\u0003\u0005\u009b\b\u0005\u0001\u0005\u0001\u0005\u0003"+
- "\u0005\u009f\b\u0005\u0001\u0005\u0001\u0005\u0001\u0005\u0001\u0005\u0001"+
- "\u0005\u0001\u0005\u0005\u0005\u00a7\b\u0005\n\u0005\f\u0005\u00aa\t\u0005"+
- "\u0001\u0006\u0001\u0006\u0003\u0006\u00ae\b\u0006\u0001\u0006\u0001\u0006"+
- "\u0001\u0006\u0001\u0006\u0001\u0006\u0003\u0006\u00b5\b\u0006\u0001\u0006"+
- "\u0001\u0006\u0001\u0006\u0003\u0006\u00ba\b\u0006\u0001\u0007\u0001\u0007"+
- "\u0001\u0007\u0001\u0007\u0001\u0007\u0003\u0007\u00c1\b\u0007\u0001\b"+
- "\u0001\b\u0001\b\u0001\b\u0003\b\u00c7\b\b\u0001\b\u0001\b\u0001\b\u0001"+
- "\b\u0001\b\u0001\b\u0005\b\u00cf\b\b\n\b\f\b\u00d2\t\b\u0001\t\u0001\t"+
- "\u0001\t\u0001\t\u0001\t\u0001\t\u0001\t\u0001\t\u0001\t\u0001\t\u0001"+
- "\t\u0005\t\u00df\b\t\n\t\f\t\u00e2\t\t\u0003\t\u00e4\b\t\u0001\t\u0001"+
- "\t\u0003\t\u00e8\b\t\u0001\n\u0001\n\u0001\n\u0001\u000b\u0001\u000b\u0001"+
- "\u000b\u0005\u000b\u00f0\b\u000b\n\u000b\f\u000b\u00f3\t\u000b\u0001\f"+
- "\u0001\f\u0001\f\u0001\f\u0001\f\u0003\f\u00fa\b\f\u0001\r\u0001\r\u0001"+
- "\r\u0001\r\u0005\r\u0100\b\r\n\r\f\r\u0103\t\r\u0001\r\u0003\r\u0106\b"+
- "\r\u0001\u000e\u0001\u000e\u0001\u000e\u0001\u000e\u0001\u000e\u0005\u000e"+
- "\u010d\b\u000e\n\u000e\f\u000e\u0110\t\u000e\u0001\u000e\u0001\u000e\u0001"+
- "\u000f\u0001\u000f\u0001\u000f\u0001\u0010\u0001\u0010\u0003\u0010\u0119"+
- "\b\u0010\u0001\u0010\u0001\u0010\u0003\u0010\u011d\b\u0010\u0001\u0011"+
- "\u0001\u0011\u0001\u0011\u0001\u0011\u0003\u0011\u0123\b\u0011\u0001\u0012"+
- "\u0001\u0012\u0001\u0012\u0005\u0012\u0128\b\u0012\n\u0012\f\u0012\u012b"+
- "\t\u0012\u0001\u0013\u0001\u0013\u0001\u0014\u0001\u0014\u0001\u0014\u0005"+
- "\u0014\u0132\b\u0014\n\u0014\f\u0014\u0135\t\u0014\u0001\u0015\u0001\u0015"+
- "\u0001\u0016\u0001\u0016\u0001\u0016\u0001\u0016\u0001\u0016\u0001\u0016"+
- "\u0001\u0016\u0001\u0016\u0001\u0016\u0001\u0016\u0001\u0016\u0001\u0016"+
- "\u0001\u0016\u0005\u0016\u0146\b\u0016\n\u0016\f\u0016\u0149\t\u0016\u0001"+
- "\u0016\u0001\u0016\u0001\u0016\u0001\u0016\u0001\u0016\u0001\u0016\u0005"+
- "\u0016\u0151\b\u0016\n\u0016\f\u0016\u0154\t\u0016\u0001\u0016\u0001\u0016"+
- "\u0001\u0016\u0001\u0016\u0001\u0016\u0001\u0016\u0005\u0016\u015c\b\u0016"+
- "\n\u0016\f\u0016\u015f\t\u0016\u0001\u0016\u0001\u0016\u0003\u0016\u0163"+
- "\b\u0016\u0001\u0017\u0001\u0017\u0001\u0017\u0001\u0018\u0001\u0018\u0001"+
- "\u0018\u0001\u0018\u0005\u0018\u016c\b\u0018\n\u0018\f\u0018\u016f\t\u0018"+
- "\u0001\u0019\u0001\u0019\u0003\u0019\u0173\b\u0019\u0001\u0019\u0001\u0019"+
- "\u0003\u0019\u0177\b\u0019\u0001\u001a\u0001\u001a\u0001\u001a\u0001\u001a"+
- "\u0005\u001a\u017d\b\u001a\n\u001a\f\u001a\u0180\t\u001a\u0001\u001a\u0001"+
- "\u001a\u0001\u001a\u0001\u001a\u0005\u001a\u0186\b\u001a\n\u001a\f\u001a"+
- "\u0189\t\u001a\u0003\u001a\u018b\b\u001a\u0001\u001b\u0001\u001b\u0001"+
- "\u001b\u0001\u001b\u0005\u001b\u0191\b\u001b\n\u001b\f\u001b\u0194\t\u001b"+
- "\u0001\u001c\u0001\u001c\u0001\u001c\u0001\u001c\u0005\u001c\u019a\b\u001c"+
- "\n\u001c\f\u001c\u019d\t\u001c\u0001\u001d\u0001\u001d\u0001\u001d\u0001"+
- "\u001d\u0001\u001e\u0001\u001e\u0001\u001e\u0001\u001e\u0003\u001e\u01a7"+
- "\b\u001e\u0001\u001f\u0001\u001f\u0001\u001f\u0001\u001f\u0001 \u0001"+
- " \u0001 \u0001!\u0001!\u0001!\u0005!\u01b3\b!\n!\f!\u01b6\t!\u0001\"\u0001"+
- "\"\u0001\"\u0001\"\u0001#\u0001#\u0001$\u0001$\u0003$\u01c0\b$\u0001%"+
- "\u0003%\u01c3\b%\u0001%\u0001%\u0001&\u0003&\u01c8\b&\u0001&\u0001&\u0001"+
- "\'\u0001\'\u0001(\u0001(\u0001)\u0001)\u0001)\u0001*\u0001*\u0001*\u0001"+
- "*\u0001+\u0001+\u0001+\u0001+\u0003+\u01db\b+\u0001,\u0001,\u0001,\u0001"+
- ",\u0003,\u01e1\b,\u0001,\u0001,\u0001,\u0001,\u0005,\u01e7\b,\n,\f,\u01ea"+
- "\t,\u0003,\u01ec\b,\u0001-\u0001-\u0001-\u0003-\u01f1\b-\u0001-\u0001"+
- "-\u0001-\u0000\u0003\u0002\n\u0010.\u0000\u0002\u0004\u0006\b\n\f\u000e"+
- "\u0010\u0012\u0014\u0016\u0018\u001a\u001c\u001e \"$&(*,.02468:<>@BDF"+
- "HJLNPRTVXZ\u0000\b\u0001\u0000<=\u0001\u0000>@\u0001\u0000LM\u0001\u0000"+
- "CD\u0002\u0000 ##\u0001\u0000&\'\u0002\u0000%%33\u0001\u00006;\u0213"+
- "\u0000\\\u0001\u0000\u0000\u0000\u0002_\u0001\u0000\u0000\u0000\u0004"+
- "n\u0001\u0000\u0000\u0000\u0006}\u0001\u0000\u0000\u0000\b\u007f\u0001"+
- "\u0000\u0000\u0000\n\u009e\u0001\u0000\u0000\u0000\f\u00b9\u0001\u0000"+
- "\u0000\u0000\u000e\u00c0\u0001\u0000\u0000\u0000\u0010\u00c6\u0001\u0000"+
- "\u0000\u0000\u0012\u00e7\u0001\u0000\u0000\u0000\u0014\u00e9\u0001\u0000"+
- "\u0000\u0000\u0016\u00ec\u0001\u0000\u0000\u0000\u0018\u00f9\u0001\u0000"+
- "\u0000\u0000\u001a\u00fb\u0001\u0000\u0000\u0000\u001c\u0107\u0001\u0000"+
- "\u0000\u0000\u001e\u0113\u0001\u0000\u0000\u0000 \u0116\u0001\u0000\u0000"+
- "\u0000\"\u011e\u0001\u0000\u0000\u0000$\u0124\u0001\u0000\u0000\u0000"+
- "&\u012c\u0001\u0000\u0000\u0000(\u012e\u0001\u0000\u0000\u0000*\u0136"+
- "\u0001\u0000\u0000\u0000,\u0162\u0001\u0000\u0000\u0000.\u0164\u0001\u0000"+
- "\u0000\u00000\u0167\u0001\u0000\u0000\u00002\u0170\u0001\u0000\u0000\u0000"+
- "4\u018a\u0001\u0000\u0000\u00006\u018c\u0001\u0000\u0000\u00008\u0195"+
- "\u0001\u0000\u0000\u0000:\u019e\u0001\u0000\u0000\u0000<\u01a2\u0001\u0000"+
- "\u0000\u0000>\u01a8\u0001\u0000\u0000\u0000@\u01ac\u0001\u0000\u0000\u0000"+
- "B\u01af\u0001\u0000\u0000\u0000D\u01b7\u0001\u0000\u0000\u0000F\u01bb"+
- "\u0001\u0000\u0000\u0000H\u01bf\u0001\u0000\u0000\u0000J\u01c2\u0001\u0000"+
- "\u0000\u0000L\u01c7\u0001\u0000\u0000\u0000N\u01cb\u0001\u0000\u0000\u0000"+
- "P\u01cd\u0001\u0000\u0000\u0000R\u01cf\u0001\u0000\u0000\u0000T\u01d2"+
- "\u0001\u0000\u0000\u0000V\u01da\u0001\u0000\u0000\u0000X\u01dc\u0001\u0000"+
- "\u0000\u0000Z\u01f0\u0001\u0000\u0000\u0000\\]\u0003\u0002\u0001\u0000"+
- "]^\u0005\u0000\u0000\u0001^\u0001\u0001\u0000\u0000\u0000_`\u0006\u0001"+
- "\uffff\uffff\u0000`a\u0003\u0004\u0002\u0000ag\u0001\u0000\u0000\u0000"+
- "bc\n\u0001\u0000\u0000cd\u0005\u001a\u0000\u0000df\u0003\u0006\u0003\u0000"+
- "eb\u0001\u0000\u0000\u0000fi\u0001\u0000\u0000\u0000ge\u0001\u0000\u0000"+
- "\u0000gh\u0001\u0000\u0000\u0000h\u0003\u0001\u0000\u0000\u0000ig\u0001"+
- "\u0000\u0000\u0000jo\u0003R)\u0000ko\u0003\u001a\r\u0000lo\u0003\u0014"+
- "\n\u0000mo\u0003V+\u0000nj\u0001\u0000\u0000\u0000nk\u0001\u0000\u0000"+
- "\u0000nl\u0001\u0000\u0000\u0000nm\u0001\u0000\u0000\u0000o\u0005\u0001"+
- "\u0000\u0000\u0000p~\u0003\u001e\u000f\u0000q~\u0003\"\u0011\u0000r~\u0003"+
- ".\u0017\u0000s~\u00034\u001a\u0000t~\u00030\u0018\u0000u~\u0003 \u0010"+
- "\u0000v~\u0003\b\u0004\u0000w~\u00036\u001b\u0000x~\u00038\u001c\u0000"+
- "y~\u0003<\u001e\u0000z~\u0003>\u001f\u0000{~\u0003X,\u0000|~\u0003@ \u0000"+
- "}p\u0001\u0000\u0000\u0000}q\u0001\u0000\u0000\u0000}r\u0001\u0000\u0000"+
- "\u0000}s\u0001\u0000\u0000\u0000}t\u0001\u0000\u0000\u0000}u\u0001\u0000"+
- "\u0000\u0000}v\u0001\u0000\u0000\u0000}w\u0001\u0000\u0000\u0000}x\u0001"+
- "\u0000\u0000\u0000}y\u0001\u0000\u0000\u0000}z\u0001\u0000\u0000\u0000"+
- "}{\u0001\u0000\u0000\u0000}|\u0001\u0000\u0000\u0000~\u0007\u0001\u0000"+
- "\u0000\u0000\u007f\u0080\u0005\u0012\u0000\u0000\u0080\u0081\u0003\n\u0005"+
- "\u0000\u0081\t\u0001\u0000\u0000\u0000\u0082\u0083\u0006\u0005\uffff\uffff"+
- "\u0000\u0083\u0084\u0005,\u0000\u0000\u0084\u009f\u0003\n\u0005\u0007"+
- "\u0085\u009f\u0003\u000e\u0007\u0000\u0086\u009f\u0003\f\u0006\u0000\u0087"+
- "\u0089\u0003\u000e\u0007\u0000\u0088\u008a\u0005,\u0000\u0000\u0089\u0088"+
- "\u0001\u0000\u0000\u0000\u0089\u008a\u0001\u0000\u0000\u0000\u008a\u008b"+
- "\u0001\u0000\u0000\u0000\u008b\u008c\u0005)\u0000\u0000\u008c\u008d\u0005"+
- "(\u0000\u0000\u008d\u0092\u0003\u000e\u0007\u0000\u008e\u008f\u0005\""+
- "\u0000\u0000\u008f\u0091\u0003\u000e\u0007\u0000\u0090\u008e\u0001\u0000"+
- "\u0000\u0000\u0091\u0094\u0001\u0000\u0000\u0000\u0092\u0090\u0001\u0000"+
- "\u0000\u0000\u0092\u0093\u0001\u0000\u0000\u0000\u0093\u0095\u0001\u0000"+
- "\u0000\u0000\u0094\u0092\u0001\u0000\u0000\u0000\u0095\u0096\u00052\u0000"+
- "\u0000\u0096\u009f\u0001\u0000\u0000\u0000\u0097\u0098\u0003\u000e\u0007"+
- "\u0000\u0098\u009a\u0005*\u0000\u0000\u0099\u009b\u0005,\u0000\u0000\u009a"+
- "\u0099\u0001\u0000\u0000\u0000\u009a\u009b\u0001\u0000\u0000\u0000\u009b"+
- "\u009c\u0001\u0000\u0000\u0000\u009c\u009d\u0005-\u0000\u0000\u009d\u009f"+
- "\u0001\u0000\u0000\u0000\u009e\u0082\u0001\u0000\u0000\u0000\u009e\u0085"+
- "\u0001\u0000\u0000\u0000\u009e\u0086\u0001\u0000\u0000\u0000\u009e\u0087"+
- "\u0001\u0000\u0000\u0000\u009e\u0097\u0001\u0000\u0000\u0000\u009f\u00a8"+
- "\u0001\u0000\u0000\u0000\u00a0\u00a1\n\u0004\u0000\u0000\u00a1\u00a2\u0005"+
- "\u001f\u0000\u0000\u00a2\u00a7\u0003\n\u0005\u0005\u00a3\u00a4\n\u0003"+
- "\u0000\u0000\u00a4\u00a5\u0005/\u0000\u0000\u00a5\u00a7\u0003\n\u0005"+
- "\u0004\u00a6\u00a0\u0001\u0000\u0000\u0000\u00a6\u00a3\u0001\u0000\u0000"+
- "\u0000\u00a7\u00aa\u0001\u0000\u0000\u0000\u00a8\u00a6\u0001\u0000\u0000"+
- "\u0000\u00a8\u00a9\u0001\u0000\u0000\u0000\u00a9\u000b\u0001\u0000\u0000"+
- "\u0000\u00aa\u00a8\u0001\u0000\u0000\u0000\u00ab\u00ad\u0003\u000e\u0007"+
- "\u0000\u00ac\u00ae\u0005,\u0000\u0000\u00ad\u00ac\u0001\u0000\u0000\u0000"+
- "\u00ad\u00ae\u0001\u0000\u0000\u0000\u00ae\u00af\u0001\u0000\u0000\u0000"+
- "\u00af\u00b0\u0005+\u0000\u0000\u00b0\u00b1\u0003N\'\u0000\u00b1\u00ba"+
- "\u0001\u0000\u0000\u0000\u00b2\u00b4\u0003\u000e\u0007\u0000\u00b3\u00b5"+
- "\u0005,\u0000\u0000\u00b4\u00b3\u0001\u0000\u0000\u0000\u00b4\u00b5\u0001"+
- "\u0000\u0000\u0000\u00b5\u00b6\u0001\u0000\u0000\u0000\u00b6\u00b7\u0005"+
- "1\u0000\u0000\u00b7\u00b8\u0003N\'\u0000\u00b8\u00ba\u0001\u0000\u0000"+
- "\u0000\u00b9\u00ab\u0001\u0000\u0000\u0000\u00b9\u00b2\u0001\u0000\u0000"+
- "\u0000\u00ba\r\u0001\u0000\u0000\u0000\u00bb\u00c1\u0003\u0010\b\u0000"+
- "\u00bc\u00bd\u0003\u0010\b\u0000\u00bd\u00be\u0003P(\u0000\u00be\u00bf"+
- "\u0003\u0010\b\u0000\u00bf\u00c1\u0001\u0000\u0000\u0000\u00c0\u00bb\u0001"+
- "\u0000\u0000\u0000\u00c0\u00bc\u0001\u0000\u0000\u0000\u00c1\u000f\u0001"+
- "\u0000\u0000\u0000\u00c2\u00c3\u0006\b\uffff\uffff\u0000\u00c3\u00c7\u0003"+
- "\u0012\t\u0000\u00c4\u00c5\u0007\u0000\u0000\u0000\u00c5\u00c7\u0003\u0010"+
- "\b\u0003\u00c6\u00c2\u0001\u0000\u0000\u0000\u00c6\u00c4\u0001\u0000\u0000"+
- "\u0000\u00c7\u00d0\u0001\u0000\u0000\u0000\u00c8\u00c9\n\u0002\u0000\u0000"+
- "\u00c9\u00ca\u0007\u0001\u0000\u0000\u00ca\u00cf\u0003\u0010\b\u0003\u00cb"+
- "\u00cc\n\u0001\u0000\u0000\u00cc\u00cd\u0007\u0000\u0000\u0000\u00cd\u00cf"+
- "\u0003\u0010\b\u0002\u00ce\u00c8\u0001\u0000\u0000\u0000\u00ce\u00cb\u0001"+
- "\u0000\u0000\u0000\u00cf\u00d2\u0001\u0000\u0000\u0000\u00d0\u00ce\u0001"+
- "\u0000\u0000\u0000\u00d0\u00d1\u0001\u0000\u0000\u0000\u00d1\u0011\u0001"+
- "\u0000\u0000\u0000\u00d2\u00d0\u0001\u0000\u0000\u0000\u00d3\u00e8\u0003"+
- ",\u0016\u0000\u00d4\u00e8\u0003(\u0014\u0000\u00d5\u00d6\u0005(\u0000"+
- "\u0000\u00d6\u00d7\u0003\n\u0005\u0000\u00d7\u00d8\u00052\u0000\u0000"+
- "\u00d8\u00e8\u0001\u0000\u0000\u0000\u00d9\u00da\u0003*\u0015\u0000\u00da"+
- "\u00e3\u0005(\u0000\u0000\u00db\u00e0\u0003\n\u0005\u0000\u00dc\u00dd"+
- "\u0005\"\u0000\u0000\u00dd\u00df\u0003\n\u0005\u0000\u00de\u00dc\u0001"+
- "\u0000\u0000\u0000\u00df\u00e2\u0001\u0000\u0000\u0000\u00e0\u00de\u0001"+
- "\u0000\u0000\u0000\u00e0\u00e1\u0001\u0000\u0000\u0000\u00e1\u00e4\u0001"+
- "\u0000\u0000\u0000\u00e2\u00e0\u0001\u0000\u0000\u0000\u00e3\u00db\u0001"+
- "\u0000\u0000\u0000\u00e3\u00e4\u0001\u0000\u0000\u0000\u00e4\u00e5\u0001"+
- "\u0000\u0000\u0000\u00e5\u00e6\u00052\u0000\u0000\u00e6\u00e8\u0001\u0000"+
- "\u0000\u0000\u00e7\u00d3\u0001\u0000\u0000\u0000\u00e7\u00d4\u0001\u0000"+
- "\u0000\u0000\u00e7\u00d5\u0001\u0000\u0000\u0000\u00e7\u00d9\u0001\u0000"+
- "\u0000\u0000\u00e8\u0013\u0001\u0000\u0000\u0000\u00e9\u00ea\u0005\u000e"+
- "\u0000\u0000\u00ea\u00eb\u0003\u0016\u000b\u0000\u00eb\u0015\u0001\u0000"+
- "\u0000\u0000\u00ec\u00f1\u0003\u0018\f\u0000\u00ed\u00ee\u0005\"\u0000"+
- "\u0000\u00ee\u00f0\u0003\u0018\f\u0000\u00ef\u00ed\u0001\u0000\u0000\u0000"+
- "\u00f0\u00f3\u0001\u0000\u0000\u0000\u00f1\u00ef\u0001\u0000\u0000\u0000"+
- "\u00f1\u00f2\u0001\u0000\u0000\u0000\u00f2\u0017\u0001\u0000\u0000\u0000"+
- "\u00f3\u00f1\u0001\u0000\u0000\u0000\u00f4\u00fa\u0003\n\u0005\u0000\u00f5"+
- "\u00f6\u0003(\u0014\u0000\u00f6\u00f7\u0005!\u0000\u0000\u00f7\u00f8\u0003"+
- "\n\u0005\u0000\u00f8\u00fa\u0001\u0000\u0000\u0000\u00f9\u00f4\u0001\u0000"+
- "\u0000\u0000\u00f9\u00f5\u0001\u0000\u0000\u0000\u00fa\u0019\u0001\u0000"+
- "\u0000\u0000\u00fb\u00fc\u0005\u0006\u0000\u0000\u00fc\u0101\u0003&\u0013"+
- "\u0000\u00fd\u00fe\u0005\"\u0000\u0000\u00fe\u0100\u0003&\u0013\u0000"+
- "\u00ff\u00fd\u0001\u0000\u0000\u0000\u0100\u0103\u0001\u0000\u0000\u0000"+
- "\u0101\u00ff\u0001\u0000\u0000\u0000\u0101\u0102\u0001\u0000\u0000\u0000"+
- "\u0102\u0105\u0001\u0000\u0000\u0000\u0103\u0101\u0001\u0000\u0000\u0000"+
- "\u0104\u0106\u0003\u001c\u000e\u0000\u0105\u0104\u0001\u0000\u0000\u0000"+
- "\u0105\u0106\u0001\u0000\u0000\u0000\u0106\u001b\u0001\u0000\u0000\u0000"+
- "\u0107\u0108\u0005A\u0000\u0000\u0108\u0109\u0005I\u0000\u0000\u0109\u010e"+
- "\u0003&\u0013\u0000\u010a\u010b\u0005\"\u0000\u0000\u010b\u010d\u0003"+
- "&\u0013\u0000\u010c\u010a\u0001\u0000\u0000\u0000\u010d\u0110\u0001\u0000"+
- "\u0000\u0000\u010e\u010c\u0001\u0000\u0000\u0000\u010e\u010f\u0001\u0000"+
- "\u0000\u0000\u010f\u0111\u0001\u0000\u0000\u0000\u0110\u010e\u0001\u0000"+
- "\u0000\u0000\u0111\u0112\u0005B\u0000\u0000\u0112\u001d\u0001\u0000\u0000"+
- "\u0000\u0113\u0114\u0005\u0004\u0000\u0000\u0114\u0115\u0003\u0016\u000b"+
- "\u0000\u0115\u001f\u0001\u0000\u0000\u0000\u0116\u0118\u0005\u0011\u0000"+
- "\u0000\u0117\u0119\u0003\u0016\u000b\u0000\u0118\u0117\u0001\u0000\u0000"+
- "\u0000\u0118\u0119\u0001\u0000\u0000\u0000\u0119\u011c\u0001\u0000\u0000"+
- "\u0000\u011a\u011b\u0005\u001e\u0000\u0000\u011b\u011d\u0003$\u0012\u0000"+
- "\u011c\u011a\u0001\u0000\u0000\u0000\u011c\u011d\u0001\u0000\u0000\u0000"+
- "\u011d!\u0001\u0000\u0000\u0000\u011e\u011f\u0005\b\u0000\u0000\u011f"+
- "\u0122\u0003\u0016\u000b\u0000\u0120\u0121\u0005\u001e\u0000\u0000\u0121"+
- "\u0123\u0003$\u0012\u0000\u0122\u0120\u0001\u0000\u0000\u0000\u0122\u0123"+
- "\u0001\u0000\u0000\u0000\u0123#\u0001\u0000\u0000\u0000\u0124\u0129\u0003"+
- "(\u0014\u0000\u0125\u0126\u0005\"\u0000\u0000\u0126\u0128\u0003(\u0014"+
- "\u0000\u0127\u0125\u0001\u0000\u0000\u0000\u0128\u012b\u0001\u0000\u0000"+
- "\u0000\u0129\u0127\u0001\u0000\u0000\u0000\u0129\u012a\u0001\u0000\u0000"+
- "\u0000\u012a%\u0001\u0000\u0000\u0000\u012b\u0129\u0001\u0000\u0000\u0000"+
- "\u012c\u012d\u0007\u0002\u0000\u0000\u012d\'\u0001\u0000\u0000\u0000\u012e"+
- "\u0133\u0003*\u0015\u0000\u012f\u0130\u0005$\u0000\u0000\u0130\u0132\u0003"+
- "*\u0015\u0000\u0131\u012f\u0001\u0000\u0000\u0000\u0132\u0135\u0001\u0000"+
- "\u0000\u0000\u0133\u0131\u0001\u0000\u0000\u0000\u0133\u0134\u0001\u0000"+
- "\u0000\u0000\u0134)\u0001\u0000\u0000\u0000\u0135\u0133\u0001\u0000\u0000"+
- "\u0000\u0136\u0137\u0007\u0003\u0000\u0000\u0137+\u0001\u0000\u0000\u0000"+
- "\u0138\u0163\u0005-\u0000\u0000\u0139\u013a\u0003L&\u0000\u013a\u013b"+
- "\u0005C\u0000\u0000\u013b\u0163\u0001\u0000\u0000\u0000\u013c\u0163\u0003"+
- "J%\u0000\u013d\u0163\u0003L&\u0000\u013e\u0163\u0003F#\u0000\u013f\u0163"+
- "\u00050\u0000\u0000\u0140\u0163\u0003N\'\u0000\u0141\u0142\u0005A\u0000"+
- "\u0000\u0142\u0147\u0003H$\u0000\u0143\u0144\u0005\"\u0000\u0000\u0144"+
- "\u0146\u0003H$\u0000\u0145\u0143\u0001\u0000\u0000\u0000\u0146\u0149\u0001"+
- "\u0000\u0000\u0000\u0147\u0145\u0001\u0000\u0000\u0000\u0147\u0148\u0001"+
- "\u0000\u0000\u0000\u0148\u014a\u0001\u0000\u0000\u0000\u0149\u0147\u0001"+
- "\u0000\u0000\u0000\u014a\u014b\u0005B\u0000\u0000\u014b\u0163\u0001\u0000"+
- "\u0000\u0000\u014c\u014d\u0005A\u0000\u0000\u014d\u0152\u0003F#\u0000"+
- "\u014e\u014f\u0005\"\u0000\u0000\u014f\u0151\u0003F#\u0000\u0150\u014e"+
- "\u0001\u0000\u0000\u0000\u0151\u0154\u0001\u0000\u0000\u0000\u0152\u0150"+
- "\u0001\u0000\u0000\u0000\u0152\u0153\u0001\u0000\u0000\u0000\u0153\u0155"+
- "\u0001\u0000\u0000\u0000\u0154\u0152\u0001\u0000\u0000\u0000\u0155\u0156"+
- "\u0005B\u0000\u0000\u0156\u0163\u0001\u0000\u0000\u0000\u0157\u0158\u0005"+
- "A\u0000\u0000\u0158\u015d\u0003N\'\u0000\u0159\u015a\u0005\"\u0000\u0000"+
- "\u015a\u015c\u0003N\'\u0000\u015b\u0159\u0001\u0000\u0000\u0000\u015c"+
- "\u015f\u0001\u0000\u0000\u0000\u015d\u015b\u0001\u0000\u0000\u0000\u015d"+
- "\u015e\u0001\u0000\u0000\u0000\u015e\u0160\u0001\u0000\u0000\u0000\u015f"+
- "\u015d\u0001\u0000\u0000\u0000\u0160\u0161\u0005B\u0000\u0000\u0161\u0163"+
- "\u0001\u0000\u0000\u0000\u0162\u0138\u0001\u0000\u0000\u0000\u0162\u0139"+
- "\u0001\u0000\u0000\u0000\u0162\u013c\u0001\u0000\u0000\u0000\u0162\u013d"+
- "\u0001\u0000\u0000\u0000\u0162\u013e\u0001\u0000\u0000\u0000\u0162\u013f"+
- "\u0001\u0000\u0000\u0000\u0162\u0140\u0001\u0000\u0000\u0000\u0162\u0141"+
- "\u0001\u0000\u0000\u0000\u0162\u014c\u0001\u0000\u0000\u0000\u0162\u0157"+
- "\u0001\u0000\u0000\u0000\u0163-\u0001\u0000\u0000\u0000\u0164\u0165\u0005"+
- "\n\u0000\u0000\u0165\u0166\u0005\u001c\u0000\u0000\u0166/\u0001\u0000"+
- "\u0000\u0000\u0167\u0168\u0005\u0010\u0000\u0000\u0168\u016d\u00032\u0019"+
- "\u0000\u0169\u016a\u0005\"\u0000\u0000\u016a\u016c\u00032\u0019\u0000"+
- "\u016b\u0169\u0001\u0000\u0000\u0000\u016c\u016f\u0001\u0000\u0000\u0000"+
- "\u016d\u016b\u0001\u0000\u0000\u0000\u016d\u016e\u0001\u0000\u0000\u0000"+
- "\u016e1\u0001\u0000\u0000\u0000\u016f\u016d\u0001\u0000\u0000\u0000\u0170"+
- "\u0172\u0003\n\u0005\u0000\u0171\u0173\u0007\u0004\u0000\u0000\u0172\u0171"+
- "\u0001\u0000\u0000\u0000\u0172\u0173\u0001\u0000\u0000\u0000\u0173\u0176"+
- "\u0001\u0000\u0000\u0000\u0174\u0175\u0005.\u0000\u0000\u0175\u0177\u0007"+
- "\u0005\u0000\u0000\u0176\u0174\u0001\u0000\u0000\u0000\u0176\u0177\u0001"+
- "\u0000\u0000\u0000\u01773\u0001\u0000\u0000\u0000\u0178\u0179\u0005\t"+
- "\u0000\u0000\u0179\u017e\u0003&\u0013\u0000\u017a\u017b\u0005\"\u0000"+
- "\u0000\u017b\u017d\u0003&\u0013\u0000\u017c\u017a\u0001\u0000\u0000\u0000"+
- "\u017d\u0180\u0001\u0000\u0000\u0000\u017e\u017c\u0001\u0000\u0000\u0000"+
- "\u017e\u017f\u0001\u0000\u0000\u0000\u017f\u018b\u0001\u0000\u0000\u0000"+
- "\u0180\u017e\u0001\u0000\u0000\u0000\u0181\u0182\u0005\f\u0000\u0000\u0182"+
- "\u0187\u0003&\u0013\u0000\u0183\u0184\u0005\"\u0000\u0000\u0184\u0186"+
- "\u0003&\u0013\u0000\u0185\u0183\u0001\u0000\u0000\u0000\u0186\u0189\u0001"+
- "\u0000\u0000\u0000\u0187\u0185\u0001\u0000\u0000\u0000\u0187\u0188\u0001"+
- "\u0000\u0000\u0000\u0188\u018b\u0001\u0000\u0000\u0000\u0189\u0187\u0001"+
- "\u0000\u0000\u0000\u018a\u0178\u0001\u0000\u0000\u0000\u018a\u0181\u0001"+
- "\u0000\u0000\u0000\u018b5\u0001\u0000\u0000\u0000\u018c\u018d\u0005\u0002"+
- "\u0000\u0000\u018d\u0192\u0003&\u0013\u0000\u018e\u018f\u0005\"\u0000"+
- "\u0000\u018f\u0191\u0003&\u0013\u0000\u0190\u018e\u0001\u0000\u0000\u0000"+
- "\u0191\u0194\u0001\u0000\u0000\u0000\u0192\u0190\u0001\u0000\u0000\u0000"+
- "\u0192\u0193\u0001\u0000\u0000\u0000\u01937\u0001\u0000\u0000\u0000\u0194"+
- "\u0192\u0001\u0000\u0000\u0000\u0195\u0196\u0005\r\u0000\u0000\u0196\u019b"+
- "\u0003:\u001d\u0000\u0197\u0198\u0005\"\u0000\u0000\u0198\u019a\u0003"+
- ":\u001d\u0000\u0199\u0197\u0001\u0000\u0000\u0000\u019a\u019d\u0001\u0000"+
- "\u0000\u0000\u019b\u0199\u0001\u0000\u0000\u0000\u019b\u019c\u0001\u0000"+
- "\u0000\u0000\u019c9\u0001\u0000\u0000\u0000\u019d\u019b\u0001\u0000\u0000"+
- "\u0000\u019e\u019f\u0003&\u0013\u0000\u019f\u01a0\u0005H\u0000\u0000\u01a0"+
- "\u01a1\u0003&\u0013\u0000\u01a1;\u0001\u0000\u0000\u0000\u01a2\u01a3\u0005"+
- "\u0001\u0000\u0000\u01a3\u01a4\u0003\u0012\t\u0000\u01a4\u01a6\u0003N"+
- "\'\u0000\u01a5\u01a7\u0003B!\u0000\u01a6\u01a5\u0001\u0000\u0000\u0000"+
- "\u01a6\u01a7\u0001\u0000\u0000\u0000\u01a7=\u0001\u0000\u0000\u0000\u01a8"+
- "\u01a9\u0005\u0007\u0000\u0000\u01a9\u01aa\u0003\u0012\t\u0000\u01aa\u01ab"+
- "\u0003N\'\u0000\u01ab?\u0001\u0000\u0000\u0000\u01ac\u01ad\u0005\u000b"+
- "\u0000\u0000\u01ad\u01ae\u0003&\u0013\u0000\u01aeA\u0001\u0000\u0000\u0000"+
- "\u01af\u01b4\u0003D\"\u0000\u01b0\u01b1\u0005\"\u0000\u0000\u01b1\u01b3"+
- "\u0003D\"\u0000\u01b2\u01b0\u0001\u0000\u0000\u0000\u01b3\u01b6\u0001"+
- "\u0000\u0000\u0000\u01b4\u01b2\u0001\u0000\u0000\u0000\u01b4\u01b5\u0001"+
- "\u0000\u0000\u0000\u01b5C\u0001\u0000\u0000\u0000\u01b6\u01b4\u0001\u0000"+
- "\u0000\u0000\u01b7\u01b8\u0003*\u0015\u0000\u01b8\u01b9\u0005!\u0000\u0000"+
- "\u01b9\u01ba\u0003,\u0016\u0000\u01baE\u0001\u0000\u0000\u0000\u01bb\u01bc"+
- "\u0007\u0006\u0000\u0000\u01bcG\u0001\u0000\u0000\u0000\u01bd\u01c0\u0003"+
- "J%\u0000\u01be\u01c0\u0003L&\u0000\u01bf\u01bd\u0001\u0000\u0000\u0000"+
- "\u01bf\u01be\u0001\u0000\u0000\u0000\u01c0I\u0001\u0000\u0000\u0000\u01c1"+
- "\u01c3\u0007\u0000\u0000\u0000\u01c2\u01c1\u0001\u0000\u0000\u0000\u01c2"+
- "\u01c3\u0001\u0000\u0000\u0000\u01c3\u01c4\u0001\u0000\u0000\u0000\u01c4"+
- "\u01c5\u0005\u001d\u0000\u0000\u01c5K\u0001\u0000\u0000\u0000\u01c6\u01c8"+
- "\u0007\u0000\u0000\u0000\u01c7\u01c6\u0001\u0000\u0000\u0000\u01c7\u01c8"+
- "\u0001\u0000\u0000\u0000\u01c8\u01c9\u0001\u0000\u0000\u0000\u01c9\u01ca"+
- "\u0005\u001c\u0000\u0000\u01caM\u0001\u0000\u0000\u0000\u01cb\u01cc\u0005"+
- "\u001b\u0000\u0000\u01ccO\u0001\u0000\u0000\u0000\u01cd\u01ce\u0007\u0007"+
- "\u0000\u0000\u01ceQ\u0001\u0000\u0000\u0000\u01cf\u01d0\u0005\u0005\u0000"+
- "\u0000\u01d0\u01d1\u0003T*\u0000\u01d1S\u0001\u0000\u0000\u0000\u01d2"+
- "\u01d3\u0005A\u0000\u0000\u01d3\u01d4\u0003\u0002\u0001\u0000\u01d4\u01d5"+
- "\u0005B\u0000\u0000\u01d5U\u0001\u0000\u0000\u0000\u01d6\u01d7\u0005\u000f"+
- "\u0000\u0000\u01d7\u01db\u00054\u0000\u0000\u01d8\u01d9\u0005\u000f\u0000"+
- "\u0000\u01d9\u01db\u00055\u0000\u0000\u01da\u01d6\u0001\u0000\u0000\u0000"+
- "\u01da\u01d8\u0001\u0000\u0000\u0000\u01dbW\u0001\u0000\u0000\u0000\u01dc"+
- "\u01dd\u0005\u0003\u0000\u0000\u01dd\u01e0\u0003&\u0013\u0000\u01de\u01df"+
- "\u0005J\u0000\u0000\u01df\u01e1\u0003&\u0013\u0000\u01e0\u01de\u0001\u0000"+
- "\u0000\u0000\u01e0\u01e1\u0001\u0000\u0000\u0000\u01e1\u01eb\u0001\u0000"+
- "\u0000\u0000\u01e2\u01e3\u0005K\u0000\u0000\u01e3\u01e8\u0003Z-\u0000"+
- "\u01e4\u01e5\u0005\"\u0000\u0000\u01e5\u01e7\u0003Z-\u0000\u01e6\u01e4"+
- "\u0001\u0000\u0000\u0000\u01e7\u01ea\u0001\u0000\u0000\u0000\u01e8\u01e6"+
- "\u0001\u0000\u0000\u0000\u01e8\u01e9\u0001\u0000\u0000\u0000\u01e9\u01ec"+
- "\u0001\u0000\u0000\u0000\u01ea\u01e8\u0001\u0000\u0000\u0000\u01eb\u01e2"+
- "\u0001\u0000\u0000\u0000\u01eb\u01ec\u0001\u0000\u0000\u0000\u01ecY\u0001"+
- "\u0000\u0000\u0000\u01ed\u01ee\u0003&\u0013\u0000\u01ee\u01ef\u0005!\u0000"+
- "\u0000\u01ef\u01f1\u0001\u0000\u0000\u0000\u01f0\u01ed\u0001\u0000\u0000"+
- "\u0000\u01f0\u01f1\u0001\u0000\u0000\u0000\u01f1\u01f2\u0001\u0000\u0000"+
- "\u0000\u01f2\u01f3\u0003&\u0013\u0000\u01f3[\u0001\u0000\u0000\u00003"+
- "gn}\u0089\u0092\u009a\u009e\u00a6\u00a8\u00ad\u00b4\u00b9\u00c0\u00c6"+
- "\u00ce\u00d0\u00e0\u00e3\u00e7\u00f1\u00f9\u0101\u0105\u010e\u0118\u011c"+
- "\u0122\u0129\u0133\u0147\u0152\u015d\u0162\u016d\u0172\u0176\u017e\u0187"+
- "\u018a\u0192\u019b\u01a6\u01b4\u01bf\u01c2\u01c7\u01da\u01e0\u01e8\u01eb"+
- "\u01f0";
+ "-\u0007-\u0002.\u0007.\u0001\u0000\u0001\u0000\u0001\u0000\u0001\u0001"+
+ "\u0001\u0001\u0001\u0001\u0001\u0001\u0001\u0001\u0001\u0001\u0005\u0001"+
+ "h\b\u0001\n\u0001\f\u0001k\t\u0001\u0001\u0002\u0001\u0002\u0001\u0002"+
+ "\u0001\u0002\u0003\u0002q\b\u0002\u0001\u0003\u0001\u0003\u0001\u0003"+
+ "\u0001\u0003\u0001\u0003\u0001\u0003\u0001\u0003\u0001\u0003\u0001\u0003"+
+ "\u0001\u0003\u0001\u0003\u0001\u0003\u0001\u0003\u0003\u0003\u0080\b\u0003"+
+ "\u0001\u0004\u0001\u0004\u0001\u0004\u0001\u0005\u0001\u0005\u0001\u0005"+
+ "\u0001\u0005\u0001\u0005\u0001\u0005\u0001\u0005\u0003\u0005\u008c\b\u0005"+
+ "\u0001\u0005\u0001\u0005\u0001\u0005\u0001\u0005\u0001\u0005\u0005\u0005"+
+ "\u0093\b\u0005\n\u0005\f\u0005\u0096\t\u0005\u0001\u0005\u0001\u0005\u0001"+
+ "\u0005\u0001\u0005\u0001\u0005\u0003\u0005\u009d\b\u0005\u0001\u0005\u0001"+
+ "\u0005\u0003\u0005\u00a1\b\u0005\u0001\u0005\u0001\u0005\u0001\u0005\u0001"+
+ "\u0005\u0001\u0005\u0001\u0005\u0005\u0005\u00a9\b\u0005\n\u0005\f\u0005"+
+ "\u00ac\t\u0005\u0001\u0006\u0001\u0006\u0003\u0006\u00b0\b\u0006\u0001"+
+ "\u0006\u0001\u0006\u0001\u0006\u0001\u0006\u0001\u0006\u0003\u0006\u00b7"+
+ "\b\u0006\u0001\u0006\u0001\u0006\u0001\u0006\u0003\u0006\u00bc\b\u0006"+
+ "\u0001\u0007\u0001\u0007\u0001\u0007\u0001\u0007\u0001\u0007\u0003\u0007"+
+ "\u00c3\b\u0007\u0001\b\u0001\b\u0001\b\u0001\b\u0003\b\u00c9\b\b\u0001"+
+ "\b\u0001\b\u0001\b\u0001\b\u0001\b\u0001\b\u0005\b\u00d1\b\b\n\b\f\b\u00d4"+
+ "\t\b\u0001\t\u0001\t\u0001\t\u0001\t\u0001\t\u0001\t\u0001\t\u0003\t\u00dd"+
+ "\b\t\u0001\n\u0001\n\u0001\n\u0001\n\u0001\n\u0001\n\u0005\n\u00e5\b\n"+
+ "\n\n\f\n\u00e8\t\n\u0003\n\u00ea\b\n\u0001\n\u0001\n\u0001\u000b\u0001"+
+ "\u000b\u0001\u000b\u0001\f\u0001\f\u0001\f\u0005\f\u00f4\b\f\n\f\f\f\u00f7"+
+ "\t\f\u0001\r\u0001\r\u0001\r\u0001\r\u0001\r\u0003\r\u00fe\b\r\u0001\u000e"+
+ "\u0001\u000e\u0001\u000e\u0001\u000e\u0005\u000e\u0104\b\u000e\n\u000e"+
+ "\f\u000e\u0107\t\u000e\u0001\u000e\u0003\u000e\u010a\b\u000e\u0001\u000f"+
+ "\u0001\u000f\u0001\u000f\u0001\u000f\u0001\u000f\u0005\u000f\u0111\b\u000f"+
+ "\n\u000f\f\u000f\u0114\t\u000f\u0001\u000f\u0001\u000f\u0001\u0010\u0001"+
+ "\u0010\u0001\u0010\u0001\u0011\u0001\u0011\u0003\u0011\u011d\b\u0011\u0001"+
+ "\u0011\u0001\u0011\u0003\u0011\u0121\b\u0011\u0001\u0012\u0001\u0012\u0001"+
+ "\u0012\u0001\u0012\u0003\u0012\u0127\b\u0012\u0001\u0013\u0001\u0013\u0001"+
+ "\u0013\u0005\u0013\u012c\b\u0013\n\u0013\f\u0013\u012f\t\u0013\u0001\u0014"+
+ "\u0001\u0014\u0001\u0015\u0001\u0015\u0001\u0015\u0005\u0015\u0136\b\u0015"+
+ "\n\u0015\f\u0015\u0139\t\u0015\u0001\u0016\u0001\u0016\u0001\u0017\u0001"+
+ "\u0017\u0001\u0017\u0001\u0017\u0001\u0017\u0001\u0017\u0001\u0017\u0001"+
+ "\u0017\u0001\u0017\u0001\u0017\u0001\u0017\u0001\u0017\u0001\u0017\u0005"+
+ "\u0017\u014a\b\u0017\n\u0017\f\u0017\u014d\t\u0017\u0001\u0017\u0001\u0017"+
+ "\u0001\u0017\u0001\u0017\u0001\u0017\u0001\u0017\u0005\u0017\u0155\b\u0017"+
+ "\n\u0017\f\u0017\u0158\t\u0017\u0001\u0017\u0001\u0017\u0001\u0017\u0001"+
+ "\u0017\u0001\u0017\u0001\u0017\u0005\u0017\u0160\b\u0017\n\u0017\f\u0017"+
+ "\u0163\t\u0017\u0001\u0017\u0001\u0017\u0003\u0017\u0167\b\u0017\u0001"+
+ "\u0018\u0001\u0018\u0001\u0018\u0001\u0019\u0001\u0019\u0001\u0019\u0001"+
+ "\u0019\u0005\u0019\u0170\b\u0019\n\u0019\f\u0019\u0173\t\u0019\u0001\u001a"+
+ "\u0001\u001a\u0003\u001a\u0177\b\u001a\u0001\u001a\u0001\u001a\u0003\u001a"+
+ "\u017b\b\u001a\u0001\u001b\u0001\u001b\u0001\u001b\u0001\u001b\u0005\u001b"+
+ "\u0181\b\u001b\n\u001b\f\u001b\u0184\t\u001b\u0001\u001b\u0001\u001b\u0001"+
+ "\u001b\u0001\u001b\u0005\u001b\u018a\b\u001b\n\u001b\f\u001b\u018d\t\u001b"+
+ "\u0003\u001b\u018f\b\u001b\u0001\u001c\u0001\u001c\u0001\u001c\u0001\u001c"+
+ "\u0005\u001c\u0195\b\u001c\n\u001c\f\u001c\u0198\t\u001c\u0001\u001d\u0001"+
+ "\u001d\u0001\u001d\u0001\u001d\u0005\u001d\u019e\b\u001d\n\u001d\f\u001d"+
+ "\u01a1\t\u001d\u0001\u001e\u0001\u001e\u0001\u001e\u0001\u001e\u0001\u001f"+
+ "\u0001\u001f\u0001\u001f\u0001\u001f\u0003\u001f\u01ab\b\u001f\u0001 "+
+ "\u0001 \u0001 \u0001 \u0001!\u0001!\u0001!\u0001\"\u0001\"\u0001\"\u0005"+
+ "\"\u01b7\b\"\n\"\f\"\u01ba\t\"\u0001#\u0001#\u0001#\u0001#\u0001$\u0001"+
+ "$\u0001%\u0001%\u0003%\u01c4\b%\u0001&\u0003&\u01c7\b&\u0001&\u0001&\u0001"+
+ "\'\u0003\'\u01cc\b\'\u0001\'\u0001\'\u0001(\u0001(\u0001)\u0001)\u0001"+
+ "*\u0001*\u0001*\u0001+\u0001+\u0001+\u0001+\u0001,\u0001,\u0001,\u0001"+
+ ",\u0003,\u01df\b,\u0001-\u0001-\u0001-\u0001-\u0003-\u01e5\b-\u0001-\u0001"+
+ "-\u0001-\u0001-\u0005-\u01eb\b-\n-\f-\u01ee\t-\u0003-\u01f0\b-\u0001."+
+ "\u0001.\u0001.\u0003.\u01f5\b.\u0001.\u0001.\u0001.\u0000\u0003\u0002"+
+ "\n\u0010/\u0000\u0002\u0004\u0006\b\n\f\u000e\u0010\u0012\u0014\u0016"+
+ "\u0018\u001a\u001c\u001e \"$&(*,.02468:<>@BDFHJLNPRTVXZ\\\u0000\b\u0001"+
+ "\u0000<=\u0001\u0000>@\u0001\u0000LM\u0001\u0000CD\u0002\u0000 ##\u0001"+
+ "\u0000&\'\u0002\u0000%%33\u0001\u00006;\u0217\u0000^\u0001\u0000\u0000"+
+ "\u0000\u0002a\u0001\u0000\u0000\u0000\u0004p\u0001\u0000\u0000\u0000\u0006"+
+ "\u007f\u0001\u0000\u0000\u0000\b\u0081\u0001\u0000\u0000\u0000\n\u00a0"+
+ "\u0001\u0000\u0000\u0000\f\u00bb\u0001\u0000\u0000\u0000\u000e\u00c2\u0001"+
+ "\u0000\u0000\u0000\u0010\u00c8\u0001\u0000\u0000\u0000\u0012\u00dc\u0001"+
+ "\u0000\u0000\u0000\u0014\u00de\u0001\u0000\u0000\u0000\u0016\u00ed\u0001"+
+ "\u0000\u0000\u0000\u0018\u00f0\u0001\u0000\u0000\u0000\u001a\u00fd\u0001"+
+ "\u0000\u0000\u0000\u001c\u00ff\u0001\u0000\u0000\u0000\u001e\u010b\u0001"+
+ "\u0000\u0000\u0000 \u0117\u0001\u0000\u0000\u0000\"\u011a\u0001\u0000"+
+ "\u0000\u0000$\u0122\u0001\u0000\u0000\u0000&\u0128\u0001\u0000\u0000\u0000"+
+ "(\u0130\u0001\u0000\u0000\u0000*\u0132\u0001\u0000\u0000\u0000,\u013a"+
+ "\u0001\u0000\u0000\u0000.\u0166\u0001\u0000\u0000\u00000\u0168\u0001\u0000"+
+ "\u0000\u00002\u016b\u0001\u0000\u0000\u00004\u0174\u0001\u0000\u0000\u0000"+
+ "6\u018e\u0001\u0000\u0000\u00008\u0190\u0001\u0000\u0000\u0000:\u0199"+
+ "\u0001\u0000\u0000\u0000<\u01a2\u0001\u0000\u0000\u0000>\u01a6\u0001\u0000"+
+ "\u0000\u0000@\u01ac\u0001\u0000\u0000\u0000B\u01b0\u0001\u0000\u0000\u0000"+
+ "D\u01b3\u0001\u0000\u0000\u0000F\u01bb\u0001\u0000\u0000\u0000H\u01bf"+
+ "\u0001\u0000\u0000\u0000J\u01c3\u0001\u0000\u0000\u0000L\u01c6\u0001\u0000"+
+ "\u0000\u0000N\u01cb\u0001\u0000\u0000\u0000P\u01cf\u0001\u0000\u0000\u0000"+
+ "R\u01d1\u0001\u0000\u0000\u0000T\u01d3\u0001\u0000\u0000\u0000V\u01d6"+
+ "\u0001\u0000\u0000\u0000X\u01de\u0001\u0000\u0000\u0000Z\u01e0\u0001\u0000"+
+ "\u0000\u0000\\\u01f4\u0001\u0000\u0000\u0000^_\u0003\u0002\u0001\u0000"+
+ "_`\u0005\u0000\u0000\u0001`\u0001\u0001\u0000\u0000\u0000ab\u0006\u0001"+
+ "\uffff\uffff\u0000bc\u0003\u0004\u0002\u0000ci\u0001\u0000\u0000\u0000"+
+ "de\n\u0001\u0000\u0000ef\u0005\u001a\u0000\u0000fh\u0003\u0006\u0003\u0000"+
+ "gd\u0001\u0000\u0000\u0000hk\u0001\u0000\u0000\u0000ig\u0001\u0000\u0000"+
+ "\u0000ij\u0001\u0000\u0000\u0000j\u0003\u0001\u0000\u0000\u0000ki\u0001"+
+ "\u0000\u0000\u0000lq\u0003T*\u0000mq\u0003\u001c\u000e\u0000nq\u0003\u0016"+
+ "\u000b\u0000oq\u0003X,\u0000pl\u0001\u0000\u0000\u0000pm\u0001\u0000\u0000"+
+ "\u0000pn\u0001\u0000\u0000\u0000po\u0001\u0000\u0000\u0000q\u0005\u0001"+
+ "\u0000\u0000\u0000r\u0080\u0003 \u0010\u0000s\u0080\u0003$\u0012\u0000"+
+ "t\u0080\u00030\u0018\u0000u\u0080\u00036\u001b\u0000v\u0080\u00032\u0019"+
+ "\u0000w\u0080\u0003\"\u0011\u0000x\u0080\u0003\b\u0004\u0000y\u0080\u0003"+
+ "8\u001c\u0000z\u0080\u0003:\u001d\u0000{\u0080\u0003>\u001f\u0000|\u0080"+
+ "\u0003@ \u0000}\u0080\u0003Z-\u0000~\u0080\u0003B!\u0000\u007fr\u0001"+
+ "\u0000\u0000\u0000\u007fs\u0001\u0000\u0000\u0000\u007ft\u0001\u0000\u0000"+
+ "\u0000\u007fu\u0001\u0000\u0000\u0000\u007fv\u0001\u0000\u0000\u0000\u007f"+
+ "w\u0001\u0000\u0000\u0000\u007fx\u0001\u0000\u0000\u0000\u007fy\u0001"+
+ "\u0000\u0000\u0000\u007fz\u0001\u0000\u0000\u0000\u007f{\u0001\u0000\u0000"+
+ "\u0000\u007f|\u0001\u0000\u0000\u0000\u007f}\u0001\u0000\u0000\u0000\u007f"+
+ "~\u0001\u0000\u0000\u0000\u0080\u0007\u0001\u0000\u0000\u0000\u0081\u0082"+
+ "\u0005\u0012\u0000\u0000\u0082\u0083\u0003\n\u0005\u0000\u0083\t\u0001"+
+ "\u0000\u0000\u0000\u0084\u0085\u0006\u0005\uffff\uffff\u0000\u0085\u0086"+
+ "\u0005,\u0000\u0000\u0086\u00a1\u0003\n\u0005\u0007\u0087\u00a1\u0003"+
+ "\u000e\u0007\u0000\u0088\u00a1\u0003\f\u0006\u0000\u0089\u008b\u0003\u000e"+
+ "\u0007\u0000\u008a\u008c\u0005,\u0000\u0000\u008b\u008a\u0001\u0000\u0000"+
+ "\u0000\u008b\u008c\u0001\u0000\u0000\u0000\u008c\u008d\u0001\u0000\u0000"+
+ "\u0000\u008d\u008e\u0005)\u0000\u0000\u008e\u008f\u0005(\u0000\u0000\u008f"+
+ "\u0094\u0003\u000e\u0007\u0000\u0090\u0091\u0005\"\u0000\u0000\u0091\u0093"+
+ "\u0003\u000e\u0007\u0000\u0092\u0090\u0001\u0000\u0000\u0000\u0093\u0096"+
+ "\u0001\u0000\u0000\u0000\u0094\u0092\u0001\u0000\u0000\u0000\u0094\u0095"+
+ "\u0001\u0000\u0000\u0000\u0095\u0097\u0001\u0000\u0000\u0000\u0096\u0094"+
+ "\u0001\u0000\u0000\u0000\u0097\u0098\u00052\u0000\u0000\u0098\u00a1\u0001"+
+ "\u0000\u0000\u0000\u0099\u009a\u0003\u000e\u0007\u0000\u009a\u009c\u0005"+
+ "*\u0000\u0000\u009b\u009d\u0005,\u0000\u0000\u009c\u009b\u0001\u0000\u0000"+
+ "\u0000\u009c\u009d\u0001\u0000\u0000\u0000\u009d\u009e\u0001\u0000\u0000"+
+ "\u0000\u009e\u009f\u0005-\u0000\u0000\u009f\u00a1\u0001\u0000\u0000\u0000"+
+ "\u00a0\u0084\u0001\u0000\u0000\u0000\u00a0\u0087\u0001\u0000\u0000\u0000"+
+ "\u00a0\u0088\u0001\u0000\u0000\u0000\u00a0\u0089\u0001\u0000\u0000\u0000"+
+ "\u00a0\u0099\u0001\u0000\u0000\u0000\u00a1\u00aa\u0001\u0000\u0000\u0000"+
+ "\u00a2\u00a3\n\u0004\u0000\u0000\u00a3\u00a4\u0005\u001f\u0000\u0000\u00a4"+
+ "\u00a9\u0003\n\u0005\u0005\u00a5\u00a6\n\u0003\u0000\u0000\u00a6\u00a7"+
+ "\u0005/\u0000\u0000\u00a7\u00a9\u0003\n\u0005\u0004\u00a8\u00a2\u0001"+
+ "\u0000\u0000\u0000\u00a8\u00a5\u0001\u0000\u0000\u0000\u00a9\u00ac\u0001"+
+ "\u0000\u0000\u0000\u00aa\u00a8\u0001\u0000\u0000\u0000\u00aa\u00ab\u0001"+
+ "\u0000\u0000\u0000\u00ab\u000b\u0001\u0000\u0000\u0000\u00ac\u00aa\u0001"+
+ "\u0000\u0000\u0000\u00ad\u00af\u0003\u000e\u0007\u0000\u00ae\u00b0\u0005"+
+ ",\u0000\u0000\u00af\u00ae\u0001\u0000\u0000\u0000\u00af\u00b0\u0001\u0000"+
+ "\u0000\u0000\u00b0\u00b1\u0001\u0000\u0000\u0000\u00b1\u00b2\u0005+\u0000"+
+ "\u0000\u00b2\u00b3\u0003P(\u0000\u00b3\u00bc\u0001\u0000\u0000\u0000\u00b4"+
+ "\u00b6\u0003\u000e\u0007\u0000\u00b5\u00b7\u0005,\u0000\u0000\u00b6\u00b5"+
+ "\u0001\u0000\u0000\u0000\u00b6\u00b7\u0001\u0000\u0000\u0000\u00b7\u00b8"+
+ "\u0001\u0000\u0000\u0000\u00b8\u00b9\u00051\u0000\u0000\u00b9\u00ba\u0003"+
+ "P(\u0000\u00ba\u00bc\u0001\u0000\u0000\u0000\u00bb\u00ad\u0001\u0000\u0000"+
+ "\u0000\u00bb\u00b4\u0001\u0000\u0000\u0000\u00bc\r\u0001\u0000\u0000\u0000"+
+ "\u00bd\u00c3\u0003\u0010\b\u0000\u00be\u00bf\u0003\u0010\b\u0000\u00bf"+
+ "\u00c0\u0003R)\u0000\u00c0\u00c1\u0003\u0010\b\u0000\u00c1\u00c3\u0001"+
+ "\u0000\u0000\u0000\u00c2\u00bd\u0001\u0000\u0000\u0000\u00c2\u00be\u0001"+
+ "\u0000\u0000\u0000\u00c3\u000f\u0001\u0000\u0000\u0000\u00c4\u00c5\u0006"+
+ "\b\uffff\uffff\u0000\u00c5\u00c9\u0003\u0012\t\u0000\u00c6\u00c7\u0007"+
+ "\u0000\u0000\u0000\u00c7\u00c9\u0003\u0010\b\u0003\u00c8\u00c4\u0001\u0000"+
+ "\u0000\u0000\u00c8\u00c6\u0001\u0000\u0000\u0000\u00c9\u00d2\u0001\u0000"+
+ "\u0000\u0000\u00ca\u00cb\n\u0002\u0000\u0000\u00cb\u00cc\u0007\u0001\u0000"+
+ "\u0000\u00cc\u00d1\u0003\u0010\b\u0003\u00cd\u00ce\n\u0001\u0000\u0000"+
+ "\u00ce\u00cf\u0007\u0000\u0000\u0000\u00cf\u00d1\u0003\u0010\b\u0002\u00d0"+
+ "\u00ca\u0001\u0000\u0000\u0000\u00d0\u00cd\u0001\u0000\u0000\u0000\u00d1"+
+ "\u00d4\u0001\u0000\u0000\u0000\u00d2\u00d0\u0001\u0000\u0000\u0000\u00d2"+
+ "\u00d3\u0001\u0000\u0000\u0000\u00d3\u0011\u0001\u0000\u0000\u0000\u00d4"+
+ "\u00d2\u0001\u0000\u0000\u0000\u00d5\u00dd\u0003.\u0017\u0000\u00d6\u00dd"+
+ "\u0003*\u0015\u0000\u00d7\u00dd\u0003\u0014\n\u0000\u00d8\u00d9\u0005"+
+ "(\u0000\u0000\u00d9\u00da\u0003\n\u0005\u0000\u00da\u00db\u00052\u0000"+
+ "\u0000\u00db\u00dd\u0001\u0000\u0000\u0000\u00dc\u00d5\u0001\u0000\u0000"+
+ "\u0000\u00dc\u00d6\u0001\u0000\u0000\u0000\u00dc\u00d7\u0001\u0000\u0000"+
+ "\u0000\u00dc\u00d8\u0001\u0000\u0000\u0000\u00dd\u0013\u0001\u0000\u0000"+
+ "\u0000\u00de\u00df\u0003,\u0016\u0000\u00df\u00e9\u0005(\u0000\u0000\u00e0"+
+ "\u00ea\u0005>\u0000\u0000\u00e1\u00e6\u0003\n\u0005\u0000\u00e2\u00e3"+
+ "\u0005\"\u0000\u0000\u00e3\u00e5\u0003\n\u0005\u0000\u00e4\u00e2\u0001"+
+ "\u0000\u0000\u0000\u00e5\u00e8\u0001\u0000\u0000\u0000\u00e6\u00e4\u0001"+
+ "\u0000\u0000\u0000\u00e6\u00e7\u0001\u0000\u0000\u0000\u00e7\u00ea\u0001"+
+ "\u0000\u0000\u0000\u00e8\u00e6\u0001\u0000\u0000\u0000\u00e9\u00e0\u0001"+
+ "\u0000\u0000\u0000\u00e9\u00e1\u0001\u0000\u0000\u0000\u00e9\u00ea\u0001"+
+ "\u0000\u0000\u0000\u00ea\u00eb\u0001\u0000\u0000\u0000\u00eb\u00ec\u0005"+
+ "2\u0000\u0000\u00ec\u0015\u0001\u0000\u0000\u0000\u00ed\u00ee\u0005\u000e"+
+ "\u0000\u0000\u00ee\u00ef\u0003\u0018\f\u0000\u00ef\u0017\u0001\u0000\u0000"+
+ "\u0000\u00f0\u00f5\u0003\u001a\r\u0000\u00f1\u00f2\u0005\"\u0000\u0000"+
+ "\u00f2\u00f4\u0003\u001a\r\u0000\u00f3\u00f1\u0001\u0000\u0000\u0000\u00f4"+
+ "\u00f7\u0001\u0000\u0000\u0000\u00f5\u00f3\u0001\u0000\u0000\u0000\u00f5"+
+ "\u00f6\u0001\u0000\u0000\u0000\u00f6\u0019\u0001\u0000\u0000\u0000\u00f7"+
+ "\u00f5\u0001\u0000\u0000\u0000\u00f8\u00fe\u0003\n\u0005\u0000\u00f9\u00fa"+
+ "\u0003*\u0015\u0000\u00fa\u00fb\u0005!\u0000\u0000\u00fb\u00fc\u0003\n"+
+ "\u0005\u0000\u00fc\u00fe\u0001\u0000\u0000\u0000\u00fd\u00f8\u0001\u0000"+
+ "\u0000\u0000\u00fd\u00f9\u0001\u0000\u0000\u0000\u00fe\u001b\u0001\u0000"+
+ "\u0000\u0000\u00ff\u0100\u0005\u0006\u0000\u0000\u0100\u0105\u0003(\u0014"+
+ "\u0000\u0101\u0102\u0005\"\u0000\u0000\u0102\u0104\u0003(\u0014\u0000"+
+ "\u0103\u0101\u0001\u0000\u0000\u0000\u0104\u0107\u0001\u0000\u0000\u0000"+
+ "\u0105\u0103\u0001\u0000\u0000\u0000\u0105\u0106\u0001\u0000\u0000\u0000"+
+ "\u0106\u0109\u0001\u0000\u0000\u0000\u0107\u0105\u0001\u0000\u0000\u0000"+
+ "\u0108\u010a\u0003\u001e\u000f\u0000\u0109\u0108\u0001\u0000\u0000\u0000"+
+ "\u0109\u010a\u0001\u0000\u0000\u0000\u010a\u001d\u0001\u0000\u0000\u0000"+
+ "\u010b\u010c\u0005A\u0000\u0000\u010c\u010d\u0005I\u0000\u0000\u010d\u0112"+
+ "\u0003(\u0014\u0000\u010e\u010f\u0005\"\u0000\u0000\u010f\u0111\u0003"+
+ "(\u0014\u0000\u0110\u010e\u0001\u0000\u0000\u0000\u0111\u0114\u0001\u0000"+
+ "\u0000\u0000\u0112\u0110\u0001\u0000\u0000\u0000\u0112\u0113\u0001\u0000"+
+ "\u0000\u0000\u0113\u0115\u0001\u0000\u0000\u0000\u0114\u0112\u0001\u0000"+
+ "\u0000\u0000\u0115\u0116\u0005B\u0000\u0000\u0116\u001f\u0001\u0000\u0000"+
+ "\u0000\u0117\u0118\u0005\u0004\u0000\u0000\u0118\u0119\u0003\u0018\f\u0000"+
+ "\u0119!\u0001\u0000\u0000\u0000\u011a\u011c\u0005\u0011\u0000\u0000\u011b"+
+ "\u011d\u0003\u0018\f\u0000\u011c\u011b\u0001\u0000\u0000\u0000\u011c\u011d"+
+ "\u0001\u0000\u0000\u0000\u011d\u0120\u0001\u0000\u0000\u0000\u011e\u011f"+
+ "\u0005\u001e\u0000\u0000\u011f\u0121\u0003&\u0013\u0000\u0120\u011e\u0001"+
+ "\u0000\u0000\u0000\u0120\u0121\u0001\u0000\u0000\u0000\u0121#\u0001\u0000"+
+ "\u0000\u0000\u0122\u0123\u0005\b\u0000\u0000\u0123\u0126\u0003\u0018\f"+
+ "\u0000\u0124\u0125\u0005\u001e\u0000\u0000\u0125\u0127\u0003&\u0013\u0000"+
+ "\u0126\u0124\u0001\u0000\u0000\u0000\u0126\u0127\u0001\u0000\u0000\u0000"+
+ "\u0127%\u0001\u0000\u0000\u0000\u0128\u012d\u0003*\u0015\u0000\u0129\u012a"+
+ "\u0005\"\u0000\u0000\u012a\u012c\u0003*\u0015\u0000\u012b\u0129\u0001"+
+ "\u0000\u0000\u0000\u012c\u012f\u0001\u0000\u0000\u0000\u012d\u012b\u0001"+
+ "\u0000\u0000\u0000\u012d\u012e\u0001\u0000\u0000\u0000\u012e\'\u0001\u0000"+
+ "\u0000\u0000\u012f\u012d\u0001\u0000\u0000\u0000\u0130\u0131\u0007\u0002"+
+ "\u0000\u0000\u0131)\u0001\u0000\u0000\u0000\u0132\u0137\u0003,\u0016\u0000"+
+ "\u0133\u0134\u0005$\u0000\u0000\u0134\u0136\u0003,\u0016\u0000\u0135\u0133"+
+ "\u0001\u0000\u0000\u0000\u0136\u0139\u0001\u0000\u0000\u0000\u0137\u0135"+
+ "\u0001\u0000\u0000\u0000\u0137\u0138\u0001\u0000\u0000\u0000\u0138+\u0001"+
+ "\u0000\u0000\u0000\u0139\u0137\u0001\u0000\u0000\u0000\u013a\u013b\u0007"+
+ "\u0003\u0000\u0000\u013b-\u0001\u0000\u0000\u0000\u013c\u0167\u0005-\u0000"+
+ "\u0000\u013d\u013e\u0003N\'\u0000\u013e\u013f\u0005C\u0000\u0000\u013f"+
+ "\u0167\u0001\u0000\u0000\u0000\u0140\u0167\u0003L&\u0000\u0141\u0167\u0003"+
+ "N\'\u0000\u0142\u0167\u0003H$\u0000\u0143\u0167\u00050\u0000\u0000\u0144"+
+ "\u0167\u0003P(\u0000\u0145\u0146\u0005A\u0000\u0000\u0146\u014b\u0003"+
+ "J%\u0000\u0147\u0148\u0005\"\u0000\u0000\u0148\u014a\u0003J%\u0000\u0149"+
+ "\u0147\u0001\u0000\u0000\u0000\u014a\u014d\u0001\u0000\u0000\u0000\u014b"+
+ "\u0149\u0001\u0000\u0000\u0000\u014b\u014c\u0001\u0000\u0000\u0000\u014c"+
+ "\u014e\u0001\u0000\u0000\u0000\u014d\u014b\u0001\u0000\u0000\u0000\u014e"+
+ "\u014f\u0005B\u0000\u0000\u014f\u0167\u0001\u0000\u0000\u0000\u0150\u0151"+
+ "\u0005A\u0000\u0000\u0151\u0156\u0003H$\u0000\u0152\u0153\u0005\"\u0000"+
+ "\u0000\u0153\u0155\u0003H$\u0000\u0154\u0152\u0001\u0000\u0000\u0000\u0155"+
+ "\u0158\u0001\u0000\u0000\u0000\u0156\u0154\u0001\u0000\u0000\u0000\u0156"+
+ "\u0157\u0001\u0000\u0000\u0000\u0157\u0159\u0001\u0000\u0000\u0000\u0158"+
+ "\u0156\u0001\u0000\u0000\u0000\u0159\u015a\u0005B\u0000\u0000\u015a\u0167"+
+ "\u0001\u0000\u0000\u0000\u015b\u015c\u0005A\u0000\u0000\u015c\u0161\u0003"+
+ "P(\u0000\u015d\u015e\u0005\"\u0000\u0000\u015e\u0160\u0003P(\u0000\u015f"+
+ "\u015d\u0001\u0000\u0000\u0000\u0160\u0163\u0001\u0000\u0000\u0000\u0161"+
+ "\u015f\u0001\u0000\u0000\u0000\u0161\u0162\u0001\u0000\u0000\u0000\u0162"+
+ "\u0164\u0001\u0000\u0000\u0000\u0163\u0161\u0001\u0000\u0000\u0000\u0164"+
+ "\u0165\u0005B\u0000\u0000\u0165\u0167\u0001\u0000\u0000\u0000\u0166\u013c"+
+ "\u0001\u0000\u0000\u0000\u0166\u013d\u0001\u0000\u0000\u0000\u0166\u0140"+
+ "\u0001\u0000\u0000\u0000\u0166\u0141\u0001\u0000\u0000\u0000\u0166\u0142"+
+ "\u0001\u0000\u0000\u0000\u0166\u0143\u0001\u0000\u0000\u0000\u0166\u0144"+
+ "\u0001\u0000\u0000\u0000\u0166\u0145\u0001\u0000\u0000\u0000\u0166\u0150"+
+ "\u0001\u0000\u0000\u0000\u0166\u015b\u0001\u0000\u0000\u0000\u0167/\u0001"+
+ "\u0000\u0000\u0000\u0168\u0169\u0005\n\u0000\u0000\u0169\u016a\u0005\u001c"+
+ "\u0000\u0000\u016a1\u0001\u0000\u0000\u0000\u016b\u016c\u0005\u0010\u0000"+
+ "\u0000\u016c\u0171\u00034\u001a\u0000\u016d\u016e\u0005\"\u0000\u0000"+
+ "\u016e\u0170\u00034\u001a\u0000\u016f\u016d\u0001\u0000\u0000\u0000\u0170"+
+ "\u0173\u0001\u0000\u0000\u0000\u0171\u016f\u0001\u0000\u0000\u0000\u0171"+
+ "\u0172\u0001\u0000\u0000\u0000\u01723\u0001\u0000\u0000\u0000\u0173\u0171"+
+ "\u0001\u0000\u0000\u0000\u0174\u0176\u0003\n\u0005\u0000\u0175\u0177\u0007"+
+ "\u0004\u0000\u0000\u0176\u0175\u0001\u0000\u0000\u0000\u0176\u0177\u0001"+
+ "\u0000\u0000\u0000\u0177\u017a\u0001\u0000\u0000\u0000\u0178\u0179\u0005"+
+ ".\u0000\u0000\u0179\u017b\u0007\u0005\u0000\u0000\u017a\u0178\u0001\u0000"+
+ "\u0000\u0000\u017a\u017b\u0001\u0000\u0000\u0000\u017b5\u0001\u0000\u0000"+
+ "\u0000\u017c\u017d\u0005\t\u0000\u0000\u017d\u0182\u0003(\u0014\u0000"+
+ "\u017e\u017f\u0005\"\u0000\u0000\u017f\u0181\u0003(\u0014\u0000\u0180"+
+ "\u017e\u0001\u0000\u0000\u0000\u0181\u0184\u0001\u0000\u0000\u0000\u0182"+
+ "\u0180\u0001\u0000\u0000\u0000\u0182\u0183\u0001\u0000\u0000\u0000\u0183"+
+ "\u018f\u0001\u0000\u0000\u0000\u0184\u0182\u0001\u0000\u0000\u0000\u0185"+
+ "\u0186\u0005\f\u0000\u0000\u0186\u018b\u0003(\u0014\u0000\u0187\u0188"+
+ "\u0005\"\u0000\u0000\u0188\u018a\u0003(\u0014\u0000\u0189\u0187\u0001"+
+ "\u0000\u0000\u0000\u018a\u018d\u0001\u0000\u0000\u0000\u018b\u0189\u0001"+
+ "\u0000\u0000\u0000\u018b\u018c\u0001\u0000\u0000\u0000\u018c\u018f\u0001"+
+ "\u0000\u0000\u0000\u018d\u018b\u0001\u0000\u0000\u0000\u018e\u017c\u0001"+
+ "\u0000\u0000\u0000\u018e\u0185\u0001\u0000\u0000\u0000\u018f7\u0001\u0000"+
+ "\u0000\u0000\u0190\u0191\u0005\u0002\u0000\u0000\u0191\u0196\u0003(\u0014"+
+ "\u0000\u0192\u0193\u0005\"\u0000\u0000\u0193\u0195\u0003(\u0014\u0000"+
+ "\u0194\u0192\u0001\u0000\u0000\u0000\u0195\u0198\u0001\u0000\u0000\u0000"+
+ "\u0196\u0194\u0001\u0000\u0000\u0000\u0196\u0197\u0001\u0000\u0000\u0000"+
+ "\u01979\u0001\u0000\u0000\u0000\u0198\u0196\u0001\u0000\u0000\u0000\u0199"+
+ "\u019a\u0005\r\u0000\u0000\u019a\u019f\u0003<\u001e\u0000\u019b\u019c"+
+ "\u0005\"\u0000\u0000\u019c\u019e\u0003<\u001e\u0000\u019d\u019b\u0001"+
+ "\u0000\u0000\u0000\u019e\u01a1\u0001\u0000\u0000\u0000\u019f\u019d\u0001"+
+ "\u0000\u0000\u0000\u019f\u01a0\u0001\u0000\u0000\u0000\u01a0;\u0001\u0000"+
+ "\u0000\u0000\u01a1\u019f\u0001\u0000\u0000\u0000\u01a2\u01a3\u0003(\u0014"+
+ "\u0000\u01a3\u01a4\u0005H\u0000\u0000\u01a4\u01a5\u0003(\u0014\u0000\u01a5"+
+ "=\u0001\u0000\u0000\u0000\u01a6\u01a7\u0005\u0001\u0000\u0000\u01a7\u01a8"+
+ "\u0003\u0012\t\u0000\u01a8\u01aa\u0003P(\u0000\u01a9\u01ab\u0003D\"\u0000"+
+ "\u01aa\u01a9\u0001\u0000\u0000\u0000\u01aa\u01ab\u0001\u0000\u0000\u0000"+
+ "\u01ab?\u0001\u0000\u0000\u0000\u01ac\u01ad\u0005\u0007\u0000\u0000\u01ad"+
+ "\u01ae\u0003\u0012\t\u0000\u01ae\u01af\u0003P(\u0000\u01afA\u0001\u0000"+
+ "\u0000\u0000\u01b0\u01b1\u0005\u000b\u0000\u0000\u01b1\u01b2\u0003(\u0014"+
+ "\u0000\u01b2C\u0001\u0000\u0000\u0000\u01b3\u01b8\u0003F#\u0000\u01b4"+
+ "\u01b5\u0005\"\u0000\u0000\u01b5\u01b7\u0003F#\u0000\u01b6\u01b4\u0001"+
+ "\u0000\u0000\u0000\u01b7\u01ba\u0001\u0000\u0000\u0000\u01b8\u01b6\u0001"+
+ "\u0000\u0000\u0000\u01b8\u01b9\u0001\u0000\u0000\u0000\u01b9E\u0001\u0000"+
+ "\u0000\u0000\u01ba\u01b8\u0001\u0000\u0000\u0000\u01bb\u01bc\u0003,\u0016"+
+ "\u0000\u01bc\u01bd\u0005!\u0000\u0000\u01bd\u01be\u0003.\u0017\u0000\u01be"+
+ "G\u0001\u0000\u0000\u0000\u01bf\u01c0\u0007\u0006\u0000\u0000\u01c0I\u0001"+
+ "\u0000\u0000\u0000\u01c1\u01c4\u0003L&\u0000\u01c2\u01c4\u0003N\'\u0000"+
+ "\u01c3\u01c1\u0001\u0000\u0000\u0000\u01c3\u01c2\u0001\u0000\u0000\u0000"+
+ "\u01c4K\u0001\u0000\u0000\u0000\u01c5\u01c7\u0007\u0000\u0000\u0000\u01c6"+
+ "\u01c5\u0001\u0000\u0000\u0000\u01c6\u01c7\u0001\u0000\u0000\u0000\u01c7"+
+ "\u01c8\u0001\u0000\u0000\u0000\u01c8\u01c9\u0005\u001d\u0000\u0000\u01c9"+
+ "M\u0001\u0000\u0000\u0000\u01ca\u01cc\u0007\u0000\u0000\u0000\u01cb\u01ca"+
+ "\u0001\u0000\u0000\u0000\u01cb\u01cc\u0001\u0000\u0000\u0000\u01cc\u01cd"+
+ "\u0001\u0000\u0000\u0000\u01cd\u01ce\u0005\u001c\u0000\u0000\u01ceO\u0001"+
+ "\u0000\u0000\u0000\u01cf\u01d0\u0005\u001b\u0000\u0000\u01d0Q\u0001\u0000"+
+ "\u0000\u0000\u01d1\u01d2\u0007\u0007\u0000\u0000\u01d2S\u0001\u0000\u0000"+
+ "\u0000\u01d3\u01d4\u0005\u0005\u0000\u0000\u01d4\u01d5\u0003V+\u0000\u01d5"+
+ "U\u0001\u0000\u0000\u0000\u01d6\u01d7\u0005A\u0000\u0000\u01d7\u01d8\u0003"+
+ "\u0002\u0001\u0000\u01d8\u01d9\u0005B\u0000\u0000\u01d9W\u0001\u0000\u0000"+
+ "\u0000\u01da\u01db\u0005\u000f\u0000\u0000\u01db\u01df\u00054\u0000\u0000"+
+ "\u01dc\u01dd\u0005\u000f\u0000\u0000\u01dd\u01df\u00055\u0000\u0000\u01de"+
+ "\u01da\u0001\u0000\u0000\u0000\u01de\u01dc\u0001\u0000\u0000\u0000\u01df"+
+ "Y\u0001\u0000\u0000\u0000\u01e0\u01e1\u0005\u0003\u0000\u0000\u01e1\u01e4"+
+ "\u0003(\u0014\u0000\u01e2\u01e3\u0005J\u0000\u0000\u01e3\u01e5\u0003("+
+ "\u0014\u0000\u01e4\u01e2\u0001\u0000\u0000\u0000\u01e4\u01e5\u0001\u0000"+
+ "\u0000\u0000\u01e5\u01ef\u0001\u0000\u0000\u0000\u01e6\u01e7\u0005K\u0000"+
+ "\u0000\u01e7\u01ec\u0003\\.\u0000\u01e8\u01e9\u0005\"\u0000\u0000\u01e9"+
+ "\u01eb\u0003\\.\u0000\u01ea\u01e8\u0001\u0000\u0000\u0000\u01eb\u01ee"+
+ "\u0001\u0000\u0000\u0000\u01ec\u01ea\u0001\u0000\u0000\u0000\u01ec\u01ed"+
+ "\u0001\u0000\u0000\u0000\u01ed\u01f0\u0001\u0000\u0000\u0000\u01ee\u01ec"+
+ "\u0001\u0000\u0000\u0000\u01ef\u01e6\u0001\u0000\u0000\u0000\u01ef\u01f0"+
+ "\u0001\u0000\u0000\u0000\u01f0[\u0001\u0000\u0000\u0000\u01f1\u01f2\u0003"+
+ "(\u0014\u0000\u01f2\u01f3\u0005!\u0000\u0000\u01f3\u01f5\u0001\u0000\u0000"+
+ "\u0000\u01f4\u01f1\u0001\u0000\u0000\u0000\u01f4\u01f5\u0001\u0000\u0000"+
+ "\u0000\u01f5\u01f6\u0001\u0000\u0000\u0000\u01f6\u01f7\u0003(\u0014\u0000"+
+ "\u01f7]\u0001\u0000\u0000\u00003ip\u007f\u008b\u0094\u009c\u00a0\u00a8"+
+ "\u00aa\u00af\u00b6\u00bb\u00c2\u00c8\u00d0\u00d2\u00dc\u00e6\u00e9\u00f5"+
+ "\u00fd\u0105\u0109\u0112\u011c\u0120\u0126\u012d\u0137\u014b\u0156\u0161"+
+ "\u0166\u0171\u0176\u017a\u0182\u018b\u018e\u0196\u019f\u01aa\u01b8\u01c3"+
+ "\u01c6\u01cb\u01de\u01e4\u01ec\u01ef\u01f4";
public static final ATN _ATN =
new ATNDeserializer().deserialize(_serializedATN.toCharArray());
static {
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParserBaseListener.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParserBaseListener.java
index ceef1b4e681a7..3137eff0b6550 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParserBaseListener.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParserBaseListener.java
@@ -252,6 +252,18 @@ public class EsqlBaseParserBaseListener implements EsqlBaseParserListener {
* The default implementation does nothing.
*/
@Override public void exitDereference(EsqlBaseParser.DereferenceContext ctx) { }
+ /**
+ * {@inheritDoc}
+ *
+ * The default implementation does nothing.
+ */
+ @Override public void enterFunction(EsqlBaseParser.FunctionContext ctx) { }
+ /**
+ * {@inheritDoc}
+ *
+ * The default implementation does nothing.
+ */
+ @Override public void exitFunction(EsqlBaseParser.FunctionContext ctx) { }
/**
* {@inheritDoc}
*
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParserBaseVisitor.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParserBaseVisitor.java
index 48f5b33fcfec1..d7b2f359e3c83 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParserBaseVisitor.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParserBaseVisitor.java
@@ -152,6 +152,13 @@ public class EsqlBaseParserBaseVisitor extends AbstractParseTreeVisitor im
* {@link #visitChildren} on {@code ctx}.
*/
@Override public T visitDereference(EsqlBaseParser.DereferenceContext ctx) { return visitChildren(ctx); }
+ /**
+ * {@inheritDoc}
+ *
+ * The default implementation returns the result of calling
+ * {@link #visitChildren} on {@code ctx}.
+ */
+ @Override public T visitFunction(EsqlBaseParser.FunctionContext ctx) { return visitChildren(ctx); }
/**
* {@inheritDoc}
*
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParserListener.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParserListener.java
index 04f0d6da3dbe4..dd6cdaacddbef 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParserListener.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParserListener.java
@@ -237,6 +237,18 @@ public interface EsqlBaseParserListener extends ParseTreeListener {
* @param ctx the parse tree
*/
void exitDereference(EsqlBaseParser.DereferenceContext ctx);
+ /**
+ * Enter a parse tree produced by the {@code function}
+ * labeled alternative in {@link EsqlBaseParser#primaryExpression}.
+ * @param ctx the parse tree
+ */
+ void enterFunction(EsqlBaseParser.FunctionContext ctx);
+ /**
+ * Exit a parse tree produced by the {@code function}
+ * labeled alternative in {@link EsqlBaseParser#primaryExpression}.
+ * @param ctx the parse tree
+ */
+ void exitFunction(EsqlBaseParser.FunctionContext ctx);
/**
* Enter a parse tree produced by the {@code parenthesizedExpression}
* labeled alternative in {@link EsqlBaseParser#primaryExpression}.
@@ -250,14 +262,12 @@ public interface EsqlBaseParserListener extends ParseTreeListener {
*/
void exitParenthesizedExpression(EsqlBaseParser.ParenthesizedExpressionContext ctx);
/**
- * Enter a parse tree produced by the {@code functionExpression}
- * labeled alternative in {@link EsqlBaseParser#primaryExpression}.
+ * Enter a parse tree produced by {@link EsqlBaseParser#functionExpression}.
* @param ctx the parse tree
*/
void enterFunctionExpression(EsqlBaseParser.FunctionExpressionContext ctx);
/**
- * Exit a parse tree produced by the {@code functionExpression}
- * labeled alternative in {@link EsqlBaseParser#primaryExpression}.
+ * Exit a parse tree produced by {@link EsqlBaseParser#functionExpression}.
* @param ctx the parse tree
*/
void exitFunctionExpression(EsqlBaseParser.FunctionExpressionContext ctx);
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParserVisitor.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParserVisitor.java
index 681de2590d575..35297f3d4f336 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParserVisitor.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParserVisitor.java
@@ -145,6 +145,13 @@ public interface EsqlBaseParserVisitor extends ParseTreeVisitor {
* @return the visitor result
*/
T visitDereference(EsqlBaseParser.DereferenceContext ctx);
+ /**
+ * Visit a parse tree produced by the {@code function}
+ * labeled alternative in {@link EsqlBaseParser#primaryExpression}.
+ * @param ctx the parse tree
+ * @return the visitor result
+ */
+ T visitFunction(EsqlBaseParser.FunctionContext ctx);
/**
* Visit a parse tree produced by the {@code parenthesizedExpression}
* labeled alternative in {@link EsqlBaseParser#primaryExpression}.
@@ -153,8 +160,7 @@ public interface EsqlBaseParserVisitor extends ParseTreeVisitor {
*/
T visitParenthesizedExpression(EsqlBaseParser.ParenthesizedExpressionContext ctx);
/**
- * Visit a parse tree produced by the {@code functionExpression}
- * labeled alternative in {@link EsqlBaseParser#primaryExpression}.
+ * Visit a parse tree produced by {@link EsqlBaseParser#functionExpression}.
* @param ctx the parse tree
* @return the visitor result
*/
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/ExpressionBuilder.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/ExpressionBuilder.java
index bbffa6c63e9b1..2113dc158203f 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/ExpressionBuilder.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/ExpressionBuilder.java
@@ -20,6 +20,7 @@
import org.elasticsearch.xpack.esql.evaluator.predicate.operator.regex.RLike;
import org.elasticsearch.xpack.esql.evaluator.predicate.operator.regex.WildcardLike;
import org.elasticsearch.xpack.esql.expression.Order;
+import org.elasticsearch.xpack.esql.expression.function.EsqlFunctionRegistry;
import org.elasticsearch.xpack.esql.expression.predicate.operator.arithmetic.Add;
import org.elasticsearch.xpack.esql.expression.predicate.operator.arithmetic.Div;
import org.elasticsearch.xpack.esql.expression.predicate.operator.arithmetic.Mod;
@@ -62,6 +63,7 @@
import java.util.function.BiFunction;
import static java.util.Collections.emptyList;
+import static java.util.Collections.singletonList;
import static org.elasticsearch.xpack.esql.type.EsqlDataTypes.DATE_PERIOD;
import static org.elasticsearch.xpack.esql.type.EsqlDataTypes.TIME_DURATION;
import static org.elasticsearch.xpack.ql.parser.ParserUtils.source;
@@ -312,12 +314,15 @@ public UnresolvedAttribute visitDereference(EsqlBaseParser.DereferenceContext ct
@Override
public Expression visitFunctionExpression(EsqlBaseParser.FunctionExpressionContext ctx) {
- return new UnresolvedFunction(
- source(ctx),
- visitIdentifier(ctx.identifier()),
- FunctionResolutionStrategy.DEFAULT,
- ctx.booleanExpression().stream().map(this::expression).toList()
- );
+ String name = visitIdentifier(ctx.identifier());
+ List args = expressions(ctx.booleanExpression());
+ if ("count".equals(EsqlFunctionRegistry.normalizeName(name))) {
+ // to simplify the registration, handle in the parser the special count cases
+ if (args.isEmpty() || ctx.ASTERISK() != null) {
+ args = singletonList(new Literal(source(ctx), 1, DataTypes.INTEGER));
+ }
+ }
+ return new UnresolvedFunction(source(ctx), name, FunctionResolutionStrategy.DEFAULT, args);
}
@Override
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/LocalExecutionPlanner.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/LocalExecutionPlanner.java
index cd653c64213c3..d93f1032209d8 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/LocalExecutionPlanner.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/LocalExecutionPlanner.java
@@ -48,6 +48,7 @@
import org.elasticsearch.xpack.esql.enrich.EnrichLookupService;
import org.elasticsearch.xpack.esql.evaluator.EvalMapper;
import org.elasticsearch.xpack.esql.evaluator.command.GrokEvaluatorExtracter;
+import org.elasticsearch.xpack.esql.expression.function.aggregate.Count;
import org.elasticsearch.xpack.esql.plan.physical.AggregateExec;
import org.elasticsearch.xpack.esql.plan.physical.DissectExec;
import org.elasticsearch.xpack.esql.plan.physical.EnrichExec;
@@ -73,6 +74,7 @@
import org.elasticsearch.xpack.ql.expression.Attribute;
import org.elasticsearch.xpack.ql.expression.Expression;
import org.elasticsearch.xpack.ql.expression.Expressions;
+import org.elasticsearch.xpack.ql.expression.FieldAttribute;
import org.elasticsearch.xpack.ql.expression.Literal;
import org.elasticsearch.xpack.ql.expression.NameId;
import org.elasticsearch.xpack.ql.expression.NamedExpression;
@@ -104,6 +106,8 @@
*/
public class LocalExecutionPlanner {
+ private static Object COUNT_ALL_MARKER = new Object();
+
private final String sessionId;
private final CancellableTask parentTask;
private final BigArrays bigArrays;
@@ -206,8 +210,50 @@ else if (node instanceof OutputExec outputExec) {
}
private PhysicalOperation planAggregation(AggregateExec aggregate, LocalExecutionPlannerContext context) {
- var source = plan(aggregate.child(), context);
- return physicalOperationProviders.groupingPhysicalOperation(aggregate, source, context);
+ var child = aggregate.child();
+ PhysicalOperation operation;
+ // optimize Lucene count
+ if (child instanceof EsQueryExec queryExec) {
+ var target = nonGroupedCountTarget(aggregate);
+ if (target != null) {
+ return luceneCount(context, target instanceof String ? target.toString() : null);
+ }
+ }
+
+ var source = plan(child, context);
+ operation = physicalOperationProviders.groupingPhysicalOperation(aggregate, source, context);
+ return operation;
+ }
+
+ /**
+ * Returns the target count - null if it doesn't apply, Boolean.TRUE for count all or a String representing the underlying field name.
+ *
+ * TODO: need to handle runtime fields
+ */
+ private Object nonGroupedCountTarget(AggregateExec aggregate) {
+ if (aggregate.groupings().isEmpty()) {
+ var aggs = aggregate.aggregates();
+ if (aggs.size() == 1) {
+ var agg = aggs.get(0);
+ if (agg instanceof Alias as && as.child() instanceof Count count) {
+ var expression = count.field();
+ // if the count points to an actual field, push it down
+ if (expression instanceof FieldAttribute fa) {
+ return fa.name();
+ }
+ // literal means count all
+ if (expression.foldable()) {
+ return Boolean.TRUE;
+ }
+ }
+ }
+ }
+ // count cannot be pushed
+ return null;
+ }
+
+ private PhysicalOperation luceneCount(LocalExecutionPlannerContext context, String fieldName) {
+ throw new UnsupportedOperationException();
}
private PhysicalOperation planEsQueryNode(EsQueryExec esQuery, LocalExecutionPlannerContext context) {
diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizerTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizerTests.java
index 746a34eaedce4..568774e190803 100644
--- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizerTests.java
+++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizerTests.java
@@ -1837,6 +1837,47 @@ public void testAvgSurrogateFunctionAfterRenameAndLimit() {
assertThat(source.limit().fold(), equalTo(10));
}
+ public void testCountOneFieldWithFilter() {
+ var plan = optimizedPlan(physicalPlan("""
+ from test
+ | where salary > 1000
+ | stats c = count(salary)
+ """));
+ }
+
+ public void testCountOneFieldWithFilterAndLimit() {
+ var plan = optimizedPlan(physicalPlan("""
+ from test
+ | where salary > 1000
+ | limit 10
+ | stats c = count(salary)
+ """));
+ }
+
+ public void testCountMultipleFieldsWithFilter() {
+ var plan = optimizedPlan(physicalPlan("""
+ from test
+ | where salary > 1000 and emp_no > 10010
+ | stats cs = count(salary), ce = count(emp_no)
+ """));
+ }
+
+ public void testCountAllWithFilter() {
+ var plan = optimizedPlan(physicalPlan("""
+ from test
+ | where emp_no > 10010
+ | stats c = count(), call = count(*), c_literal = count(1)
+ """));
+ }
+
+ public void testCountFieldsAndAllWithFilter() {
+ var plan = optimizedPlan(physicalPlan("""
+ from test
+ | where emp_no > 10010
+ | stats c = count(), cs = count(salary), ce = count(emp_no)
+ """));
+ }
+
private static EsQueryExec source(PhysicalPlan plan) {
if (plan instanceof ExchangeExec exchange) {
plan = exchange.child();